ClickHouse/src/Storages/System/StorageSystemStackTrace.cpp

270 lines
9.3 KiB
C++
Raw Normal View History

2019-12-23 18:56:57 +00:00
#ifdef OS_LINUX /// Because of 'sigqueue' functions and RT signals.
#include <csignal>
2019-12-22 20:17:16 +00:00
#include <poll.h>
#include <mutex>
#include <filesystem>
2021-10-02 07:13:14 +00:00
#include <base/scope_guard.h>
#include <Storages/System/StorageSystemStackTrace.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <IO/ReadHelpers.h>
2021-05-17 04:44:57 +00:00
#include <IO/ReadBufferFromFile.h>
2019-12-22 20:17:16 +00:00
#include <Common/PipeFDs.h>
2020-07-09 01:00:16 +00:00
#include <Common/CurrentThread.h>
2021-10-02 07:13:14 +00:00
#include <base/getThreadId.h>
2022-04-27 15:05:45 +00:00
#include <Common/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_SIGQUEUE;
2019-12-22 20:17:16 +00:00
extern const int CANNOT_MANIPULATE_SIGSET;
extern const int CANNOT_SET_SIGNAL_HANDLER;
extern const int CANNOT_READ_FROM_FILE_DESCRIPTOR;
extern const int LOGICAL_ERROR;
}
namespace
{
2020-11-30 08:53:07 +00:00
// Initialized in StorageSystemStackTrace's ctor and used in signalHandler.
2020-12-01 12:33:42 +00:00
std::atomic<pid_t> expected_pid;
2019-12-22 20:17:16 +00:00
const int sig = SIGRTMIN;
2019-12-22 20:37:29 +00:00
2020-03-23 16:25:21 +00:00
std::atomic<int> sequence_num = 0; /// For messages sent via pipe.
2020-12-15 22:43:07 +00:00
std::atomic<int> data_ready_num = 0;
2020-12-23 01:40:23 +00:00
std::atomic<bool> signal_latch = false; /// Only need for thread sanitizer.
2019-12-23 19:23:54 +00:00
2020-12-15 22:43:07 +00:00
/** Notes:
* Only one query from the table can be processed at the moment of time.
* This is ensured by the mutex in fillData function.
* We obtain information about threads by sending signal and receiving info from the signal handler.
* Information is passed via global variables and pipe is used for signaling.
* Actually we can send all information via pipe, but we read from it with timeout just in case,
* so it's convenient to use is only for signaling.
*/
StackTrace stack_trace{NoCapture{}};
2019-12-22 20:37:29 +00:00
2020-03-09 03:14:24 +00:00
constexpr size_t max_query_id_size = 128;
2019-12-22 20:37:29 +00:00
char query_id_data[max_query_id_size];
size_t query_id_size = 0;
2019-12-22 20:17:16 +00:00
LazyPipeFDs notification_pipe;
2020-12-15 22:43:07 +00:00
void signalHandler(int, siginfo_t * info, void * context)
2019-12-22 20:17:16 +00:00
{
2021-01-12 14:34:50 +00:00
DENY_ALLOCATIONS_IN_SCOPE;
2020-03-01 14:23:43 +00:00
auto saved_errno = errno; /// We must restore previous value of errno in signal handler.
2019-12-22 20:17:16 +00:00
/// In case malicious user is sending signals manually (for unknown reason).
/// If we don't check - it may break our synchronization.
if (info->si_pid != expected_pid)
return;
2019-12-23 19:23:54 +00:00
/// Signal received too late.
2020-12-15 22:43:07 +00:00
int notification_num = info->si_value.sival_int;
if (notification_num != sequence_num.load(std::memory_order_acquire))
2019-12-23 19:23:54 +00:00
return;
2020-12-22 08:16:36 +00:00
bool expected = false;
if (!signal_latch.compare_exchange_strong(expected, true, std::memory_order_acquire))
return;
2019-12-22 20:17:16 +00:00
/// All these methods are signal-safe.
const ucontext_t signal_context = *reinterpret_cast<ucontext_t *>(context);
2020-12-15 22:43:07 +00:00
stack_trace = StackTrace(signal_context);
2019-12-22 20:37:29 +00:00
StringRef query_id = CurrentThread::getQueryId();
query_id_size = std::min(query_id.size, max_query_id_size);
if (query_id.data && query_id.size)
memcpy(query_id_data, query_id.data, query_id_size);
2019-12-22 20:37:29 +00:00
2020-12-15 22:43:07 +00:00
/// This is unneeded (because we synchronize through pipe) but makes TSan happy.
2020-12-22 08:16:36 +00:00
data_ready_num.store(notification_num, std::memory_order_release);
2020-12-15 22:43:07 +00:00
2019-12-23 19:23:54 +00:00
ssize_t res = ::write(notification_pipe.fds_rw[1], &notification_num, sizeof(notification_num));
2019-12-23 18:58:42 +00:00
2019-12-22 20:17:16 +00:00
/// We cannot do anything if write failed.
2019-12-23 18:58:42 +00:00
(void)res;
2020-03-01 14:23:43 +00:00
errno = saved_errno;
2020-12-22 08:16:36 +00:00
signal_latch.store(false, std::memory_order_release);
2019-12-22 20:17:16 +00:00
}
2019-12-22 20:37:29 +00:00
/// Wait for data in pipe and read it.
2020-12-15 22:43:07 +00:00
bool wait(int timeout_ms)
2019-12-22 20:17:16 +00:00
{
while (true)
{
2019-12-22 20:17:16 +00:00
int fd = notification_pipe.fds_rw[0];
pollfd poll_fd{fd, POLLIN, 0};
int poll_res = poll(&poll_fd, 1, timeout_ms);
if (poll_res < 0)
{
if (errno == EINTR)
{
--timeout_ms; /// Quite a hacky way to update timeout. Just to make sure we avoid infinite waiting.
if (timeout_ms == 0)
return false;
continue;
}
throwFromErrno("Cannot poll pipe", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
}
if (poll_res == 0)
return false;
2019-12-23 19:23:54 +00:00
int notification_num = 0;
ssize_t read_res = ::read(fd, &notification_num, sizeof(notification_num));
2019-12-22 20:17:16 +00:00
if (read_res < 0)
{
if (errno == EINTR)
continue;
throwFromErrno("Cannot read from pipe", ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
}
2019-12-23 19:23:54 +00:00
if (read_res == sizeof(notification_num))
{
2020-03-23 16:25:21 +00:00
if (notification_num == sequence_num.load(std::memory_order_relaxed))
2019-12-23 19:23:54 +00:00
return true;
else
continue; /// Drain delayed notifications.
}
throw Exception("Logical error: read wrong number of bytes from pipe", ErrorCodes::LOGICAL_ERROR);
}
2019-12-22 20:17:16 +00:00
}
}
2020-12-15 22:43:07 +00:00
StorageSystemStackTrace::StorageSystemStackTrace(const StorageID & table_id_)
2020-08-12 20:40:13 +00:00
: IStorageSystemOneBlock<StorageSystemStackTrace>(table_id_)
, log(&Poco::Logger::get("StorageSystemStackTrace"))
2019-12-22 20:17:16 +00:00
{
notification_pipe.open();
2019-12-22 20:17:16 +00:00
/// Setup signal handler.
2020-12-01 12:33:42 +00:00
expected_pid = getpid();
2019-12-22 20:17:16 +00:00
struct sigaction sa{};
sa.sa_sigaction = signalHandler;
sa.sa_flags = SA_SIGINFO;
2019-12-22 20:17:16 +00:00
if (sigemptyset(&sa.sa_mask))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_MANIPULATE_SIGSET);
2019-12-22 20:17:16 +00:00
if (sigaddset(&sa.sa_mask, sig))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_MANIPULATE_SIGSET);
2019-12-22 20:17:16 +00:00
if (sigaction(sig, &sa, nullptr))
throwFromErrno("Cannot set signal handler.", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER);
}
2019-12-22 20:17:16 +00:00
NamesAndTypesList StorageSystemStackTrace::getNamesAndTypes()
{
return
{
2021-05-17 04:44:57 +00:00
{ "thread_name", std::make_shared<DataTypeString>() },
{ "thread_id", std::make_shared<DataTypeUInt64>() },
2019-12-22 20:17:16 +00:00
{ "query_id", std::make_shared<DataTypeString>() },
{ "trace", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>()) }
};
}
2019-12-22 20:17:16 +00:00
void StorageSystemStackTrace::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const
{
2019-12-22 20:17:16 +00:00
/// It shouldn't be possible to do concurrent reads from this table.
std::lock_guard lock(mutex);
/// Send a signal to every thread and wait for result.
/// We must wait for every thread one by one sequentially,
/// because there is a limit on number of queued signals in OS and otherwise signals may get lost.
/// Also, non-RT signals are not delivered if previous signal is handled right now (by default; but we use RT signals).
2019-12-22 20:17:16 +00:00
/// Obviously, results for different threads may be out of sync.
2019-12-22 20:17:16 +00:00
/// There is no better way to enumerate threads in a process other than looking into procfs.
std::filesystem::directory_iterator end;
for (std::filesystem::directory_iterator it("/proc/self/task"); it != end; ++it)
{
pid_t tid = parse<pid_t>(it->path().filename());
2019-12-22 20:17:16 +00:00
2019-12-23 19:23:54 +00:00
sigval sig_value{};
2020-12-15 22:43:07 +00:00
sig_value.sival_int = sequence_num.load(std::memory_order_acquire);
2019-12-22 20:17:16 +00:00
if (0 != ::sigqueue(tid, sig, sig_value))
{
2019-12-22 20:17:16 +00:00
/// The thread may has been already finished.
if (ESRCH == errno)
continue;
throwFromErrno("Cannot send signal with sigqueue", ErrorCodes::CANNOT_SIGQUEUE);
}
2021-05-17 04:44:57 +00:00
std::filesystem::path thread_name_path = it->path();
thread_name_path.append("comm");
String thread_name;
if (std::filesystem::exists(thread_name_path))
{
2021-05-17 09:07:50 +00:00
constexpr size_t comm_buf_size = 32; /// More than enough for thread name
ReadBufferFromFile comm(thread_name_path.string(), comm_buf_size);
2021-07-11 00:34:00 +00:00
readEscapedStringUntilEOL(thread_name, comm);
2021-05-17 04:44:57 +00:00
comm.close();
}
2019-12-22 20:17:16 +00:00
/// Just in case we will wait for pipe with timeout. In case signal didn't get processed.
2019-12-22 20:37:29 +00:00
2020-12-15 22:43:07 +00:00
if (wait(100) && sig_value.sival_int == data_ready_num.load(std::memory_order_acquire))
2019-12-22 20:17:16 +00:00
{
2020-12-15 22:43:07 +00:00
size_t stack_trace_size = stack_trace.getSize();
size_t stack_trace_offset = stack_trace.getOffset();
2019-12-22 20:17:16 +00:00
Array arr;
arr.reserve(stack_trace_size - stack_trace_offset);
for (size_t i = stack_trace_offset; i < stack_trace_size; ++i)
2020-12-15 22:43:07 +00:00
arr.emplace_back(reinterpret_cast<intptr_t>(stack_trace.getFramePointers()[i]));
2019-12-22 20:17:16 +00:00
2021-05-17 04:44:57 +00:00
res_columns[0]->insert(thread_name);
res_columns[1]->insert(tid);
res_columns[2]->insertData(query_id_data, query_id_size);
res_columns[3]->insert(arr);
}
else
{
LOG_DEBUG(log, "Cannot obtain a stack trace for thread {}", tid);
2019-12-22 20:17:16 +00:00
/// Cannot obtain a stack trace. But create a record in result nevertheless.
2021-05-17 04:44:57 +00:00
res_columns[0]->insert(thread_name);
res_columns[1]->insert(tid);
2019-12-22 20:17:16 +00:00
res_columns[2]->insertDefault();
2021-05-17 04:44:57 +00:00
res_columns[3]->insertDefault();
}
2019-12-23 19:23:54 +00:00
2020-11-30 08:53:07 +00:00
/// Signed integer overflow is undefined behavior in both C and C++. However, according to
/// C++ standard, Atomic signed integer arithmetic is defined to use two's complement; there
/// are no undefined results. See https://en.cppreference.com/w/cpp/atomic/atomic and
/// http://eel.is/c++draft/atomics.types.generic#atomics.types.int-8
++sequence_num;
}
}
}
2019-12-23 18:56:57 +00:00
#endif