linux: support multi client sockets in ExceptionHandlerServer

Multi client socket connections allow multiple clients to request crash
dumps from a handler process using a single, shared socket connection.

This connection mode does not support using a broker process which
requires a dedicated socket connection to ensure handler messages
aren't intercepted by the wrong clients.

The handler uses SIGCONT to indicate to the crasher when a crash dump
is complete (or has failed) and may continue.

Bug: crashpad:284
Change-Id: I2031029cd254f17497cbf7e7d8740c289581e8aa
Reviewed-on: https://chromium-review.googlesource.com/c/crashpad/crashpad/+/1559306
Reviewed-by: Mark Mentovai <mark@chromium.org>
This commit is contained in:
Joshua Peraza 2019-04-29 14:28:53 -07:00
parent 48675b4bd3
commit 59cdfbb031
15 changed files with 498 additions and 83 deletions

View File

@ -356,7 +356,7 @@ class StartHandlerForClientTest {
FromPointerCast<VMAddress>(&sanitization_info);
}
ExceptionHandlerClient handler_client(state->client_sock_);
ExceptionHandlerClient handler_client(state->client_sock_, false);
CHECK_EQ(handler_client.RequestCrashDump(info), 0);
Signals::RestoreHandlerAndReraiseSignalOnReturn(siginfo, nullptr);

View File

@ -993,8 +993,8 @@ int HandlerMain(int argc,
}
#elif defined(OS_LINUX) || defined(OS_ANDROID)
if (options.initial_client_fd == kInvalidFileHandle ||
!exception_handler_server.InitializeWithClient(
ScopedFileHandle(options.initial_client_fd))) {
!exception_handler_server.InitializeWithClient(
ScopedFileHandle(options.initial_client_fd), false)) {
return ExitFailure();
}
#endif // OS_WIN

View File

@ -32,6 +32,7 @@
#include "build/build_config.h"
#include "util/file/file_io.h"
#include "util/file/filesystem.h"
#include "util/linux/proc_task_reader.h"
#include "util/misc/as_underlying_type.h"
namespace crashpad {
@ -122,20 +123,48 @@ bool SendMessageToClient(
return LoggingWriteFile(client_sock, &message, sizeof(message));
}
int tgkill(pid_t pid, pid_t tid, int signo) {
return syscall(SYS_tgkill, pid, tid, signo);
}
void SendSIGCONT(pid_t pid, pid_t tid) {
if (tid > 0) {
if (tgkill(pid, tid, ExceptionHandlerProtocol::kDumpDoneSignal) != 0) {
PLOG(ERROR) << "tgkill";
}
return;
}
std::vector<pid_t> threads;
if (!ReadThreadIDs(pid, &threads)) {
return;
}
for (const auto& thread : threads) {
if (tgkill(pid, thread, ExceptionHandlerProtocol::kDumpDoneSignal) != 0) {
PLOG(ERROR) << "tgkill";
}
}
}
class PtraceStrategyDeciderImpl : public PtraceStrategyDecider {
public:
PtraceStrategyDeciderImpl() : PtraceStrategyDecider() {}
~PtraceStrategyDeciderImpl() = default;
Strategy ChooseStrategy(int sock, const ucred& client_credentials) override {
Strategy ChooseStrategy(int sock,
bool multiple_clients,
const ucred& client_credentials) override {
switch (GetPtraceScope()) {
case PtraceScope::kClassic:
if (getuid() == client_credentials.uid) {
if (getuid() == client_credentials.uid || HaveCapSysPtrace()) {
return Strategy::kDirectPtrace;
}
return TryForkingBroker(sock);
return multiple_clients ? Strategy::kNoPtrace : TryForkingBroker(sock);
case PtraceScope::kRestricted:
if (multiple_clients) {
return Strategy::kDirectPtrace;
}
if (!SendMessageToClient(sock,
ExceptionHandlerProtocol::
ServerToClientMessage::kTypeSetPtracer)) {
@ -196,12 +225,6 @@ class PtraceStrategyDeciderImpl : public PtraceStrategyDecider {
} // namespace
struct ExceptionHandlerServer::Event {
enum class Type { kShutdown, kClientMessage } type;
ScopedFileHandle fd;
};
ExceptionHandlerServer::ExceptionHandlerServer()
: clients_(),
shutdown_event_(),
@ -217,7 +240,8 @@ void ExceptionHandlerServer::SetPtraceStrategyDecider(
strategy_decider_ = std::move(decider);
}
bool ExceptionHandlerServer::InitializeWithClient(ScopedFileHandle sock) {
bool ExceptionHandlerServer::InitializeWithClient(ScopedFileHandle sock,
bool multiple_clients) {
INITIALIZATION_STATE_SET_INITIALIZING(initialized_);
pollfd_.reset(epoll_create1(EPOLL_CLOEXEC));
@ -245,7 +269,9 @@ bool ExceptionHandlerServer::InitializeWithClient(ScopedFileHandle sock) {
return false;
}
if (!InstallClientSocket(std::move(sock))) {
if (!InstallClientSocket(std::move(sock),
multiple_clients ? Event::Type::kSharedSocketMessage
: Event::Type::kClientMessage)) {
return false;
}
@ -287,8 +313,8 @@ void ExceptionHandlerServer::Stop() {
}
void ExceptionHandlerServer::HandleEvent(Event* event, uint32_t event_type) {
DCHECK_EQ(AsUnderlyingType(event->type),
AsUnderlyingType(Event::Type::kClientMessage));
DCHECK_NE(AsUnderlyingType(event->type),
AsUnderlyingType(Event::Type::kShutdown));
if (event_type & EPOLLERR) {
LogSocketError(event->fd.get());
@ -312,7 +338,8 @@ void ExceptionHandlerServer::HandleEvent(Event* event, uint32_t event_type) {
return;
}
bool ExceptionHandlerServer::InstallClientSocket(ScopedFileHandle socket) {
bool ExceptionHandlerServer::InstallClientSocket(ScopedFileHandle socket,
Event::Type type) {
// The handler may not have permission to set SO_PASSCRED on the socket, but
// it doesn't need to if the client has already set it.
// https://bugs.chromium.org/p/crashpad/issues/detail?id=252
@ -334,7 +361,7 @@ bool ExceptionHandlerServer::InstallClientSocket(ScopedFileHandle socket) {
}
auto event = std::make_unique<Event>();
event->type = Event::Type::kClientMessage;
event->type = type;
event->fd.reset(socket.release());
Event* eventp = event.get();
@ -421,10 +448,12 @@ bool ExceptionHandlerServer::ReceiveClientMessage(Event* event) {
switch (client_msg->type) {
case ExceptionHandlerProtocol::ClientToServerMessage::kCrashDumpRequest:
return HandleCrashDumpRequest(msg,
client_msg->client_info,
client_msg->requesting_thread_stack_address,
event->fd.get());
return HandleCrashDumpRequest(
msg,
client_msg->client_info,
client_msg->requesting_thread_stack_address,
event->fd.get(),
event->type == Event::Type::kSharedSocketMessage);
}
DCHECK(false);
@ -436,7 +465,8 @@ bool ExceptionHandlerServer::HandleCrashDumpRequest(
const msghdr& msg,
const ExceptionHandlerProtocol::ClientInformation& client_info,
VMAddress requesting_thread_stack_address,
int client_sock) {
int client_sock,
bool multiple_clients) {
cmsghdr* cmsg = CMSG_FIRSTHDR(&msg);
if (cmsg == nullptr) {
LOG(ERROR) << "missing credentials";
@ -460,23 +490,40 @@ bool ExceptionHandlerServer::HandleCrashDumpRequest(
ucred* client_credentials = reinterpret_cast<ucred*>(CMSG_DATA(cmsg));
pid_t client_process_id = client_credentials->pid;
pid_t requesting_thread_id = -1;
switch (strategy_decider_->ChooseStrategy(client_sock, *client_credentials)) {
switch (strategy_decider_->ChooseStrategy(
client_sock, multiple_clients, *client_credentials)) {
case PtraceStrategyDecider::Strategy::kError:
if (multiple_clients) {
SendSIGCONT(client_process_id, requesting_thread_id);
}
return false;
case PtraceStrategyDecider::Strategy::kNoPtrace:
if (multiple_clients) {
SendSIGCONT(client_process_id, requesting_thread_id);
return true;
}
return SendMessageToClient(
client_sock,
ExceptionHandlerProtocol::ServerToClientMessage::
kTypeCrashDumpFailed);
case PtraceStrategyDecider::Strategy::kDirectPtrace:
delegate_->HandleException(
client_process_id, client_info, requesting_thread_stack_address);
case PtraceStrategyDecider::Strategy::kDirectPtrace: {
delegate_->HandleException(client_process_id,
client_info,
requesting_thread_stack_address,
&requesting_thread_id);
if (multiple_clients) {
SendSIGCONT(client_process_id, requesting_thread_id);
return true;
}
break;
}
case PtraceStrategyDecider::Strategy::kUseBroker:
DCHECK(!multiple_clients);
delegate_->HandleExceptionWithBroker(
client_process_id, client_info, client_sock);
break;

View File

@ -54,9 +54,12 @@ class PtraceStrategyDecider {
//! \brief Chooses an appropriate `ptrace` strategy.
//!
//! \param[in] sock A socket conncted to a ExceptionHandlerClient.
//! \param[in] multiple_clients `true` if the socket is connected to multiple
//! clients. The broker is not supported in this configuration.
//! \param[in] client_credentials The credentials for the connected client.
//! \return the chosen #Strategy.
virtual Strategy ChooseStrategy(int sock,
bool multiple_clients,
const ucred& client_credentials) = 0;
protected:
@ -122,8 +125,11 @@ class ExceptionHandlerServer {
//! This method must be successfully called before Run().
//!
//! \param[in] sock A socket on which to receive client requests.
//! \param[in] multiple_clients `true` if this socket is used by multiple
//! clients. Using a broker process is not supported in this
//! configuration.
//! \return `true` on success. `false` on failure with a message logged.
bool InitializeWithClient(ScopedFileHandle sock);
bool InitializeWithClient(ScopedFileHandle sock, bool multiple_clients);
//! \brief Runs the exception-handling server.
//!
@ -143,17 +149,32 @@ class ExceptionHandlerServer {
void Stop();
private:
struct Event;
struct Event {
enum class Type {
// Used by Stop() to shutdown the server.
kShutdown,
// A message from a client on a private socket connection.
kClientMessage,
// A message from a client on a shared socket connection.
kSharedSocketMessage
};
Type type;
ScopedFileHandle fd;
};
void HandleEvent(Event* event, uint32_t event_type);
bool InstallClientSocket(ScopedFileHandle socket);
bool InstallClientSocket(ScopedFileHandle socket, Event::Type type);
bool UninstallClientSocket(Event* event);
bool ReceiveClientMessage(Event* event);
bool HandleCrashDumpRequest(
const msghdr& msg,
const ExceptionHandlerProtocol::ClientInformation& client_info,
VMAddress requesting_thread_stack_address,
int client_sock);
int client_sock,
bool multiple_clients);
std::unordered_map<int, std::unique_ptr<Event>> clients_;
std::unique_ptr<Event> shutdown_event_;

View File

@ -18,6 +18,7 @@
#include <unistd.h>
#include "base/logging.h"
#include "build/build_config.h"
#include "gtest/gtest.h"
#include "snapshot/linux/process_snapshot_linux.h"
#include "test/errors.h"
@ -30,6 +31,10 @@
#include "util/synchronization/semaphore.h"
#include "util/thread/thread.h"
#if defined(OS_ANDROID)
#include <android/api-level.h>
#endif
namespace crashpad {
namespace test {
namespace {
@ -164,7 +169,9 @@ class MockPtraceStrategyDecider : public PtraceStrategyDecider {
~MockPtraceStrategyDecider() {}
Strategy ChooseStrategy(int sock, const ucred& client_credentials) override {
Strategy ChooseStrategy(int sock,
bool multiple_clients,
const ucred& client_credentials) override {
if (strategy_ == Strategy::kUseBroker) {
ExceptionHandlerProtocol::ServerToClientMessage message = {};
message.type =
@ -194,13 +201,14 @@ class MockPtraceStrategyDecider : public PtraceStrategyDecider {
DISALLOW_COPY_AND_ASSIGN(MockPtraceStrategyDecider);
};
class ExceptionHandlerServerTest : public testing::Test {
class ExceptionHandlerServerTest : public testing::TestWithParam<bool> {
public:
ExceptionHandlerServerTest()
: server_(),
delegate_(),
server_thread_(&server_, &delegate_),
sock_to_handler_() {}
sock_to_handler_(),
use_multi_client_socket_(GetParam()) {}
~ExceptionHandlerServerTest() = default;
@ -243,7 +251,6 @@ class ExceptionHandlerServerTest : public testing::Test {
ExceptionHandlerProtocol::ClientInformation info;
info.exception_information_address = 42;
ASSERT_TRUE(LoggingWriteFile(WritePipeHandle(), &info, sizeof(info)));
// If the current ptrace_scope is restricted, the broker needs to be set
@ -251,7 +258,8 @@ class ExceptionHandlerServerTest : public testing::Test {
// ptracer allows the broker to inherit this condition.
ScopedPrSetPtracer set_ptracer(getpid(), /* may_log= */ true);
ExceptionHandlerClient client(server_test_->SockToHandler());
ExceptionHandlerClient client(server_test_->SockToHandler(),
server_test_->use_multi_client_socket_);
ASSERT_EQ(client.RequestCrashDump(info), 0);
}
@ -274,6 +282,8 @@ class ExceptionHandlerServerTest : public testing::Test {
test.Run();
}
bool UsingMultiClientSocket() const { return use_multi_client_socket_; }
protected:
void SetUp() override {
int socks[2];
@ -281,7 +291,8 @@ class ExceptionHandlerServerTest : public testing::Test {
sock_to_handler_.reset(socks[0]);
sock_to_client_ = socks[1];
ASSERT_TRUE(server_.InitializeWithClient(ScopedFileHandle(socks[1])));
ASSERT_TRUE(server_.InitializeWithClient(ScopedFileHandle(socks[1]),
use_multi_client_socket_));
}
private:
@ -290,36 +301,37 @@ class ExceptionHandlerServerTest : public testing::Test {
RunServerThread server_thread_;
ScopedFileHandle sock_to_handler_;
int sock_to_client_;
bool use_multi_client_socket_;
DISALLOW_COPY_AND_ASSIGN(ExceptionHandlerServerTest);
};
TEST_F(ExceptionHandlerServerTest, ShutdownWithNoClients) {
TEST_P(ExceptionHandlerServerTest, ShutdownWithNoClients) {
ServerThread()->Start();
Hangup();
ASSERT_TRUE(ServerThread()->JoinWithTimeout(5.0));
}
TEST_F(ExceptionHandlerServerTest, StopWithClients) {
TEST_P(ExceptionHandlerServerTest, StopWithClients) {
ServerThread()->Start();
Server()->Stop();
ASSERT_TRUE(ServerThread()->JoinWithTimeout(5.0));
}
TEST_F(ExceptionHandlerServerTest, StopBeforeRun) {
TEST_P(ExceptionHandlerServerTest, StopBeforeRun) {
Server()->Stop();
ServerThread()->Start();
ASSERT_TRUE(ServerThread()->JoinWithTimeout(5.0));
}
TEST_F(ExceptionHandlerServerTest, MultipleStops) {
TEST_P(ExceptionHandlerServerTest, MultipleStops) {
ServerThread()->Start();
Server()->Stop();
Server()->Stop();
ASSERT_TRUE(ServerThread()->JoinWithTimeout(5.0));
}
TEST_F(ExceptionHandlerServerTest, RequestCrashDumpDefault) {
TEST_P(ExceptionHandlerServerTest, RequestCrashDumpDefault) {
ScopedStopServerAndJoinThread stop_server(Server(), ServerThread());
ServerThread()->Start();
@ -327,25 +339,44 @@ TEST_F(ExceptionHandlerServerTest, RequestCrashDumpDefault) {
test.Run();
}
TEST_F(ExceptionHandlerServerTest, RequestCrashDumpNoPtrace) {
TEST_P(ExceptionHandlerServerTest, RequestCrashDumpNoPtrace) {
ExpectCrashDumpUsingStrategy(PtraceStrategyDecider::Strategy::kNoPtrace,
false);
}
TEST_F(ExceptionHandlerServerTest, RequestCrashDumpForkBroker) {
TEST_P(ExceptionHandlerServerTest, RequestCrashDumpForkBroker) {
if (UsingMultiClientSocket()) {
// The broker is not supported with multiple clients connected on a single
// socket.
return;
}
ExpectCrashDumpUsingStrategy(PtraceStrategyDecider::Strategy::kUseBroker,
true);
}
TEST_F(ExceptionHandlerServerTest, RequestCrashDumpDirectPtrace) {
TEST_P(ExceptionHandlerServerTest, RequestCrashDumpDirectPtrace) {
ExpectCrashDumpUsingStrategy(PtraceStrategyDecider::Strategy::kDirectPtrace,
true);
}
TEST_F(ExceptionHandlerServerTest, RequestCrashDumpError) {
TEST_P(ExceptionHandlerServerTest, RequestCrashDumpError) {
ExpectCrashDumpUsingStrategy(PtraceStrategyDecider::Strategy::kError, false);
}
INSTANTIATE_TEST_SUITE_P(ExceptionHandlerServerTestSuite,
ExceptionHandlerServerTest,
#if defined(OS_ANDROID) && __ANDROID_API__ < 23
// TODO(jperaza): Using a multi-client socket is not
// supported on Android until an lss sigtimedwait()
// wrapper is available to use in
// ExceptionHandlerClient::SignalCrashDump().
// https://crbug.com/crashpad/265
testing::Values(false)
#else
testing::Bool()
#endif
);
} // namespace
} // namespace test
} // namespace crashpad

View File

@ -295,6 +295,8 @@ static_library("util") {
"linux/memory_map.h",
"linux/proc_stat_reader.cc",
"linux/proc_stat_reader.h",
"linux/proc_task_reader.cc",
"linux/proc_task_reader.h",
"linux/ptrace_broker.cc",
"linux/ptrace_broker.h",
"linux/ptrace_client.cc",
@ -620,6 +622,7 @@ source_set("util_test") {
"linux/auxiliary_vector_test.cc",
"linux/memory_map_test.cc",
"linux/proc_stat_reader_test.cc",
"linux/proc_task_reader_test.cc",
"linux/ptrace_broker_test.cc",
"linux/ptracer_test.cc",
"linux/scoped_ptrace_attach_test.cc",
@ -668,6 +671,10 @@ source_set("util_test") {
"../third_party/zlib",
]
if (crashpad_is_android || crashpad_is_linux) {
deps += [ "../third_party/lss" ]
}
if (!crashpad_is_android) {
data_deps = [
":http_transport_test_server",

View File

@ -14,16 +14,10 @@
#include "util/linux/direct_ptrace_connection.h"
#include <stdio.h>
#include <utility>
#include "base/logging.h"
#include "base/stl_util.h"
#include "base/strings/string_number_conversions.h"
#include "util/file/directory_reader.h"
#include "util/file/file_io.h"
#include "util/misc/as_underlying_type.h"
#include "util/linux/proc_task_reader.h"
namespace crashpad {
@ -90,33 +84,7 @@ ProcessMemory* DirectPtraceConnection::Memory() {
bool DirectPtraceConnection::Threads(std::vector<pid_t>* threads) {
INITIALIZATION_STATE_DCHECK_VALID(initialized_);
DCHECK(threads->empty());
char path[32];
snprintf(path, base::size(path), "/proc/%d/task", pid_);
DirectoryReader reader;
if (!reader.Open(base::FilePath(path))) {
return false;
}
std::vector<pid_t> local_threads;
base::FilePath tid_str;
DirectoryReader::Result result;
while ((result = reader.NextFile(&tid_str)) ==
DirectoryReader::Result::kSuccess) {
pid_t tid;
if (!base::StringToInt(tid_str.value(), &tid)) {
LOG(ERROR) << "format error";
continue;
}
local_threads.push_back(tid);
}
DCHECK_EQ(AsUnderlyingType(result),
AsUnderlyingType(DirectoryReader::Result::kNoMoreFiles));
threads->swap(local_threads);
return true;
return ReadThreadIDs(pid_, threads);
}
} // namespace crashpad

View File

@ -15,6 +15,7 @@
#include "util/linux/exception_handler_client.h"
#include <errno.h>
#include <signal.h>
#include <sys/prctl.h>
#include <sys/socket.h>
#include <sys/wait.h>
@ -28,10 +29,42 @@
#include "util/misc/from_pointer_cast.h"
#include "util/posix/signals.h"
#if defined(OS_ANDROID)
#include <android/api-level.h>
#endif
namespace crashpad {
ExceptionHandlerClient::ExceptionHandlerClient(int sock)
: server_sock_(sock), ptracer_(-1), can_set_ptracer_(true) {}
namespace {
class ScopedSigprocmaskRestore {
public:
explicit ScopedSigprocmaskRestore(const sigset_t& set_to_block)
: orig_mask_(), mask_is_set_(false) {
mask_is_set_ = sigprocmask(SIG_BLOCK, &set_to_block, &orig_mask_) == 0;
DPLOG_IF(ERROR, !mask_is_set_) << "sigprocmask";
}
~ScopedSigprocmaskRestore() {
if (mask_is_set_ && sigprocmask(SIG_SETMASK, &orig_mask_, nullptr) != 0) {
DPLOG(ERROR) << "sigprocmask";
}
}
private:
sigset_t orig_mask_;
bool mask_is_set_;
DISALLOW_COPY_AND_ASSIGN(ScopedSigprocmaskRestore);
};
} // namespace
ExceptionHandlerClient::ExceptionHandlerClient(int sock, bool multiple_clients)
: server_sock_(sock),
ptracer_(-1),
can_set_ptracer_(true),
multiple_clients_(multiple_clients) {}
ExceptionHandlerClient::~ExceptionHandlerClient() = default;
@ -39,6 +72,10 @@ int ExceptionHandlerClient::RequestCrashDump(
const ExceptionHandlerProtocol::ClientInformation& info) {
VMAddress sp = FromPointerCast<VMAddress>(&sp);
if (multiple_clients_) {
return SignalCrashDump(info, sp);
}
int status = SendCrashDumpRequest(info, sp);
if (status != 0) {
return status;
@ -65,6 +102,38 @@ void ExceptionHandlerClient::SetCanSetPtracer(bool can_set_ptracer) {
can_set_ptracer_ = can_set_ptracer;
}
int ExceptionHandlerClient::SignalCrashDump(
const ExceptionHandlerProtocol::ClientInformation& info,
VMAddress stack_pointer) {
// TODO(jperaza): Use lss for system calls when sys_sigtimedwait() exists.
// https://crbug.com/crashpad/265
sigset_t dump_done_sigset;
sigemptyset(&dump_done_sigset);
sigaddset(&dump_done_sigset, ExceptionHandlerProtocol::kDumpDoneSignal);
ScopedSigprocmaskRestore scoped_block(dump_done_sigset);
int status = SendCrashDumpRequest(info, stack_pointer);
if (status != 0) {
return status;
}
#if defined(OS_ANDROID) && __ANDROID_API__ < 23
// sigtimedwait() wrappers aren't available on Android until API 23 but this
// can use the lss wrapper when it's available.
NOTREACHED();
#else
siginfo_t siginfo = {};
timespec timeout;
timeout.tv_sec = 5;
timeout.tv_nsec = 0;
if (HANDLE_EINTR(sigtimedwait(&dump_done_sigset, &siginfo, &timeout)) < 0) {
return errno;
}
#endif
return 0;
}
int ExceptionHandlerClient::SendCrashDumpRequest(
const ExceptionHandlerProtocol::ClientInformation& info,
VMAddress stack_pointer) {

View File

@ -28,7 +28,9 @@ class ExceptionHandlerClient {
//! \brief Constructs this object.
//!
//! \param[in] sock A socket connected to an ExceptionHandlerServer.
explicit ExceptionHandlerClient(int sock);
//! \param[in] multiple_clients `true` if this socket may be used by multiple
//! clients.
ExceptionHandlerClient(int sock, bool multiple_clients);
~ExceptionHandlerClient();
@ -56,11 +58,14 @@ class ExceptionHandlerClient {
int SendCrashDumpRequest(
const ExceptionHandlerProtocol::ClientInformation& info,
VMAddress stack_pointer);
int SignalCrashDump(const ExceptionHandlerProtocol::ClientInformation& info,
VMAddress stack_pointer);
int WaitForCrashDumpComplete();
int server_sock_;
pid_t ptracer_;
bool can_set_ptracer_;
bool multiple_clients_;
DISALLOW_COPY_AND_ASSIGN(ExceptionHandlerClient);
};

View File

@ -16,6 +16,7 @@
#define CRASHPAD_UTIL_LINUX_EXCEPTION_HANDLER_PROTOCOL_H_
#include <errno.h>
#include <signal.h>
#include <stdint.h>
#include <sys/types.h>
@ -51,6 +52,13 @@ class ExceptionHandlerProtocol {
VMAddress sanitization_information_address;
};
//! \brief The signal used to indicate a crash dump is complete.
//!
//! When multiple clients share a single socket connection with the handler,
//! the handler sends this signal to the dump requestor to indicate when the
//! the dump is either done or has failed and the client may continue.
static constexpr int kDumpDoneSignal = SIGCONT;
//! \brief The message passed from client to server.
struct ClientToServerMessage {
static constexpr int32_t kVersion = 1;

View File

@ -0,0 +1,59 @@
// Copyright 2019 The Crashpad Authors. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include "util/linux/proc_task_reader.h"
#include <stdio.h>
#include "base/files/file_path.h"
#include "base/logging.h"
#include "base/stl_util.h"
#include "base/strings/string_number_conversions.h"
#include "util/file/directory_reader.h"
#include "util/misc/as_underlying_type.h"
namespace crashpad {
bool ReadThreadIDs(pid_t pid, std::vector<pid_t>* tids) {
DCHECK(tids->empty());
char path[32];
snprintf(path, base::size(path), "/proc/%d/task", pid);
DirectoryReader reader;
if (!reader.Open(base::FilePath(path))) {
return false;
}
std::vector<pid_t> local_tids;
base::FilePath tid_str;
DirectoryReader::Result result;
while ((result = reader.NextFile(&tid_str)) ==
DirectoryReader::Result::kSuccess) {
pid_t tid;
if (!base::StringToInt(tid_str.value(), &tid)) {
LOG(ERROR) << "format error";
continue;
}
local_tids.push_back(tid);
}
DCHECK_EQ(AsUnderlyingType(result),
AsUnderlyingType(DirectoryReader::Result::kNoMoreFiles));
DCHECK(!local_tids.empty());
tids->swap(local_tids);
return true;
}
} // namespace crashpad

View File

@ -0,0 +1,34 @@
// Copyright 2019 The Crashpad Authors. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#ifndef CRASHPAD_UTIL_LINUX_PROC_TASK_READER_H_
#define CRASHPAD_UTIL_LINUX_PROC_TASK_READER_H_
#include <sys/types.h>
#include <vector>
namespace crashpad {
//! \brief Enumerates the thread IDs of a process by reading /proc/<pid>/task.
//!
//! \param[in] pid The process ID for which to read thread IDs.
//! \param[out] tids The read thread IDs.
//! \return `true` if the task directory was successfully read. Format errors
//! are logged, but won't cause this function to return `false`.
bool ReadThreadIDs(pid_t pid, std::vector<pid_t>* tids);
} // namespace crashpad
#endif // CRASHPAD_UTIL_LINUX_PROC_TASK_READER_H_

View File

@ -0,0 +1,162 @@
// Copyright 2019 The Crashpad Authors. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include "util/linux/proc_task_reader.h"
#include "base/logging.h"
#include "base/macros.h"
#include "base/strings/stringprintf.h"
#include "gtest/gtest.h"
#include "test/multiprocess_exec.h"
#include "third_party/lss/lss.h"
#include "util/synchronization/semaphore.h"
#include "util/thread/thread.h"
namespace crashpad {
namespace test {
namespace {
bool FindThreadID(pid_t tid, const std::vector<pid_t>& threads) {
for (const auto& thread : threads) {
if (thread == tid) {
return true;
}
}
return false;
}
class ScopedBlockingThread : public Thread {
public:
ScopedBlockingThread() : tid_sem_(0), join_sem_(0), tid_(-1) {}
~ScopedBlockingThread() {
join_sem_.Signal();
Join();
}
pid_t ThreadID() {
tid_sem_.Wait();
return tid_;
}
private:
void ThreadMain() override {
tid_ = sys_gettid();
tid_sem_.Signal();
join_sem_.Wait();
}
Semaphore tid_sem_;
Semaphore join_sem_;
pid_t tid_;
};
TEST(ProcTaskReader, Self) {
std::vector<pid_t> tids;
ASSERT_TRUE(ReadThreadIDs(getpid(), &tids));
EXPECT_TRUE(FindThreadID(getpid(), tids));
EXPECT_TRUE(FindThreadID(sys_gettid(), tids));
ScopedBlockingThread thread1;
thread1.Start();
ScopedBlockingThread thread2;
thread2.Start();
pid_t thread1_tid = thread1.ThreadID();
pid_t thread2_tid = thread2.ThreadID();
tids.clear();
ASSERT_TRUE(ReadThreadIDs(getpid(), &tids));
EXPECT_TRUE(FindThreadID(getpid(), tids));
EXPECT_TRUE(FindThreadID(thread1_tid, tids));
EXPECT_TRUE(FindThreadID(thread2_tid, tids));
}
TEST(ProcTaskReader, BadPID) {
std::vector<pid_t> tids;
EXPECT_FALSE(ReadThreadIDs(-1, &tids));
tids.clear();
EXPECT_FALSE(ReadThreadIDs(0, &tids));
}
CRASHPAD_CHILD_TEST_MAIN(ProcTaskTestChild) {
FileHandle in = StdioFileHandle(StdioStream::kStandardInput);
FileHandle out = StdioFileHandle(StdioStream::kStandardOutput);
pid_t tid = getpid();
CheckedWriteFile(out, &tid, sizeof(tid));
tid = sys_gettid();
CheckedWriteFile(out, &tid, sizeof(tid));
ScopedBlockingThread thread1;
thread1.Start();
ScopedBlockingThread thread2;
thread2.Start();
tid = thread1.ThreadID();
CheckedWriteFile(out, &tid, sizeof(tid));
tid = thread2.ThreadID();
CheckedWriteFile(out, &tid, sizeof(tid));
CheckedReadFileAtEOF(in);
return 0;
}
class ProcTaskTest : public MultiprocessExec {
public:
ProcTaskTest() : MultiprocessExec() {
SetChildTestMainFunction("ProcTaskTestChild");
}
private:
bool ReadIDFromChild(std::vector<pid_t>* threads) {
pid_t tid;
if (!LoggingReadFileExactly(ReadPipeHandle(), &tid, sizeof(tid))) {
return false;
}
threads->push_back(tid);
return true;
}
void MultiprocessParent() override {
std::vector<pid_t> ids_to_find;
for (size_t id_count = 0; id_count < 4; ++id_count) {
ASSERT_TRUE(ReadIDFromChild(&ids_to_find));
}
std::vector<pid_t> threads;
ASSERT_TRUE(ReadThreadIDs(ChildPID(), &threads));
for (size_t index = 0; index < ids_to_find.size(); ++index) {
SCOPED_TRACE(
base::StringPrintf("index %zd, tid %d", index, ids_to_find[index]));
EXPECT_TRUE(FindThreadID(ids_to_find[index], threads));
}
}
DISALLOW_COPY_AND_ASSIGN(ProcTaskTest);
};
TEST(ProcTaskReader, ReadChild) {
ProcTaskTest test;
test.Run();
}
} // namespace
} // namespace test
} // namespace crashpad

View File

@ -67,6 +67,8 @@
'linux/memory_map.h',
'linux/proc_stat_reader.cc',
'linux/proc_stat_reader.h',
'linux/proc_task_reader.cc',
'linux/proc_task_reader.h',
'linux/ptrace_broker.cc',
'linux/ptrace_broker.h',
'linux/ptrace_client.cc',

View File

@ -28,6 +28,7 @@
'../test/test.gyp:crashpad_test',
'../third_party/gtest/gmock.gyp:gmock',
'../third_party/gtest/gtest.gyp:gtest',
'../third_party/lss/lss.gyp:lss',
'../third_party/mini_chromium/mini_chromium.gyp:base',
'../third_party/zlib/zlib.gyp:zlib',
],
@ -44,6 +45,7 @@
'linux/auxiliary_vector_test.cc',
'linux/memory_map_test.cc',
'linux/proc_stat_reader_test.cc',
'linux/proc_task_reader_test.cc',
'linux/ptrace_broker_test.cc',
'linux/ptracer_test.cc',
'linux/scoped_ptrace_attach_test.cc',