Split PtraceMonitor out of Monitor

This is a preparatory step to introduce a Sandbox2 mode that does not use ptrace.

PiperOrigin-RevId: 503919613
Change-Id: I446adecc66e697c592ad938627fbfdbea12516e1
pull/171/head
Wiktor Garbacz 2023-01-23 01:41:42 -08:00 committed by Copybara-Service
parent 93ef7eb380
commit f636cd86d6
8 changed files with 370 additions and 292 deletions

View File

@ -101,7 +101,8 @@ class Executor final {
}
private:
friend class Monitor;
friend class MonitorBase;
friend class PtraceMonitor;
friend class StackTracePeer;
// Internal constructor for executing libunwind on the given pid

View File

@ -62,7 +62,7 @@ class IPC final {
private:
friend class Executor;
friend class Monitor;
friend class MonitorBase;
friend class IpcPeer; // For testing
// Uses a pre-connected file descriptor.

View File

@ -248,20 +248,17 @@ void MaybeEnableTomoyoLsmWorkaround(Mounts& mounts, std::string& comms_fd_dev) {
} // namespace
Monitor::Monitor(Executor* executor, Policy* policy, Notify* notify)
MonitorBase::MonitorBase(Executor* executor, Policy* policy, Notify* notify)
: executor_(executor),
notify_(notify),
policy_(policy),
// NOLINTNEXTLINE clang-diagnostic-deprecated-declarations
comms_(executor_->ipc()->comms()),
ipc_(executor_->ipc()),
wait_for_execve_(executor->enable_sandboxing_pre_execve_) {
ipc_(executor_->ipc()) {
// It's a pre-connected Comms channel, no need to accept new connection.
CHECK(comms_->IsConnected());
std::string path =
absl::GetFlag(FLAGS_sandbox2_danger_danger_permit_all_and_log);
external_kill_request_flag_.test_and_set(std::memory_order_relaxed);
dump_stack_request_flag_.test_and_set(std::memory_order_relaxed);
if (!path.empty()) {
log_file_ = std::fopen(path.c_str(), "a+");
PCHECK(log_file_ != nullptr) << "Failed to open log file '" << path << "'";
@ -274,7 +271,21 @@ Monitor::Monitor(Executor* executor, Policy* policy, Notify* notify)
}
}
Monitor::~Monitor() {
PtraceMonitor::PtraceMonitor(Executor* executor, Policy* policy, Notify* notify)
: MonitorBase(executor, policy, notify),
wait_for_execve_(executor->enable_sandboxing_pre_execve_),
uses_custom_forkserver_(executor_->fork_client_ != nullptr) {
if (executor_->limits()->wall_time_limit() != absl::ZeroDuration()) {
auto deadline = absl::Now() + executor_->limits()->wall_time_limit();
LOG(INFO) << executor_->limits()->wall_time_limit();
deadline_millis_.store(absl::ToUnixMillis(deadline),
std::memory_order_relaxed);
}
external_kill_request_flag_.test_and_set(std::memory_order_relaxed);
dump_stack_request_flag_.test_and_set(std::memory_order_relaxed);
}
MonitorBase::~MonitorBase() {
if (!comms_fd_dev_.empty()) {
std::remove(comms_fd_dev_.c_str());
}
@ -297,30 +308,27 @@ void LogContainer(const std::vector<std::string>& container) {
} // namespace
void Monitor::Run() {
absl::Cleanup setup_notify = [this] { setup_notification_.Notify(); };
absl::Cleanup monitor_cleanup = [this] {
getrusage(RUSAGE_THREAD, result_.GetRUsageMonitor());
notify_->EventFinished(result_);
ipc_->InternalCleanupFdMap();
done_notification_.Notify();
};
if (executor_->limits()->wall_time_limit() != absl::ZeroDuration()) {
auto deadline = absl::Now() + executor_->limits()->wall_time_limit();
deadline_millis_.store(absl::ToUnixMillis(deadline),
std::memory_order_relaxed);
}
// It'd be costly to initialize the sigset_t for each sigtimedwait()
// invocation, so do it once per Monitor.
sigset_t sigtimedwait_sset;
if (!InitSetupSignals(&sigtimedwait_sset)) {
SetExitStatusCode(Result::SETUP_ERROR, Result::FAILED_SIGNALS);
void MonitorBase::OnDone() {
if (done_notification_.HasBeenNotified()) {
return;
}
notify_->EventFinished(result_);
ipc_->InternalCleanupFdMap();
done_notification_.Notify();
}
void MonitorBase::Launch() {
absl::Cleanup process_cleanup = [this] {
if (process_.init_pid > 0) {
kill(process_.init_pid, SIGKILL);
} else if (process_.main_pid > 0) {
kill(process_.main_pid, SIGKILL);
}
};
absl::Cleanup monitor_done = [this] { OnDone(); };
Namespace* ns = policy_->GetNamespace();
if (SAPI_VLOG_IS_ON(1) && ns != nullptr) {
std::vector<std::string> outside_entries;
@ -344,7 +352,6 @@ void Monitor::Run() {
// Get PID of the sandboxee.
bool should_have_init = ns && (ns->GetCloneFlags() & CLONE_NEWPID);
uses_custom_forkserver_ = executor_->fork_client_ != nullptr;
absl::StatusOr<Executor::Process> process =
executor_->StartSubProcess(clone_flags, ns, policy_->capabilities());
@ -354,24 +361,14 @@ void Monitor::Run() {
return;
}
pid_ = process->main_pid;
process_ = *std::move(process);
if (process->init_pid > 0) {
if (ptrace(PTRACE_SEIZE, process->init_pid, 0, PTRACE_O_EXITKILL) != 0) {
if (errno == ESRCH) {
SetExitStatusCode(Result::SETUP_ERROR, Result::FAILED_PTRACE);
return;
}
PLOG(FATAL) << "attaching to init process failed";
}
}
if (pid_ <= 0 || (should_have_init && process->init_pid <= 0)) {
if (process_.main_pid <= 0 || (should_have_init && process_.main_pid <= 0)) {
SetExitStatusCode(Result::SETUP_ERROR, Result::FAILED_SUBPROCESS);
return;
}
if (!notify_->EventStarted(pid_, comms_)) {
if (!notify_->EventStarted(process_.main_pid, comms_)) {
SetExitStatusCode(Result::SETUP_ERROR, Result::FAILED_NOTIFY);
return;
}
@ -395,35 +392,37 @@ void Monitor::Run() {
SetExitStatusCode(Result::SETUP_ERROR, Result::FAILED_LIMITS);
return;
}
// This call should be the last in the init sequence, because it can cause the
// sandboxee to enter ptrace-stopped state, in which it will not be able to
// send any messages over the Comms channel.
if (!InitPtraceAttach()) {
SetExitStatusCode(Result::SETUP_ERROR, Result::FAILED_PTRACE);
return;
}
std::move(process_cleanup).Cancel();
// Tell the parent thread (Sandbox2 object) that we're done with the initial
// set-up process of the sandboxee.
std::move(setup_notify).Invoke();
MainLoop(&sigtimedwait_sset);
RunInternal();
std::move(monitor_done).Cancel();
}
bool Monitor::IsActivelyMonitoring() {
absl::StatusOr<Result> MonitorBase::AwaitResultWithTimeout(
absl::Duration timeout) {
auto done = done_notification_.WaitForNotificationWithTimeout(timeout);
if (!done) {
return absl::DeadlineExceededError("Sandbox did not finish within timeout");
}
Join();
return result_;
}
bool PtraceMonitor::IsActivelyMonitoring() {
// If we're still waiting for execve(), then we allow all syscalls.
return !wait_for_execve_;
}
void Monitor::SetActivelyMonitoring() { wait_for_execve_ = false; }
void PtraceMonitor::SetActivelyMonitoring() { wait_for_execve_ = false; }
void Monitor::SetExitStatusCode(Result::StatusEnum final_status,
uintptr_t reason_code) {
void MonitorBase::SetExitStatusCode(Result::StatusEnum final_status,
uintptr_t reason_code) {
CHECK(result_.final_status() == Result::UNSET);
result_.SetExitStatusCode(final_status, reason_code);
}
bool Monitor::StackTraceCollectionPossible() {
bool PtraceMonitor::StackTraceCollectionPossible() const {
// Only get the stacktrace if we are not in the libunwind sandbox (avoid
// recursion).
if ((policy_->GetNamespace() ||
@ -438,7 +437,7 @@ bool Monitor::StackTraceCollectionPossible() {
}
}
bool Monitor::ShouldCollectStackTrace() {
bool PtraceMonitor::ShouldCollectStackTrace() const {
if (!StackTraceCollectionPossible()) {
return false;
}
@ -458,7 +457,7 @@ bool Monitor::ShouldCollectStackTrace() {
}
}
absl::StatusOr<std::vector<std::string>> Monitor::GetAndLogStackTrace(
absl::StatusOr<std::vector<std::string>> PtraceMonitor::GetAndLogStackTrace(
const Regs* regs) {
SAPI_ASSIGN_OR_RETURN(
std::vector<std::string> stack_trace,
@ -473,11 +472,11 @@ absl::StatusOr<std::vector<std::string>> Monitor::GetAndLogStackTrace(
return stack_trace;
}
void Monitor::SetAdditionalResultInfo(std::unique_ptr<Regs> regs) {
void PtraceMonitor::SetAdditionalResultInfo(std::unique_ptr<Regs> regs) {
pid_t pid = regs->pid();
result_.SetRegs(std::move(regs));
result_.SetProgName(util::GetProgName(pid));
result_.SetProcMaps(ReadProcMaps(pid_));
result_.SetProcMaps(ReadProcMaps(pid));
if (!ShouldCollectStackTrace()) {
VLOG(1) << "Stack traces have been disabled";
return;
@ -492,19 +491,19 @@ void Monitor::SetAdditionalResultInfo(std::unique_ptr<Regs> regs) {
result_.set_stack_trace(*stack_trace);
}
bool Monitor::KillSandboxee() {
VLOG(1) << "Sending SIGKILL to the PID: " << pid_;
if (kill(pid_, SIGKILL) != 0) {
PLOG(ERROR) << "Could not send SIGKILL to PID " << pid_;
bool PtraceMonitor::KillSandboxee() {
VLOG(1) << "Sending SIGKILL to the PID: " << process_.main_pid;
if (kill(process_.main_pid, SIGKILL) != 0) {
PLOG(ERROR) << "Could not send SIGKILL to PID " << process_.main_pid;
SetExitStatusCode(Result::INTERNAL_ERROR, Result::FAILED_KILL);
return false;
}
return true;
}
bool Monitor::InterruptSandboxee() {
if (ptrace(PTRACE_INTERRUPT, pid_, 0, 0) == -1) {
PLOG(ERROR) << "Could not send interrupt to pid=" << pid_;
bool PtraceMonitor::InterruptSandboxee() {
if (ptrace(PTRACE_INTERRUPT, process_.main_pid, 0, 0) == -1) {
PLOG(ERROR) << "Could not send interrupt to pid=" << process_.main_pid;
SetExitStatusCode(Result::INTERNAL_ERROR, Result::FAILED_INTERRUPT);
return false;
}
@ -514,13 +513,64 @@ bool Monitor::InterruptSandboxee() {
// Not defined in glibc.
#define __WPTRACEEVENT(x) ((x & 0xff0000) >> 16)
void Monitor::MainLoop(sigset_t* sset) {
void PtraceMonitor::NotifyMonitor() {
absl::ReaderMutexLock lock(&notify_mutex_);
if (thread_ != nullptr) {
pthread_kill(thread_->native_handle(), SIGCHLD);
}
}
void PtraceMonitor::Join() {
absl::MutexLock lock(&notify_mutex_);
if (thread_) {
thread_->join();
CHECK(IsDone()) << "Monitor did not terminate";
VLOG(1) << "Final execution status: " << result_.ToString();
CHECK(result_.final_status() != Result::UNSET);
thread_.reset();
}
}
void PtraceMonitor::RunInternal() {
thread_ = std::make_unique<std::thread>(&PtraceMonitor::Run, this);
// Wait for the Monitor to set-up the sandboxee correctly (or fail while
// doing that). From here on, it is safe to use the IPC object for
// non-sandbox-related data exchange.
setup_notification_.WaitForNotification();
}
void PtraceMonitor::Run() {
absl::Cleanup monitor_done = [this] {
getrusage(RUSAGE_THREAD, result_.GetRUsageMonitor());
OnDone();
};
absl::Cleanup setup_notify = [this] { setup_notification_.Notify(); };
// It'd be costly to initialize the sigset_t for each sigtimedwait()
// invocation, so do it once per Monitor.
if (!InitSetupSignals()) {
SetExitStatusCode(Result::SETUP_ERROR, Result::FAILED_SIGNALS);
return;
}
// This call should be the last in the init sequence, because it can cause the
// sandboxee to enter ptrace-stopped state, in which it will not be able to
// send any messages over the Comms channel.
if (!InitPtraceAttach()) {
SetExitStatusCode(Result::SETUP_ERROR, Result::FAILED_PTRACE);
return;
}
// Tell the parent thread (Sandbox2 object) that we're done with the initial
// set-up process of the sandboxee.
std::move(setup_notify).Invoke();
bool sandboxee_exited = false;
PidWaiter pid_waiter(pid_);
PidWaiter pid_waiter(process_.main_pid);
int status;
// All possible still running children of main process, will be killed due to
// PTRACE_O_EXITKILL ptrace() flag.
while (result_.final_status() == Result::UNSET) {
while (result().final_status() == Result::UNSET) {
int64_t deadline = deadline_millis_.load(std::memory_order_relaxed);
if (deadline != 0 && absl::Now() >= absl::FromUnixMillis(deadline)) {
VLOG(1) << "Sandbox process hit timeout due to the walltime timer";
@ -557,7 +607,7 @@ void Monitor::MainLoop(sigset_t* sset) {
pid_t ret = pid_waiter.Wait(&status);
if (ret == 0) {
constexpr timespec ts = {kWakeUpPeriodSec, kWakeUpPeriodNSec};
int signo = sigtimedwait(sset, nullptr, &ts);
int signo = sigtimedwait(&sset_, nullptr, &ts);
LOG_IF(ERROR, signo != -1 && signo != SIGCHLD)
<< "Unknown signal received: " << signo;
continue;
@ -582,7 +632,7 @@ void Monitor::MainLoop(sigset_t* sset) {
// That's the main process, set the exit code, and exit. It will kill
// all remaining processes (if there are any) because of the
// PTRACE_O_EXITKILL ptrace() flag.
if (ret == pid_) {
if (ret == process_.main_pid) {
if (IsActivelyMonitoring()) {
SetExitStatusCode(Result::OK, WEXITSTATUS(status));
} else {
@ -597,7 +647,7 @@ void Monitor::MainLoop(sigset_t* sset) {
// actual signal death. This may be changed in the future;
VLOG(1) << "PID: " << ret << " terminated with signal: "
<< util::GetSignalName(WTERMSIG(status));
if (ret == pid_) {
if (ret == process_.main_pid) {
if (network_violation_) {
SetExitStatusCode(Result::VIOLATION, Result::VIOLATION_NETWORK);
result_.SetNetworkViolation(network_proxy_server_->violation_msg_);
@ -627,7 +677,7 @@ void Monitor::MainLoop(sigset_t* sset) {
absl::GetFlag(FLAGS_sandbox2_log_all_stack_traces);
if (!log_stack_traces) {
// Try to make sure main pid is killed and reaped
kill(pid_, SIGKILL);
kill(process_.main_pid, SIGKILL);
}
constexpr auto kGracefullExitTimeout = absl::Milliseconds(200);
auto deadline = absl::Now() + kGracefullExitTimeout;
@ -648,14 +698,14 @@ void Monitor::MainLoop(sigset_t* sset) {
}
break;
}
if (!log_stack_traces && ret == pid_ &&
if (!log_stack_traces && ret == process_.main_pid &&
(WIFSIGNALED(status) || WIFEXITED(status))) {
break;
}
if (ret == 0) {
auto ts = absl::ToTimespec(left);
sigtimedwait(sset, nullptr, &ts);
sigtimedwait(&sset_, nullptr, &ts);
continue;
}
@ -672,13 +722,13 @@ void Monitor::MainLoop(sigset_t* sset) {
}
if (!log_stack_traces) {
kill(pid_, SIGKILL);
kill(process_.main_pid, SIGKILL);
}
}
}
}
void Monitor::LogStackTraceOfPid(pid_t pid) {
void PtraceMonitor::LogStackTraceOfPid(pid_t pid) {
if (!StackTraceCollectionPossible()) {
return;
}
@ -695,19 +745,19 @@ void Monitor::LogStackTraceOfPid(pid_t pid) {
}
}
bool Monitor::InitSetupSignals(sigset_t* sset) {
if (sigemptyset(sset) == -1) {
bool PtraceMonitor::InitSetupSignals() {
if (sigemptyset(&sset_) == -1) {
PLOG(ERROR) << "sigemptyset()";
return false;
}
// sigtimedwait will react (wake-up) to arrival of this signal.
if (sigaddset(sset, SIGCHLD) == -1) {
if (sigaddset(&sset_, SIGCHLD) == -1) {
PLOG(ERROR) << "sigaddset(SIGCHLD)";
return false;
}
if (pthread_sigmask(SIG_BLOCK, sset, nullptr) == -1) {
if (pthread_sigmask(SIG_BLOCK, &sset_, nullptr) == -1) {
PLOG(ERROR) << "pthread_sigmask(SIG_BLOCK, SIGCHLD)";
return false;
}
@ -715,7 +765,7 @@ bool Monitor::InitSetupSignals(sigset_t* sset) {
return true;
}
bool Monitor::InitSendPolicy() {
bool MonitorBase::InitSendPolicy() {
if (!policy_->SendPolicy(comms_)) {
LOG(ERROR) << "Couldn't send policy";
return false;
@ -724,7 +774,7 @@ bool Monitor::InitSendPolicy() {
return true;
}
bool Monitor::InitSendCwd() {
bool MonitorBase::InitSendCwd() {
if (!comms_->SendString(executor_->cwd_)) {
PLOG(ERROR) << "Couldn't send cwd";
return false;
@ -733,8 +783,8 @@ bool Monitor::InitSendCwd() {
return true;
}
bool Monitor::InitApplyLimit(pid_t pid, int resource,
const rlimit64& rlim) const {
bool MonitorBase::InitApplyLimit(pid_t pid, int resource,
const rlimit64& rlim) const {
#if defined(__ANDROID__)
using RlimitResource = int;
#else
@ -765,18 +815,20 @@ bool Monitor::InitApplyLimit(pid_t pid, int resource,
return true;
}
bool Monitor::InitApplyLimits() {
bool MonitorBase::InitApplyLimits() {
Limits* limits = executor_->limits();
return InitApplyLimit(pid_, RLIMIT_AS, limits->rlimit_as()) &&
InitApplyLimit(pid_, RLIMIT_CPU, limits->rlimit_cpu()) &&
InitApplyLimit(pid_, RLIMIT_FSIZE, limits->rlimit_fsize()) &&
InitApplyLimit(pid_, RLIMIT_NOFILE, limits->rlimit_nofile()) &&
InitApplyLimit(pid_, RLIMIT_CORE, limits->rlimit_core());
return InitApplyLimit(process_.main_pid, RLIMIT_AS, limits->rlimit_as()) &&
InitApplyLimit(process_.main_pid, RLIMIT_CPU, limits->rlimit_cpu()) &&
InitApplyLimit(process_.main_pid, RLIMIT_FSIZE,
limits->rlimit_fsize()) &&
InitApplyLimit(process_.main_pid, RLIMIT_NOFILE,
limits->rlimit_nofile()) &&
InitApplyLimit(process_.main_pid, RLIMIT_CORE, limits->rlimit_core());
}
bool Monitor::InitSendIPC() { return ipc_->SendFdsOverComms(); }
bool MonitorBase::InitSendIPC() { return ipc_->SendFdsOverComms(); }
bool Monitor::WaitForSandboxReady() {
bool MonitorBase::WaitForSandboxReady() {
uint32_t tmp;
if (!comms_->RecvUint32(&tmp)) {
LOG(ERROR) << "Couldn't receive 'Client::kClient2SandboxReady' message";
@ -790,12 +842,22 @@ bool Monitor::WaitForSandboxReady() {
return true;
}
bool Monitor::InitPtraceAttach() {
bool PtraceMonitor::InitPtraceAttach() {
sanitizer::WaitForSanitizer();
if (process_.init_pid > 0) {
if (ptrace(PTRACE_SEIZE, process_.init_pid, 0, PTRACE_O_EXITKILL) != 0) {
if (errno != ESRCH) {
PLOG(ERROR) << "attaching to init process failed";
}
return false;
}
}
// Get a list of tasks.
absl::flat_hash_set<int> tasks;
if (auto task_list = sanitizer::GetListOfTasks(pid_); task_list.ok()) {
if (auto task_list = sanitizer::GetListOfTasks(process_.main_pid);
task_list.ok()) {
tasks = *std::move(task_list);
} else {
LOG(ERROR) << "Could not get list of tasks: "
@ -803,14 +865,16 @@ bool Monitor::InitPtraceAttach() {
return false;
}
if (tasks.find(pid_) == tasks.end()) {
LOG(ERROR) << "The pid " << pid_ << " was not found in its own tasklist.";
if (tasks.find(process_.main_pid) == tasks.end()) {
LOG(ERROR) << "The pid " << process_.main_pid
<< " was not found in its own tasklist.";
return false;
}
// With TSYNC, we can allow threads: seccomp applies to all threads.
if (tasks.size() > 1) {
LOG(WARNING) << "PID " << pid_ << " has " << tasks.size() << " threads,"
LOG(WARNING) << "PID " << process_.main_pid << " has " << tasks.size()
<< " threads,"
<< " at the time of call to SandboxMeHere. If you are seeing"
<< " more sandbox violations than expected, this might be"
<< " the reason why"
@ -874,7 +938,8 @@ bool Monitor::InitPtraceAttach() {
}
// Get a list of tasks after attaching.
if (auto tasks_list = sanitizer::GetListOfTasks(pid_); tasks_list.ok()) {
if (auto tasks_list = sanitizer::GetListOfTasks(process_.main_pid);
tasks_list.ok()) {
tasks = *std::move(tasks_list);
} else {
LOG(ERROR) << "Could not get list of tasks: "
@ -884,7 +949,7 @@ bool Monitor::InitPtraceAttach() {
// Check that we attached to all the threads
if (tasks_attached != tasks) {
LOG(ERROR) << "The pid " << pid_
LOG(ERROR) << "The pid " << process_.main_pid
<< " spawned new threads while we were trying to attach to it.";
return false;
}
@ -892,7 +957,7 @@ bool Monitor::InitPtraceAttach() {
// No glibc wrapper for gettid - see 'man gettid'.
VLOG(1) << "Monitor (PID: " << getpid()
<< ", TID: " << util::Syscall(__NR_gettid)
<< ") attached to PID: " << pid_;
<< ") attached to PID: " << process_.main_pid;
// Technically, the sandboxee can be in a ptrace-stopped state right now,
// because some signal could have arrived in the meantime. Yet, this
@ -908,7 +973,7 @@ bool Monitor::InitPtraceAttach() {
return true;
}
void Monitor::ActionProcessSyscall(Regs* regs, const Syscall& syscall) {
void PtraceMonitor::ActionProcessSyscall(Regs* regs, const Syscall& syscall) {
// If the sandboxing is not enabled yet, allow the first __NR_execveat.
if (syscall.nr() == __NR_execveat && !IsActivelyMonitoring()) {
VLOG(1) << "[PERMITTED/BEFORE_EXECVEAT]: "
@ -955,8 +1020,8 @@ void Monitor::ActionProcessSyscall(Regs* regs, const Syscall& syscall) {
ActionProcessSyscallViolation(regs, syscall, kSyscallViolation);
}
void Monitor::ActionProcessSyscallViolation(Regs* regs, const Syscall& syscall,
ViolationType violation_type) {
void PtraceMonitor::ActionProcessSyscallViolation(
Regs* regs, const Syscall& syscall, ViolationType violation_type) {
LogSyscallViolation(syscall);
notify_->EventSyscallViolation(syscall, violation_type);
SetExitStatusCode(Result::VIOLATION, syscall.nr());
@ -970,7 +1035,7 @@ void Monitor::ActionProcessSyscallViolation(Regs* regs, const Syscall& syscall,
}
}
void Monitor::LogSyscallViolation(const Syscall& syscall) const {
void MonitorBase::LogSyscallViolation(const Syscall& syscall) const {
// Do not unwind libunwind.
if (executor_->libunwind_sbox_for_pid_ != 0) {
LOG(ERROR) << "Sandbox violation during execution of libunwind: "
@ -992,7 +1057,7 @@ void Monitor::LogSyscallViolation(const Syscall& syscall) const {
LogSyscallViolationExplanation(syscall);
}
void Monitor::EventPtraceSeccomp(pid_t pid, int event_msg) {
void PtraceMonitor::EventPtraceSeccomp(pid_t pid, int event_msg) {
if (event_msg < sapi::cpu::Architecture::kUnknown ||
event_msg > sapi::cpu::Architecture::kMax) {
// We've observed that, if the process has exited, the event_msg may contain
@ -1032,7 +1097,7 @@ void Monitor::EventPtraceSeccomp(pid_t pid, int event_msg) {
ActionProcessSyscall(&regs, syscall);
}
void Monitor::EventSyscallExit(pid_t pid) {
void PtraceMonitor::EventSyscallExit(pid_t pid) {
// Check that the monitor wants to inspect the current syscall's return value.
auto index = syscalls_in_progress_.find(pid);
if (index == syscalls_in_progress_.end()) {
@ -1058,7 +1123,7 @@ void Monitor::EventSyscallExit(pid_t pid) {
ContinueProcess(pid, 0);
}
void Monitor::EventPtraceNewProcess(pid_t pid, int event_msg) {
void PtraceMonitor::EventPtraceNewProcess(pid_t pid, int event_msg) {
// ptrace doesn't issue syscall-exit-stops for successful fork/vfork/clone
// system calls. Check if the monitor wanted to inspect the syscall's return
// value, and call EventSyscallReturn for the parent process if so.
@ -1087,7 +1152,7 @@ void Monitor::EventPtraceNewProcess(pid_t pid, int event_msg) {
ContinueProcess(pid, 0);
}
void Monitor::EventPtraceExec(pid_t pid, int event_msg) {
void PtraceMonitor::EventPtraceExec(pid_t pid, int event_msg) {
if (!IsActivelyMonitoring()) {
VLOG(1) << "PTRACE_EVENT_EXEC seen from PID: " << event_msg
<< ". SANDBOX ENABLED!";
@ -1112,14 +1177,14 @@ void Monitor::EventPtraceExec(pid_t pid, int event_msg) {
ContinueProcess(pid, 0);
}
void Monitor::EventPtraceExit(pid_t pid, int event_msg) {
void PtraceMonitor::EventPtraceExit(pid_t pid, int event_msg) {
// Forget about any syscalls in progress for this PID.
syscalls_in_progress_.erase(pid);
// A regular exit, let it continue (fast-path).
if (ABSL_PREDICT_TRUE(
WIFEXITED(event_msg) &&
(!policy_->collect_stacktrace_on_exit_ || pid != pid_))) {
if (ABSL_PREDICT_TRUE(WIFEXITED(event_msg) &&
(!policy_->collect_stacktrace_on_exit_ ||
pid != process_.main_pid))) {
ContinueProcess(pid, 0);
return;
}
@ -1130,7 +1195,7 @@ void Monitor::EventPtraceExit(pid_t pid, int event_msg) {
absl::GetFlag(FLAGS_sandbox2_log_all_stack_traces);
// Fetch the registers as we'll need them to fill the result in any case
auto regs = std::make_unique<Regs>(pid);
if (is_seccomp || pid == pid_ || log_stack_trace) {
if (is_seccomp || pid == process_.main_pid || log_stack_trace) {
auto status = regs->Fetch();
if (!status.ok()) {
LOG(ERROR) << "failed to fetch regs: " << status;
@ -1152,7 +1217,7 @@ void Monitor::EventPtraceExit(pid_t pid, int event_msg) {
// 2) Process was killed because it hit a timeout.
// 3) Regular signal/other exit cause.
// 4) Normal exit for which we want to obtain stack trace.
if (pid == pid_) {
if (pid == process_.main_pid) {
VLOG(1) << "PID: " << pid << " main special exit";
if (network_violation_) {
SetExitStatusCode(Result::VIOLATION, Result::VIOLATION_NETWORK);
@ -1166,6 +1231,7 @@ void Monitor::EventPtraceExit(pid_t pid, int event_msg) {
} else {
SetExitStatusCode(Result::SIGNALED, WTERMSIG(event_msg));
}
LOG(INFO) << "CPU limit: " << executor_->limits()->rlimit_cpu().rlim_cur;
SetAdditionalResultInfo(std::move(regs));
} else if (log_stack_trace) {
// In case pid == pid_ the stack trace will be logged anyway. So we need
@ -1182,7 +1248,7 @@ void Monitor::EventPtraceExit(pid_t pid, int event_msg) {
ContinueProcess(pid, 0);
}
void Monitor::EventPtraceStop(pid_t pid, int stopsig) {
void PtraceMonitor::EventPtraceStop(pid_t pid, int stopsig) {
// It's not a real stop signal. For example PTRACE_O_TRACECLONE and similar
// flags to ptrace(PTRACE_SEIZE) might generate this event with SIGTRAP.
if (stopsig != SIGSTOP && stopsig != SIGTSTP && stopsig != SIGTTIN &&
@ -1196,7 +1262,7 @@ void Monitor::EventPtraceStop(pid_t pid, int stopsig) {
StopProcess(pid, 0);
}
void Monitor::StateProcessStopped(pid_t pid, int status) {
void PtraceMonitor::StateProcessStopped(pid_t pid, int status) {
int stopsig = WSTOPSIG(status);
// We use PTRACE_O_TRACESYSGOOD, so we can tell it's a syscall stop without
// calling PTRACE_GETSIGINFO by checking the value of the reported signal.
@ -1223,7 +1289,7 @@ void Monitor::StateProcessStopped(pid_t pid, int status) {
return;
}
if (ABSL_PREDICT_FALSE(pid == pid_ && should_dump_stack_ &&
if (ABSL_PREDICT_FALSE(pid == process_.main_pid && should_dump_stack_ &&
executor_->libunwind_sbox_for_pid_ == 0 &&
policy_->GetNamespace())) {
auto stack_trace = [this,
@ -1295,7 +1361,7 @@ void Monitor::StateProcessStopped(pid_t pid, int status) {
}
}
void Monitor::LogSyscallViolationExplanation(const Syscall& syscall) const {
void MonitorBase::LogSyscallViolationExplanation(const Syscall& syscall) const {
const uintptr_t syscall_nr = syscall.nr();
const uintptr_t arg0 = syscall.args()[0];
@ -1325,7 +1391,7 @@ void Monitor::LogSyscallViolationExplanation(const Syscall& syscall) const {
}
}
void Monitor::EnableNetworkProxyServer() {
void MonitorBase::EnableNetworkProxyServer() {
int fd = ipc_->ReceiveFd(NetworkProxyClient::kFDName);
network_proxy_server_ = std::make_unique<NetworkProxyServer>(

View File

@ -29,6 +29,8 @@
#include <thread>
#include "absl/container/flat_hash_map.h"
#include "absl/status/statusor.h"
#include "absl/synchronization/mutex.h"
#include "absl/synchronization/notification.h"
#include "sandboxed_api/sandbox2/comms.h"
#include "sandboxed_api/sandbox2/executor.h"
@ -39,36 +41,66 @@
#include "sandboxed_api/sandbox2/regs.h"
#include "sandboxed_api/sandbox2/result.h"
#include "sandboxed_api/sandbox2/syscall.h"
#include "sandboxed_api/util/raw_logging.h"
namespace sandbox2 {
class Monitor final {
class MonitorBase {
public:
// executor, policy and notify are not owned by the Monitor
Monitor(Executor* executor, Policy* policy, Notify* notify);
MonitorBase(Executor* executor, Policy* policy, Notify* notify);
Monitor(const Monitor&) = delete;
Monitor& operator=(const Monitor&) = delete;
MonitorBase(const MonitorBase&) = delete;
MonitorBase& operator=(const MonitorBase&) = delete;
~Monitor();
private:
friend class Sandbox2;
// Timeout used with sigtimedwait (0.5s).
static constexpr int kWakeUpPeriodSec = 0L;
static constexpr int kWakeUpPeriodNSec = (500L * 1000L * 1000L);
virtual ~MonitorBase();
// Starts the Monitor.
void Run();
void Launch();
// Getters for private fields.
bool IsDone() const { return done_notification_.HasBeenNotified(); }
// Getter/Setter for wait_for_execve_.
bool IsActivelyMonitoring();
void SetActivelyMonitoring();
// Enable network proxy server, this will start a thread in the sandbox
// that waits for connection requests from the sandboxee.
void EnableNetworkProxyServer();
pid_t pid() const { return process_.main_pid; }
const Result& result() const { return result_; }
absl::StatusOr<Result> AwaitResultWithTimeout(absl::Duration timeout);
virtual void Kill() = 0;
virtual void DumpStackTrace() = 0;
virtual void SetWallTimeLimit(absl::Duration limit) = 0;
protected:
void OnDone();
// Sets basic info status and reason code in the result object.
void SetExitStatusCode(Result::StatusEnum final_status,
uintptr_t reason_code);
// Logs a SANDBOX VIOLATION message based on the registers and additional
// explanation for the reason of the violation.
void LogSyscallViolation(const Syscall& syscall) const;
// Internal objects, owned by the Sandbox2 object.
Executor* executor_;
Notify* notify_;
Policy* policy_;
// The sandboxee process.
Executor::Process process_;
Result result_;
// Comms channel ptr, copied from the Executor object for convenience.
Comms* comms_;
// Log file specified by
// --sandbox_danger_danger_permit_all_and_log flag.
FILE* log_file_ = nullptr;
// Handle to the class responsible for proxying and validating connect()
// requests.
std::unique_ptr<NetworkProxyServer> network_proxy_server_;
private:
// Sends Policy to the Client.
// Returns success/failure status.
bool InitSendPolicy();
@ -77,13 +109,6 @@ class Monitor final {
// Returns success/failure status.
bool WaitForSandboxReady();
// ptrace(PTRACE_SEIZE) to the Client.
// Returns success/failure status.
bool InitPtraceAttach();
// Sets up required signal masks/handlers; prepare mask for sigtimedwait().
bool InitSetupSignals(sigset_t* sset);
// Sends information about data exchange channels.
bool InitSendIPC();
@ -96,48 +121,95 @@ class Monitor final {
// Applies individual limit on the sandboxee.
bool InitApplyLimit(pid_t pid, int resource, const rlimit64& rlim) const;
// Kills the main traced PID with PTRACE_KILL.
// Returns false if an error occured and process could not be killed.
bool KillSandboxee();
// Logs an additional explanation for the possible reason of the violation
// based on the registers.
void LogSyscallViolationExplanation(const Syscall& syscall) const;
// Interrupts the main traced PID with PTRACE_INTERRUPT.
// Returns false if an error occured and process could not be interrupted.
bool InterruptSandboxee();
virtual void RunInternal() = 0;
virtual void Join() = 0;
// IPC ptr, used for exchanging data with the sandboxee.
IPC* ipc_;
// The field indicates whether the sandboxing task has been completed (either
// successfully or with error).
absl::Notification done_notification_;
// Empty temp file used for mapping the comms fd when the Tomoyo LSM is
// active.
std::string comms_fd_dev_;
std::thread network_proxy_thread_;
};
class PtraceMonitor : public MonitorBase {
public:
PtraceMonitor(Executor* executor, Policy* policy, Notify* notify);
~PtraceMonitor() { Join(); }
void Kill() override {
external_kill_request_flag_.clear(std::memory_order_relaxed);
NotifyMonitor();
}
void DumpStackTrace() override {
dump_stack_request_flag_.clear(std::memory_order_relaxed);
NotifyMonitor();
}
void SetWallTimeLimit(absl::Duration limit) override {
if (limit == absl::ZeroDuration()) {
VLOG(1) << "Disarming walltime timer to ";
deadline_millis_.store(0, std::memory_order_relaxed);
} else {
VLOG(1) << "Will set the walltime timer to " << limit;
absl::Time deadline = absl::Now() + limit;
deadline_millis_.store(absl::ToUnixMillis(deadline),
std::memory_order_relaxed);
}
}
private:
// Timeout used with sigtimedwait (0.5s).
static constexpr int kWakeUpPeriodSec = 0L;
static constexpr int kWakeUpPeriodNSec = (500L * 1000L * 1000L);
// Waits for events from monitored clients and signals from the main process.
void MainLoop(sigset_t* sset);
void RunInternal() override;
void Join() override;
void Run();
// Notifies monitor about a state change
void NotifyMonitor();
// PID called a traced syscall, or was killed due to syscall.
void ActionProcessSyscall(Regs* regs, const Syscall& syscall);
// Getter/Setter for wait_for_execve_.
bool IsActivelyMonitoring();
void SetActivelyMonitoring();
// Process with given PID changed state to a stopped state.
void StateProcessStopped(pid_t pid, int status);
// Tells if collecting stack trace is at all possible.
bool StackTraceCollectionPossible() const;
// Whether a stack trace should be collected given the current status
bool ShouldCollectStackTrace() const;
// Sets additional information in the result object, such as program name,
// stack trace etc.
void SetAdditionalResultInfo(std::unique_ptr<Regs> regs);
// Logs the syscall violation and kills the process afterwards.
void ActionProcessSyscallViolation(Regs* regs, const Syscall& syscall,
ViolationType violation_type);
// PID called a traced syscall, or was killed due to syscall.
void ActionProcessSyscall(Regs* regs, const Syscall& syscall);
// Sets basic info status and reason code in the result object.
void SetExitStatusCode(Result::StatusEnum final_status,
uintptr_t reason_code);
// Tells if collecting stack trace is at all possible.
bool StackTraceCollectionPossible();
// Whether a stack trace should be collected given the status
bool ShouldCollectStackTrace();
// Sets additional information in the result object, such as program name,
// stack trace etc.
void SetAdditionalResultInfo(std::unique_ptr<Regs> regs);
// Gets and logs stack trace.
absl::StatusOr<std::vector<std::string>> GetAndLogStackTrace(
const Regs* regs);
// Logs a SANDBOX VIOLATION message based on the registers and additional
// explanation for the reason of the violation.
void LogSyscallViolation(const Syscall& syscall) const;
// Logs an additional explanation for the possible reason of the violation
// based on the registers.
void LogSyscallViolationExplanation(const Syscall& syscall) const;
void LogStackTraceOfPid(pid_t pid);
// Ptrace events:
@ -159,37 +231,31 @@ class Monitor final {
// Processes syscall exit.
void EventSyscallExit(pid_t pid);
// Enable network proxy server, this will start a thread in the sandbox
// that waits for connection requests from the sandboxee.
void EnableNetworkProxyServer();
// Kills the main traced PID with PTRACE_KILL.
// Returns false if an error occured and process could not be killed.
bool KillSandboxee();
// Internal objects, owned by the Sandbox2 object.
Executor* executor_;
Notify* notify_;
Policy* policy_;
Result result_;
// Comms channel ptr, copied from the Executor object for convenience.
Comms* comms_;
// IPC ptr, used for exchanging data with the sandboxee.
IPC* ipc_;
// Interrupts the main traced PID with PTRACE_INTERRUPT.
// Returns false if an error occured and process could not be interrupted.
bool InterruptSandboxee();
// Sets up required signal masks/handlers; prepare mask for sigtimedwait().
bool InitSetupSignals();
// ptrace(PTRACE_SEIZE) to the Client.
// Returns success/failure status.
bool InitPtraceAttach();
// Parent (the Sandbox2 object) waits on it, until we either enable
// monitoring of a process (sandboxee) successfully, or the setup process
// fails.
absl::Notification setup_notification_;
// The field indicates whether the sandboxing task has been completed (either
// successfully or with error).
absl::Notification done_notification_;
// The main tracked PID.
pid_t pid_ = -1;
// Deadline in Unix millis
std::atomic<int64_t> deadline_millis_{0};
// False iff external kill is requested
std::atomic_flag external_kill_request_flag_ = ATOMIC_FLAG_INIT;
// False iff dump stack is requested
std::atomic_flag dump_stack_request_flag_ = ATOMIC_FLAG_INIT;
// Deadline in Unix millis
std::atomic<int64_t> deadline_millis_{0};
// Was external kill sent to the sandboxee
bool external_kill_ = false;
// Network violation occurred and process of killing sandboxee started
@ -198,27 +264,19 @@ class Monitor final {
bool timed_out_ = false;
// Should we dump the main sandboxed PID's stack?
bool should_dump_stack_ = false;
// Is the sandboxee actively monitored, or maybe we're waiting for execve()?
bool wait_for_execve_;
// Is the sandboxee forked from a custom forkserver?
bool uses_custom_forkserver_;
// Log file specified by
// --sandbox_danger_danger_permit_all_and_log flag.
FILE* log_file_ = nullptr;
// Empty temp file used for mapping the comms fd when the Tomoyo LSM is
// active.
std::string comms_fd_dev_;
// Handle to the class responsible for proxying and validating connect()
// requests.
std::unique_ptr<NetworkProxyServer> network_proxy_server_;
std::thread network_proxy_thread_;
// Syscalls that are running, whose result values we want to inspect.
absl::flat_hash_map<pid_t, Syscall> syscalls_in_progress_;
sigset_t sset_;
// Is the sandboxee forked from a custom forkserver?
bool uses_custom_forkserver_;
// Monitor thread object.
std::unique_ptr<std::thread> thread_;
// Synchronizes monitor thread deletion and notifying the monitor.
absl::Mutex notify_mutex_;
};
} // namespace sandbox2

View File

@ -59,7 +59,8 @@ class Policy final {
void GetPolicyDescription(PolicyDescription* policy) const;
private:
friend class Monitor;
friend class MonitorBase;
friend class PtraceMonitor;
friend class PolicyBuilder;
friend class PolicyBuilderPeer; // For testing
friend class StackTracePeer;

View File

@ -22,45 +22,16 @@
#include "absl/log/check.h"
#include "absl/status/statusor.h"
#include "absl/synchronization/mutex.h"
#include "absl/time/time.h"
#include "sandboxed_api/sandbox2/monitor.h"
#include "sandboxed_api/sandbox2/result.h"
#include "sandboxed_api/util/raw_logging.h"
namespace sandbox2 {
Sandbox2::~Sandbox2() {
if (monitor_thread_ && monitor_thread_->joinable()) {
monitor_thread_->join();
}
}
absl::StatusOr<Result> Sandbox2::AwaitResultWithTimeout(
absl::Duration timeout) {
CHECK(monitor_ != nullptr) << "Sandbox was not launched yet";
CHECK(monitor_thread_ != nullptr) << "Sandbox was already waited on";
auto done =
monitor_->done_notification_.WaitForNotificationWithTimeout(timeout);
if (!done) {
return absl::DeadlineExceededError("Sandbox did not finish within timeout");
}
{
absl::MutexLock lock(&monitor_notify_mutex_);
monitor_thread_->join();
CHECK(IsTerminated()) << "Monitor did not terminate";
// Reset the Monitor Thread object to its initial state, as to mark that
// this object cannot be used anymore to control behavior of the sandboxee
// (e.g. via signals).
monitor_thread_.reset();
}
VLOG(1) << "Final execution status: " << monitor_->result_.ToString();
CHECK(monitor_->result_.final_status() != Result::UNSET);
return std::move(monitor_->result_);
return monitor_->AwaitResultWithTimeout(timeout);
}
Result Sandbox2::AwaitResult() {
@ -72,61 +43,36 @@ bool Sandbox2::RunAsync() {
// If the sandboxee setup failed we return 'false' here.
if (monitor_->IsDone() &&
monitor_->result_.final_status() == Result::SETUP_ERROR) {
monitor_->result().final_status() == Result::SETUP_ERROR) {
return false;
}
return true;
}
void Sandbox2::NotifyMonitor() {
absl::ReaderMutexLock lock(&monitor_notify_mutex_);
if (monitor_thread_ != nullptr) {
pthread_kill(monitor_thread_->native_handle(), SIGCHLD);
}
}
void Sandbox2::Kill() {
CHECK(monitor_ != nullptr) << "Sandbox was not launched yet";
monitor_->external_kill_request_flag_.clear(std::memory_order_relaxed);
NotifyMonitor();
monitor_->Kill();
}
void Sandbox2::DumpStackTrace() {
CHECK(monitor_ != nullptr) << "Sandbox was not launched yet";
monitor_->dump_stack_request_flag_.clear(std::memory_order_relaxed);
NotifyMonitor();
monitor_->DumpStackTrace();
}
bool Sandbox2::IsTerminated() const {
CHECK(monitor_ != nullptr) << "Sandbox was not launched yet";
return monitor_->IsDone();
}
void Sandbox2::set_walltime_limit(absl::Duration limit) const {
if (limit == absl::ZeroDuration()) {
VLOG(1) << "Disarming walltime timer to ";
monitor_->deadline_millis_.store(0, std::memory_order_relaxed);
} else {
VLOG(1) << "Will set the walltime timer to " << limit;
absl::Time deadline = absl::Now() + limit;
monitor_->deadline_millis_.store(absl::ToUnixMillis(deadline),
std::memory_order_relaxed);
}
CHECK(monitor_ != nullptr) << "Sandbox was not launched yet";
monitor_->SetWallTimeLimit(limit);
}
void Sandbox2::Launch() {
monitor_ =
std::make_unique<Monitor>(executor_.get(), policy_.get(), notify_.get());
monitor_thread_ =
std::make_unique<std::thread>(&Monitor::Run, monitor_.get());
// Wait for the Monitor to set-up the sandboxee correctly (or fail while
// doing that). From here on, it is safe to use the IPC object for
// non-sandbox-related data exchange.
monitor_->setup_notification_.WaitForNotification();
monitor_ = std::make_unique<PtraceMonitor>(executor_.get(), policy_.get(),
notify_.get());
monitor_->Launch();
}
} // namespace sandbox2

View File

@ -20,13 +20,10 @@
#include <ctime>
#include <memory>
#include <thread> // NOLINT(build/c++11)
#include <utility>
#include "absl/base/macros.h"
#include "absl/log/log.h"
#include "absl/status/statusor.h"
#include "absl/synchronization/mutex.h"
#include "sandboxed_api/sandbox2/comms.h"
#include "sandboxed_api/sandbox2/executor.h"
#include "sandboxed_api/sandbox2/ipc.h"
@ -54,8 +51,6 @@ class Sandbox2 final {
}
}
~Sandbox2();
Sandbox2(const Sandbox2&) = delete;
Sandbox2& operator=(const Sandbox2&) = delete;
@ -106,7 +101,7 @@ class Sandbox2 final {
void set_walltime_limit(absl::Duration limit) const;
// Returns the process id inside the executor.
pid_t pid() const { return monitor_ != nullptr ? monitor_->pid_ : -1; }
pid_t pid() const { return monitor_ != nullptr ? monitor_->pid() : -1; }
// Gets the comms inside the executor.
Comms* comms() {
@ -116,8 +111,6 @@ class Sandbox2 final {
private:
// Launches the Monitor.
void Launch();
// Notifies monitor about a state change
void NotifyMonitor();
// Executor set by user - owned by Sandbox2.
std::unique_ptr<Executor> executor_;
@ -129,13 +122,7 @@ class Sandbox2 final {
std::unique_ptr<Notify> notify_;
// Monitor object - owned by Sandbox2.
std::unique_ptr<Monitor> monitor_;
// Monitor thread object - owned by Sandbox2.
std::unique_ptr<std::thread> monitor_thread_;
// Synchronizes monitor thread deletion and notifying the monitor.
absl::Mutex monitor_notify_mutex_;
std::unique_ptr<MonitorBase> monitor_;
};
} // namespace sandbox2

View File

@ -20,6 +20,7 @@
#include <csignal>
#include <memory>
#include <string>
#include <thread> // NOLINT(build/c++11)
#include <vector>
#include "gmock/gmock.h"
@ -180,6 +181,24 @@ TEST(RunAsyncTest, SandboxeeTimeoutDisabledStacktraces) {
EXPECT_THAT(result.GetStackTrace(), IsEmpty());
}
TEST(RunAsyncTest, SandboxeeNotKilledWhenStartingThreadFinishes) {
const std::string path = GetTestSourcePath("sandbox2/testcases/minimal");
std::vector<std::string> args = {path};
auto executor = std::make_unique<Executor>(path, args);
SAPI_ASSERT_OK_AND_ASSIGN(auto policy,
PolicyBuilder()
// Don't restrict the syscalls at all.
.DangerDefaultAllowAll()
.CollectStacktracesOnExit(true)
.TryBuild());
Sandbox2 sandbox(std::move(executor), std::move(policy));
std::thread sandbox_start_thread([&sandbox]() { sandbox.RunAsync(); });
sandbox_start_thread.join();
Result result = sandbox.AwaitResult();
EXPECT_EQ(result.final_status(), Result::OK);
}
TEST(StarvationTest, MonitorIsNotStarvedByTheSandboxee) {
const std::string path = GetTestSourcePath("sandbox2/testcases/starve");