write traces to trace_log

This commit is contained in:
Nikita Lapkov 2019-02-03 21:30:45 +00:00
parent 89582cfef2
commit 5c54bbb750
23 changed files with 228 additions and 281 deletions

View File

@ -273,7 +273,7 @@ include (libs/libcommon/cmake/find_gperftools.cmake)
include (libs/libcommon/cmake/find_jemalloc.cmake)
include (libs/libcommon/cmake/find_cctz.cmake)
include (libs/libmysqlxx/cmake/find_mysqlclient.cmake)
include (libs/libdaemon/cmake/find_unwind.cmake)
include (libs/libcommon/cmake/find_unwind.cmake)
include (cmake/print_flags.cmake)

View File

@ -131,7 +131,7 @@ list (APPEND dbms_headers
list (APPEND dbms_sources src/TableFunctions/ITableFunction.cpp src/TableFunctions/TableFunctionFactory.cpp)
list (APPEND dbms_headers src/TableFunctions/ITableFunction.h src/TableFunctions/TableFunctionFactory.h)
add_library(clickhouse_common_io ${LINK_MODE} ${clickhouse_common_io_headers} ${clickhouse_common_io_sources} src/Common/Pipe.h)
add_library(clickhouse_common_io ${LINK_MODE} ${clickhouse_common_io_headers} ${clickhouse_common_io_sources})
if (OS_FREEBSD)
target_compile_definitions (clickhouse_common_io PUBLIC CLOCK_MONOTONIC_COARSE=CLOCK_MONOTONIC_FAST)

View File

@ -467,6 +467,9 @@ int Server::main(const std::vector<std::string> & /*args*/)
}
LOG_DEBUG(log, "Loaded metadata.");
/// Init trace collector only after trace_log system table was created
global_context->initializeTraceCollector();
global_context->setCurrentDatabase(default_database);
SCOPE_EXIT({

View File

@ -291,6 +291,14 @@
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</query_log>
<trace_log>
<database>system</database>
<table>trace_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</trace_log>
<!-- Uncomment if use part_log
<part_log>

View File

@ -13,6 +13,10 @@
#include <csignal>
namespace DB {
namespace ErrorCodes
{
extern const int CANNOT_PIPE;
}
struct Pipe
{
@ -31,8 +35,17 @@ namespace DB {
read_fd = -1;
write_fd = -1;
#ifndef __APPLE__
if (0 != pipe2(fds, O_CLOEXEC))
DB::throwFromErrno("Cannot create pipe", DB::ErrorCodes::CANNOT_PIPE);
#else
if (0 != pipe(fds))
DB::throwFromErrno("Cannot create pipe", 0);
DB::throwFromErrno("Cannot create pipe", DB::ErrorCodes::CANNOT_PIPE);
if (0 != fcntl(read_fd, F_SETFD, FD_CLOEXEC))
DB::throwFromErrno("Cannot create pipe", DB::ErrorCodes::CANNOT_PIPE);
if (0 != fcntl(write_fd, F_SETFD, FD_CLOEXEC))
DB::throwFromErrno("Cannot create pipe", DB::ErrorCodes::CANNOT_PIPE);
#endif
}
void close()
@ -48,8 +61,6 @@ namespace DB {
::close(write_fd);
write_fd = -1;
}
LOG_INFO(&Logger::get("TraceCollector"), "Pipe is closed");
}
~Pipe()
@ -57,9 +68,4 @@ namespace DB {
close();
}
};
class PipeSingleton : public ext::singleton<Pipe>
{
};
}
}

View File

@ -16,48 +16,11 @@ namespace DB
namespace ErrorCodes
{
// extern const int CANNOT_DLSYM;
// extern const int CANNOT_FORK;
// extern const int CANNOT_WAITPID;
// extern const int CHILD_WAS_NOT_EXITED_NORMALLY;
// extern const int CANNOT_CREATE_CHILD_PROCESS;
struct Pipe
{
int fds_rw[2];
Pipe()
{
#ifndef __APPLE__
if (0 != pipe2(fds_rw, O_CLOEXEC))
DB::throwFromErrno("Cannot create pipe", DB::ErrorCodes::CANNOT_PIPE);
#else
if (0 != pipe(fds_rw))
DB::throwFromErrno("Cannot create pipe", DB::ErrorCodes::CANNOT_PIPE);
if (0 != fcntl(fds_rw[0], F_SETFD, FD_CLOEXEC))
DB::throwFromErrno("Cannot create pipe", DB::ErrorCodes::CANNOT_PIPE);
if (0 != fcntl(fds_rw[1], F_SETFD, FD_CLOEXEC))
DB::throwFromErrno("Cannot create pipe", DB::ErrorCodes::CANNOT_PIPE);
#endif
}
~Pipe()
{
if (fds_rw[0] >= 0)
close(fds_rw[0]);
if (fds_rw[1] >= 0)
close(fds_rw[1]);
}
};
/// By these return codes from the child process, we learn (for sure) about errors when creating it.
enum class ReturnCodes : int
{
CANNOT_DUP_STDIN = 0x55555555, /// The value is not important, but it is chosen so that it's rare to conflict with the program return code.
CANNOT_DUP_STDOUT = 0x55555556,
CANNOT_DUP_STDERR = 0x55555557,
CANNOT_EXEC = 0x55555558,
};
extern const int CANNOT_DLSYM;
extern const int CANNOT_FORK;
extern const int CANNOT_WAITPID;
extern const int CHILD_WAS_NOT_EXITED_NORMALLY;
extern const int CANNOT_CREATE_CHILD_PROCESS;
}
/// By these return codes from the child process, we learn (for sure) about errors when creating it.
@ -118,15 +81,15 @@ std::unique_ptr<ShellCommand> ShellCommand::executeImpl(const char * filename, c
/// And there is a lot of garbage (including, for example, mutex is blocked). And this can not be done after `vfork` - deadlock happens.
/// Replace the file descriptors with the ends of our pipes.
if (STDIN_FILENO != dup2(pipe_stdin.fds_rw[0], STDIN_FILENO))
if (STDIN_FILENO != dup2(pipe_stdin.read_fd, STDIN_FILENO))
_exit(int(ReturnCodes::CANNOT_DUP_STDIN));
if (!pipe_stdin_only)
{
if (STDOUT_FILENO != dup2(pipe_stdout.fds_rw[1], STDOUT_FILENO))
if (STDOUT_FILENO != dup2(pipe_stdout.write_fd, STDOUT_FILENO))
_exit(int(ReturnCodes::CANNOT_DUP_STDOUT));
if (STDERR_FILENO != dup2(pipe_stderr.fds_rw[1], STDERR_FILENO))
if (STDERR_FILENO != dup2(pipe_stderr.write_fd, STDERR_FILENO))
_exit(int(ReturnCodes::CANNOT_DUP_STDERR));
}
@ -136,12 +99,12 @@ std::unique_ptr<ShellCommand> ShellCommand::executeImpl(const char * filename, c
_exit(int(ReturnCodes::CANNOT_EXEC));
}
std::unique_ptr<ShellCommand> res(new ShellCommand(pid, pipe_stdin.fds_rw[1], pipe_stdout.fds_rw[0], pipe_stderr.fds_rw[0], terminate_in_destructor));
std::unique_ptr<ShellCommand> res(new ShellCommand(pid, pipe_stdin.write_fd, pipe_stdout.read_fd, pipe_stderr.read_fd, terminate_in_destructor));
/// Now the ownership of the file descriptors is passed to the result.
pipe_stdin.fds_rw[1] = -1;
pipe_stdout.fds_rw[0] = -1;
pipe_stderr.fds_rw[0] = -1;
pipe_stdin.write_fd = -1;
pipe_stdout.read_fd = -1;
pipe_stderr.read_fd = -1;
return res;
}

View File

@ -41,6 +41,7 @@
#include <Interpreters/QueryLog.h>
#include <Interpreters/QueryThreadLog.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/TraceLog.h>
#include <Interpreters/Context.h>
#include <Common/DNSResolver.h>
#include <IO/ReadBufferFromFile.h>
@ -269,14 +270,15 @@ struct ContextShared
}
}
void initializeTraceCollector(TraceLog * trace_log) {
trace_collector.reset(new TraceCollector(trace_log));
trace_collector_thread.start(*trace_collector);
}
private:
void initialize()
{
security_manager = runtime_components_factory->createSecurityManager();
/// Set up trace collector for query profiler
trace_collector.reset(new TraceCollector());
trace_collector_thread.start(*trace_collector);
}
};
@ -1552,6 +1554,11 @@ void Context::initializeSystemLogs()
system_logs = std::make_shared<SystemLogs>(*global_context, getConfigRef());
}
void Context::initializeTraceCollector()
{
shared->initializeTraceCollector(getTraceLog());
}
QueryLog * Context::getQueryLog()
{
@ -1592,6 +1599,16 @@ PartLog * Context::getPartLog(const String & part_database)
return system_logs->part_log.get();
}
TraceLog * Context::getTraceLog()
{
auto lock = getLock();
if (!system_logs || !system_logs->trace_log)
return nullptr;
return system_logs->trace_log.get();
}
CompressionCodecPtr Context::chooseCompressionCodec(size_t part_size, double part_size_ratio) const
{

View File

@ -18,9 +18,6 @@
#include <Core/Block.h>
#include <Interpreters/Settings.h>
#include <Interpreters/ClientInfo.h>
#include <IO/CompressionSettings.h>
#include <Poco/Runnable.h>
#include <Poco/Thread.h>
namespace Poco
@ -63,6 +60,7 @@ class Clusters;
class QueryLog;
class QueryThreadLog;
class PartLog;
class TraceLog;
struct MergeTreeSettings;
class IDatabase;
class DDLGuard;
@ -138,6 +136,7 @@ private:
Context * global_context = nullptr; /// Global context or nullptr. Could be equal to this.
SystemLogsPtr system_logs; /// Used to log queries and operations on parts
UInt64 session_close_cycle = 0;
bool session_is_used = false;
@ -393,10 +392,14 @@ public:
/// Call after initialization before using system logs. Call for global context.
void initializeSystemLogs();
void initializeTraceCollector();
/// Nullptr if the query log is not ready for this moment.
QueryLog * getQueryLog();
QueryThreadLog * getQueryThreadLog();
TraceLog * getTraceLog();
/// Returns an object used to log opertaions with parts if it possible.
/// Provide table name to make required cheks.
PartLog * getPartLog(const String & part_database);

View File

@ -14,6 +14,7 @@
#include <Interpreters/QueryLog.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/QueryThreadLog.h>
#include <Interpreters/TraceLog.h>
#include <Databases/IDatabase.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/StorageFactory.h>
@ -209,7 +210,8 @@ BlockIO InterpreterSystemQuery::execute()
executeCommandsAndThrowIfError(
[&] () { if (auto query_log = context.getQueryLog()) query_log->flush(); },
[&] () { if (auto part_log = context.getPartLog("")) part_log->flush(); },
[&] () { if (auto query_thread_log = context.getQueryThreadLog()) query_thread_log->flush(); }
[&] () { if (auto query_thread_log = context.getQueryThreadLog()) query_thread_log->flush(); },
[&] () { if (auto trace_log = context.getTraceLog()) trace_log->flush(); }
);
break;
case Type::STOP_LISTEN_QUERIES:

View File

@ -2,6 +2,7 @@
#include <Interpreters/QueryLog.h>
#include <Interpreters/QueryThreadLog.h>
#include <Interpreters/PartLog.h>
#include <Interpreters/TraceLog.h>
#include <Poco/Util/AbstractConfiguration.h>
@ -14,6 +15,7 @@ SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfi
query_log = createDefaultSystemLog<QueryLog>(global_context, "system", "query_log", config, "query_log");
query_thread_log = createDefaultSystemLog<QueryThreadLog>(global_context, "system", "query_thread_log", config, "query_thread_log");
part_log = createDefaultSystemLog<PartLog>(global_context, "system", "part_log", config, "part_log");
trace_log = createDefaultSystemLog<TraceLog>(global_context, "system", "trace_log", config, "trace_log");
part_log_database = config.getString("part_log.database", "system");
}

View File

@ -58,6 +58,7 @@ class Context;
class QueryLog;
class QueryThreadLog;
class PartLog;
class TraceLog;
/// System logs should be destroyed in destructor of the last Context and before tables,
@ -70,6 +71,7 @@ struct SystemLogs
std::unique_ptr<QueryLog> query_log; /// Used to log queries.
std::unique_ptr<QueryThreadLog> query_thread_log; /// Used to log query threads.
std::unique_ptr<PartLog> part_log; /// Used to log operations with parts
std::unique_ptr<TraceLog> trace_log; /// Used to log traces from query profiler
String part_log_database;
};

View File

@ -130,16 +130,14 @@ void ThreadStatus::finalizePerformanceCounters()
}
namespace {
void queryProfilerTimerHandler(int sig, siginfo_t * /* info */, void * /* context */) {
LOG_INFO(&Logger::get("laplab"), "Hello from handler!");
void queryProfilerTimerHandler(int /* sig */, siginfo_t * /* info */, void * context) {
DB::WriteBufferFromFileDescriptor out(TracePipe::instance().write_fd);
char buffer[TraceCollector::buf_size];
DB::WriteBufferFromFileDescriptor out(PipeSingleton::instance().write_fd, TraceCollector::buf_size, buffer);
std::string queryID = CurrentThread::getCurrentQueryID();
DB::writeBinary(sig, out);
DB::writePODBinary(*reinterpret_cast<const ucontext_t *>(context), out);
DB::writeStringBinary(queryID, out);
out.next();
::sleep(10);
}
}
@ -149,9 +147,11 @@ void ThreadStatus::initQueryProfiler() {
return;
}
const auto pause_signal = SIGALRM;
struct sigevent sev;
sev.sigev_notify = SIGEV_THREAD_ID;
sev.sigev_signo = SIGALRM;
sev.sigev_signo = pause_signal;
sev._sigev_un._tid = os_thread_id;
// TODO(laplab): get clock type from settings
if (timer_create(CLOCK_REALTIME, &sev, &query_profiler_timer_id)) {
@ -174,11 +174,11 @@ void ThreadStatus::initQueryProfiler() {
throw Poco::Exception("Failed to clean signal mask for query profiler");
}
if (sigaddset(&sa.sa_mask, SIGALRM)) {
if (sigaddset(&sa.sa_mask, pause_signal)) {
throw Poco::Exception("Failed to add signal to mask for query profiler");
}
if (sigaction(SIGALRM, &sa, nullptr)) {
if (sigaction(pause_signal, &sa, nullptr)) {
throw Poco::Exception("Failed to setup signal handler for query profiler");
}

View File

@ -1,31 +1,48 @@
#include "TraceCollector.h"
#include <common/Backtrace.h>
#include <common/logger_useful.h>
#include <IO/ReadHelpers.h>
#include <ctime>
namespace DB {
const size_t TraceCollector::buf_size = sizeof(int);
const size_t TraceCollector::buf_size = sizeof(int) + sizeof(siginfo_t) + sizeof(ucontext_t);
TraceCollector::TraceCollector()
TraceCollector::TraceCollector(TraceLog * trace_log)
: log(&Logger::get("TraceCollector"))
, trace_log(trace_log)
{
}
void TraceCollector::run()
{
LOG_INFO(log, "TraceCollector started");
char buf[buf_size];
DB::ReadBufferFromFileDescriptor in(PipeSingleton::instance().read_fd, buf_size, buf);
LOG_INFO(log, "Preparing to read from: " << PipeSingleton::instance().read_fd);
DB::ReadBufferFromFileDescriptor in(TracePipe::instance().read_fd);
while (true)
{
int sig = 0;
DB::readBinary(sig, in);
ucontext_t context;
std::string queryID;
LOG_INFO(log, "Received signal: " << sig);
DB::readPODBinary(context, in);
DB::readStringBinary(queryID, in);
LOG_INFO(log, queryID);
if (trace_log != nullptr) {
std::vector<void *> frames = getBacktraceFrames(context);
std::vector<UInt64> trace;
trace.reserve(frames.size());
for (void * frame : frames) {
trace.push_back(reinterpret_cast<uintptr_t>(frame));
}
TraceLogElement element{std::time(nullptr), queryID, trace};
trace_log->add(element);
LOG_INFO(log, "TraceCollector added row");
}
}
LOG_INFO(log, "TraceCollector exited");

View File

@ -4,6 +4,8 @@
#include <Poco/Logger.h>
#include <Common/Pipe.h>
#include <ext/singleton.h>
#include <Interpreters/Context.h>
#include <Interpreters/TraceLog.h>
namespace DB
{
@ -14,12 +16,17 @@ namespace DB
{
private:
Logger * log;
TraceLog * trace_log;
public:
explicit TraceCollector();
explicit TraceCollector(TraceLog * trace_log);
void run() override;
static const size_t buf_size;
};
class TracePipe : public ext::singleton<Pipe>
{
};
}

View File

@ -0,0 +1,42 @@
#include <Interpreters/TraceLog.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
namespace DB
{
Block TraceLogElement::createBlock()
{
return
{
{std::make_shared<DataTypeDate>(), "event_date"},
{std::make_shared<DataTypeDateTime>(), "event_time"},
{std::make_shared<DataTypeString>(), "query_id"},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>()), "trace"}
};
}
void TraceLogElement::appendToBlock(Block &block) const
{
MutableColumns columns = block.mutateColumns();
size_t i = 0;
columns[i++]->insert(DateLUT::instance().toDayNum(event_time));
columns[i++]->insert(event_time);
columns[i++]->insertData(query_id.data(), query_id.size());
{
Array trace_array;
trace_array.reserve(trace.size());
for (const UInt32 trace_address : trace)
trace_array.emplace_back(UInt64(trace_address));
columns[i++]->insert(trace_array);
}
block.setColumns(std::move(columns));
}
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <Interpreters/SystemLog.h>
namespace DB
{
struct TraceLogElement
{
time_t event_time{};
String query_id{};
std::vector<UInt64> trace{};
static std::string name() { return "TraceLog"; }
static Block createBlock();
void appendToBlock(Block & block) const;
};
class TraceLog : public SystemLog<TraceLogElement>
{
using SystemLog<TraceLogElement>::SystemLog;
};
}

View File

@ -11,6 +11,7 @@ if (DEFINED APPLE_HAVE_CLOCK_GETTIME)
endif ()
add_library (common ${LINK_MODE}
src/Backtrace.cpp
src/DateLUT.cpp
src/DateLUTImpl.cpp
src/preciseExp10.c
@ -21,6 +22,7 @@ add_library (common ${LINK_MODE}
src/demangle.cpp
src/setTerminalEcho.cpp
include/common/Backtrace.h
include/common/Types.h
include/common/DayNum.h
include/common/DateLUT.h
@ -56,6 +58,12 @@ add_library (common ${LINK_MODE}
${CONFIG_COMMON}
)
if (USE_UNWIND)
target_compile_definitions (common PRIVATE USE_UNWIND=1)
target_include_directories (common BEFORE PRIVATE ${UNWIND_INCLUDE_DIR})
target_link_libraries (common PRIVATE ${UNWIND_LIBRARY})
endif ()
# When testing for memory leaks with Valgrind, dont link tcmalloc or jemalloc.
if (USE_JEMALLOC)

View File

@ -0,0 +1,25 @@
#pragma once
#include <string>
#include <vector>
#include <signal.h>
#if USE_UNWIND
#define UNW_LOCAL_ONLY
#include <libunwind.h>
#endif
#ifdef __APPLE__
// ucontext is not available without _XOPEN_SOURCE
#define _XOPEN_SOURCE
#endif
#include <ucontext.h>
std::string signalToErrorMessage(int sig, siginfo_t & info, ucontext_t & context);
void * getCallerAddress(ucontext_t & context);
std::vector<void *> getBacktraceFrames(ucontext_t & context);
std::string backtraceFramesToString(const std::vector<void *> & frames, const std::string delimiter = "");

View File

@ -1,73 +1,9 @@
#include <daemon/BaseDaemon.h>
#include <daemon/OwnFormattingChannel.h>
#include <daemon/OwnPatternFormatter.h>
#include <common/Backtrace.h>
#include <Common/Config/ConfigProcessor.h>
#include <daemon/OwnSplitChannel.h>
#include <sys/stat.h>
#include <sys/types.h>
#include <sys/fcntl.h>
#include <sys/time.h>
#include <errno.h>
#include <string.h>
#include <signal.h>
#include <sstream>
#include <cstring>
#include <cxxabi.h>
#include <execinfo.h>
#include <unistd.h>
#if USE_UNWIND
#define UNW_LOCAL_ONLY
#include <libunwind.h>
#endif
#ifdef __APPLE__
// ucontext is not available without _XOPEN_SOURCE
#define _XOPEN_SOURCE
#endif
#include <ucontext.h>
#include <typeinfo>
#include <common/logger_useful.h>
#include <common/ErrorHandlers.h>
#include <sys/time.h>
#include <sys/resource.h>
#include <iostream>
#include <fstream>
#include <sstream>
#include <memory>
#include <Poco/Observer.h>
#include <Poco/Logger.h>
#include <Poco/AutoPtr.h>
#include <Poco/Ext/LevelFilterChannel.h>
#include <Poco/Ext/ThreadNumber.h>
#include <Poco/PatternFormatter.h>
#include <Poco/ConsoleChannel.h>
#include <Poco/TaskManager.h>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Poco/Message.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Poco/Util/MapConfiguration.h>
#include <Poco/Util/Application.h>
#include <Poco/Exception.h>
#include <Poco/ErrorHandler.h>
#include <Poco/Condition.h>
#include <Poco/SyslogChannel.h>
#include <Poco/DirectoryIterator.h>
#include <Common/Exception.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Common/ClickHouseRevision.h>
#include <Common/config_version.h>
#include <daemon/OwnPatternFormatter.h>
#include <Common/CurrentThread.h>
#include <Poco/Net/RemoteSyslogChannel.h>
#if USE_UNWIND
std::vector<void *> backtraceLibUnwind(size_t max_frames, ucontext_t & context)
@ -273,7 +209,7 @@ std::vector<void *> getBacktraceFrames(ucontext_t & context)
return frames;
}
std::string backtraceFramesToString(const std::vector<void *> & frames)
std::string backtraceFramesToString(const std::vector<void *> & frames, const std::string delimiter)
{
std::stringstream backtrace;
char ** symbols = backtrace_symbols(frames.data(), frames.size());
@ -311,7 +247,7 @@ std::string backtraceFramesToString(const std::vector<void *> & frames)
backtrace << demangled_name << name_end;
}
else
backtrace << symbols[i];
backtrace << symbols[i] << delimiter;
}
}

View File

@ -5,7 +5,6 @@ add_library (daemon ${LINK_MODE}
src/OwnPatternFormatter.cpp
src/OwnFormattingChannel.cpp
src/OwnSplitChannel.cpp
src/Backtrace.cpp
include/daemon/BaseDaemon.h
include/daemon/GraphiteWriter.h
@ -13,14 +12,8 @@ add_library (daemon ${LINK_MODE}
include/daemon/OwnPatternFormatter.h
include/daemon/OwnFormattingChannel.h
include/daemon/OwnSplitChannel.h
include/daemon/Backtrace.h
include/daemon/Pipe.h)
if (USE_UNWIND)
target_compile_definitions (daemon PRIVATE USE_UNWIND=1)
target_include_directories (daemon BEFORE PRIVATE ${UNWIND_INCLUDE_DIR})
target_link_libraries (daemon PRIVATE ${UNWIND_LIBRARY})
endif ()
include/daemon/Pipe.h
)
target_include_directories (daemon PUBLIC include)
target_include_directories (daemon PRIVATE ${ClickHouse_SOURCE_DIR}/libs/libpocoext/include)

View File

@ -1,80 +0,0 @@
#pragma once
#include <daemon/BaseDaemon.h>
#include <daemon/OwnFormattingChannel.h>
#include <daemon/OwnPatternFormatter.h>
#include <Common/Config/ConfigProcessor.h>
#include <daemon/OwnSplitChannel.h>
#include <sys/stat.h>
#include <sys/types.h>
#include <sys/fcntl.h>
#include <sys/time.h>
#include <errno.h>
#include <string.h>
#include <signal.h>
#include <cxxabi.h>
#include <execinfo.h>
#include <unistd.h>
#if USE_UNWIND
#define UNW_LOCAL_ONLY
#include <libunwind.h>
#endif
#ifdef __APPLE__
// ucontext is not available without _XOPEN_SOURCE
#define _XOPEN_SOURCE
#endif
#include <ucontext.h>
#include <typeinfo>
#include <common/logger_useful.h>
#include <common/ErrorHandlers.h>
#include <sys/time.h>
#include <sys/resource.h>
#include <iostream>
#include <fstream>
#include <sstream>
#include <memory>
#include <Poco/Observer.h>
#include <Poco/Logger.h>
#include <Poco/AutoPtr.h>
#include <Poco/Ext/LevelFilterChannel.h>
#include <Poco/Ext/ThreadNumber.h>
#include <Poco/PatternFormatter.h>
#include <Poco/ConsoleChannel.h>
#include <Poco/TaskManager.h>
#include <Poco/File.h>
#include <Poco/Path.h>
#include <Poco/Message.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Poco/Util/MapConfiguration.h>
#include <Poco/Util/Application.h>
#include <Poco/Exception.h>
#include <Poco/ErrorHandler.h>
#include <Poco/Condition.h>
#include <Poco/SyslogChannel.h>
#include <Poco/DirectoryIterator.h>
#include <Common/Exception.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/getMultipleKeysFromConfig.h>
#include <Common/ClickHouseRevision.h>
#include <Common/config_version.h>
#include <daemon/OwnPatternFormatter.h>
#include <Common/CurrentThread.h>
#include <Poco/Net/RemoteSyslogChannel.h>
std::string signalToErrorMessage(int sig, siginfo_t & info, ucontext_t & context);
void * getCallerAddress(ucontext_t & context);
std::vector<void *> getBacktraceFrames(ucontext_t & context);
std::string backtraceFramesToString(const std::vector<void *> & frames);

View File

@ -1,5 +1,5 @@
#include <common/Backtrace.h>
#include <daemon/BaseDaemon.h>
#include <daemon/Backtrace.h>
#include <daemon/Pipe.h>
#include <daemon/OwnFormattingChannel.h>
#include <daemon/OwnPatternFormatter.h>
@ -18,11 +18,6 @@
#include <execinfo.h>
#include <unistd.h>
#if USE_UNWIND
#define UNW_LOCAL_ONLY
#include <libunwind.h>
#endif
#ifdef __APPLE__
// ucontext is not available without _XOPEN_SOURCE
#define _XOPEN_SOURCE
@ -134,33 +129,6 @@ static void faultSignalHandler(int sig, siginfo_t * info, void * context)
call_default_signal_handler(sig);
}
#if USE_UNWIND
size_t backtraceLibUnwind(void ** out_frames, size_t max_frames, ucontext_t & context)
{
unw_cursor_t cursor;
if (unw_init_local2(&cursor, &context, UNW_INIT_SIGNAL_FRAME) < 0)
return 0;
size_t i = 0;
for (; i < max_frames; ++i)
{
unw_word_t ip;
unw_get_reg(&cursor, UNW_REG_IP, &ip);
out_frames[i] = reinterpret_cast<void*>(ip);
/// NOTE This triggers "AddressSanitizer: stack-buffer-overflow". Looks like false positive.
/// It's Ok, because we use this method if the program is crashed nevertheless.
if (!unw_step(&cursor))
break;
}
return i;
}
#endif
/** The thread that read info about signal or std::terminate from pipe.
* On HUP / USR1, close log files (for new files to be opened later).
* On information about std::terminate, write it to log.