mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge pull request #6529 from yandex/better-signals
Better debug-ability of fault signals.
This commit is contained in:
commit
f36d20915b
@ -172,7 +172,7 @@
|
|||||||
M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \
|
M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \
|
||||||
M(CreatedHTTPConnections, "Total amount of created HTTP connections (closed or opened).") \
|
M(CreatedHTTPConnections, "Total amount of created HTTP connections (closed or opened).") \
|
||||||
\
|
\
|
||||||
M(QueryProfilerCannotWriteTrace, "Number of stack traces dropped by query profiler because pipe is full or cannot write to pipe.") \
|
M(CannotWriteToWriteBufferDiscard, "Number of stack traces dropped by query profiler or signal handler because pipe is full or cannot write to pipe.") \
|
||||||
M(QueryProfilerSignalOverruns, "Number of times we drop processing of a signal due to overrun plus the number of signals that OS has not delivered due to overrun.") \
|
M(QueryProfilerSignalOverruns, "Number of times we drop processing of a signal due to overrun plus the number of signals that OS has not delivered due to overrun.") \
|
||||||
|
|
||||||
namespace ProfileEvents
|
namespace ProfileEvents
|
||||||
|
@ -11,12 +11,11 @@
|
|||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
#include <Common/thread_local_rng.h>
|
#include <Common/thread_local_rng.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>
|
||||||
|
|
||||||
|
|
||||||
namespace ProfileEvents
|
namespace ProfileEvents
|
||||||
{
|
{
|
||||||
extern const Event QueryProfilerCannotWriteTrace;
|
|
||||||
extern const Event QueryProfilerSignalOverruns;
|
extern const Event QueryProfilerSignalOverruns;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -27,36 +26,6 @@ extern LazyPipe trace_pipe;
|
|||||||
|
|
||||||
namespace
|
namespace
|
||||||
{
|
{
|
||||||
/** Write to file descriptor but drop the data if write would block or fail.
|
|
||||||
* To use within signal handler. Motivating example: a signal handler invoked during execution of malloc
|
|
||||||
* should not block because some mutex (or even worse - a spinlock) may be held.
|
|
||||||
*/
|
|
||||||
class WriteBufferDiscardOnFailure : public WriteBufferFromFileDescriptor
|
|
||||||
{
|
|
||||||
protected:
|
|
||||||
void nextImpl() override
|
|
||||||
{
|
|
||||||
size_t bytes_written = 0;
|
|
||||||
while (bytes_written != offset())
|
|
||||||
{
|
|
||||||
ssize_t res = ::write(fd, working_buffer.begin() + bytes_written, offset() - bytes_written);
|
|
||||||
|
|
||||||
if ((-1 == res || 0 == res) && errno != EINTR)
|
|
||||||
{
|
|
||||||
ProfileEvents::increment(ProfileEvents::QueryProfilerCannotWriteTrace);
|
|
||||||
break; /// Discard
|
|
||||||
}
|
|
||||||
|
|
||||||
if (res > 0)
|
|
||||||
bytes_written += res;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public:
|
|
||||||
using WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor;
|
|
||||||
~WriteBufferDiscardOnFailure() override {}
|
|
||||||
};
|
|
||||||
|
|
||||||
/// Normally query_id is a UUID (string with a fixed length) but user can provide custom query_id.
|
/// Normally query_id is a UUID (string with a fixed length) but user can provide custom query_id.
|
||||||
/// Thus upper bound on query_id length should be introduced to avoid buffer overflow in signal handler.
|
/// Thus upper bound on query_id length should be introduced to avoid buffer overflow in signal handler.
|
||||||
constexpr size_t QUERY_ID_MAX_LEN = 1024;
|
constexpr size_t QUERY_ID_MAX_LEN = 1024;
|
||||||
@ -90,7 +59,7 @@ namespace
|
|||||||
sizeof(TimerType) + // timer type
|
sizeof(TimerType) + // timer type
|
||||||
sizeof(UInt32); // thread_number
|
sizeof(UInt32); // thread_number
|
||||||
char buffer[buf_size];
|
char buffer[buf_size];
|
||||||
WriteBufferDiscardOnFailure out(trace_pipe.fds_rw[1], buf_size, buffer);
|
WriteBufferFromFileDescriptorDiscardOnFailure out(trace_pipe.fds_rw[1], buf_size, buffer);
|
||||||
|
|
||||||
StringRef query_id = CurrentThread::getQueryId();
|
StringRef query_id = CurrentThread::getQueryId();
|
||||||
query_id.size = std::min(query_id.size, QUERY_ID_MAX_LEN);
|
query_id.size = std::min(query_id.size, QUERY_ID_MAX_LEN);
|
||||||
|
@ -151,6 +151,12 @@ std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext
|
|||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
case SIGPROF:
|
||||||
|
{
|
||||||
|
error << "This is a signal used for debugging purposes by the user.";
|
||||||
|
break;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return error.str();
|
return error.str();
|
||||||
@ -239,10 +245,10 @@ const StackTrace::Frames & StackTrace::getFrames() const
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
static std::string toStringImpl(const StackTrace::Frames & frames, size_t offset, size_t size)
|
static void toStringEveryLineImpl(const StackTrace::Frames & frames, size_t offset, size_t size, std::function<void(const std::string &)> callback)
|
||||||
{
|
{
|
||||||
if (size == 0)
|
if (size == 0)
|
||||||
return "<Empty trace>";
|
return callback("<Empty trace>");
|
||||||
|
|
||||||
const DB::SymbolIndex & symbol_index = DB::SymbolIndex::instance();
|
const DB::SymbolIndex & symbol_index = DB::SymbolIndex::instance();
|
||||||
std::unordered_map<std::string, DB::Dwarf> dwarfs;
|
std::unordered_map<std::string, DB::Dwarf> dwarfs;
|
||||||
@ -281,12 +287,23 @@ static std::string toStringImpl(const StackTrace::Frames & frames, size_t offset
|
|||||||
else
|
else
|
||||||
out << "?";
|
out << "?";
|
||||||
|
|
||||||
out << "\n";
|
callback(out.str());
|
||||||
|
out.str({});
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
static std::string toStringImpl(const StackTrace::Frames & frames, size_t offset, size_t size)
|
||||||
|
{
|
||||||
|
std::stringstream out;
|
||||||
|
toStringEveryLineImpl(frames, offset, size, [&](const std::string & str) { out << str << '\n'; });
|
||||||
return out.str();
|
return out.str();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
void StackTrace::toStringEveryLine(std::function<void(const std::string &)> callback) const
|
||||||
|
{
|
||||||
|
toStringEveryLineImpl(frames, offset, size, std::move(callback));
|
||||||
|
}
|
||||||
|
|
||||||
std::string StackTrace::toString() const
|
std::string StackTrace::toString() const
|
||||||
{
|
{
|
||||||
/// Calculation of stack trace text is extremely slow.
|
/// Calculation of stack trace text is extremely slow.
|
||||||
|
@ -4,6 +4,7 @@
|
|||||||
#include <vector>
|
#include <vector>
|
||||||
#include <array>
|
#include <array>
|
||||||
#include <optional>
|
#include <optional>
|
||||||
|
#include <functional>
|
||||||
#include <signal.h>
|
#include <signal.h>
|
||||||
|
|
||||||
#ifdef __APPLE__
|
#ifdef __APPLE__
|
||||||
@ -39,6 +40,8 @@ public:
|
|||||||
const Frames & getFrames() const;
|
const Frames & getFrames() const;
|
||||||
std::string toString() const;
|
std::string toString() const;
|
||||||
|
|
||||||
|
void toStringEveryLine(std::function<void(const std::string &)> callback) const;
|
||||||
|
|
||||||
protected:
|
protected:
|
||||||
void tryCapture();
|
void tryCapture();
|
||||||
|
|
||||||
|
@ -0,0 +1,29 @@
|
|||||||
|
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>
|
||||||
|
|
||||||
|
namespace ProfileEvents
|
||||||
|
{
|
||||||
|
extern const Event CannotWriteToWriteBufferDiscard;
|
||||||
|
}
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void WriteBufferFromFileDescriptorDiscardOnFailure::nextImpl()
|
||||||
|
{
|
||||||
|
size_t bytes_written = 0;
|
||||||
|
while (bytes_written != offset())
|
||||||
|
{
|
||||||
|
ssize_t res = ::write(fd, working_buffer.begin() + bytes_written, offset() - bytes_written);
|
||||||
|
|
||||||
|
if ((-1 == res || 0 == res) && errno != EINTR)
|
||||||
|
{
|
||||||
|
ProfileEvents::increment(ProfileEvents::CannotWriteToWriteBufferDiscard);
|
||||||
|
break; /// Discard
|
||||||
|
}
|
||||||
|
|
||||||
|
if (res > 0)
|
||||||
|
bytes_written += res;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
23
dbms/src/IO/WriteBufferFromFileDescriptorDiscardOnFailure.h
Normal file
23
dbms/src/IO/WriteBufferFromFileDescriptorDiscardOnFailure.h
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||||
|
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/** Write to file descriptor but drop the data if write would block or fail.
|
||||||
|
* To use within signal handler. Motivating example: a signal handler invoked during execution of malloc
|
||||||
|
* should not block because some mutex (or even worse - a spinlock) may be held.
|
||||||
|
*/
|
||||||
|
class WriteBufferFromFileDescriptorDiscardOnFailure : public WriteBufferFromFileDescriptor
|
||||||
|
{
|
||||||
|
protected:
|
||||||
|
void nextImpl() override;
|
||||||
|
|
||||||
|
public:
|
||||||
|
using WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor;
|
||||||
|
~WriteBufferFromFileDescriptorDiscardOnFailure() override {}
|
||||||
|
};
|
||||||
|
|
||||||
|
}
|
@ -180,19 +180,22 @@ void AsynchronousMetrics::update()
|
|||||||
calculateMaxAndSum(max_inserts_in_queue, sum_inserts_in_queue, status.queue.inserts_in_queue);
|
calculateMaxAndSum(max_inserts_in_queue, sum_inserts_in_queue, status.queue.inserts_in_queue);
|
||||||
calculateMaxAndSum(max_merges_in_queue, sum_merges_in_queue, status.queue.merges_in_queue);
|
calculateMaxAndSum(max_merges_in_queue, sum_merges_in_queue, status.queue.merges_in_queue);
|
||||||
|
|
||||||
try
|
if (!status.is_readonly)
|
||||||
{
|
{
|
||||||
time_t absolute_delay = 0;
|
try
|
||||||
time_t relative_delay = 0;
|
{
|
||||||
table_replicated_merge_tree->getReplicaDelays(absolute_delay, relative_delay);
|
time_t absolute_delay = 0;
|
||||||
|
time_t relative_delay = 0;
|
||||||
|
table_replicated_merge_tree->getReplicaDelays(absolute_delay, relative_delay);
|
||||||
|
|
||||||
calculateMax(max_absolute_delay, absolute_delay);
|
calculateMax(max_absolute_delay, absolute_delay);
|
||||||
calculateMax(max_relative_delay, relative_delay);
|
calculateMax(max_relative_delay, relative_delay);
|
||||||
}
|
}
|
||||||
catch (...)
|
catch (...)
|
||||||
{
|
{
|
||||||
tryLogCurrentException(__PRETTY_FUNCTION__,
|
tryLogCurrentException(__PRETTY_FUNCTION__,
|
||||||
"Cannot get replica delay for table: " + backQuoteIfNeed(db.first) + "." + backQuoteIfNeed(iterator->name()));
|
"Cannot get replica delay for table: " + backQuoteIfNeed(db.first) + "." + backQuoteIfNeed(iterator->name()));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
calculateMax(max_part_count_for_partition, table_replicated_merge_tree->getMaxPartsCountForPartition());
|
calculateMax(max_part_count_for_partition, table_replicated_merge_tree->getMaxPartsCountForPartition());
|
||||||
|
@ -38,7 +38,7 @@
|
|||||||
#include <Poco/DirectoryIterator.h>
|
#include <Poco/DirectoryIterator.h>
|
||||||
#include <Common/Exception.h>
|
#include <Common/Exception.h>
|
||||||
#include <IO/WriteBufferFromFile.h>
|
#include <IO/WriteBufferFromFile.h>
|
||||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>
|
||||||
#include <IO/ReadBufferFromFileDescriptor.h>
|
#include <IO/ReadBufferFromFileDescriptor.h>
|
||||||
#include <IO/ReadHelpers.h>
|
#include <IO/ReadHelpers.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
@ -92,18 +92,12 @@ static void terminateRequestedSignalHandler(int sig, siginfo_t * info, void * co
|
|||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
thread_local bool already_signal_handled = false;
|
|
||||||
|
|
||||||
/** Handler for "fault" signals. Send data about fault to separate thread to write into log.
|
/** Handler for "fault" signals. Send data about fault to separate thread to write into log.
|
||||||
*/
|
*/
|
||||||
static void faultSignalHandler(int sig, siginfo_t * info, void * context)
|
static void faultSignalHandler(int sig, siginfo_t * info, void * context)
|
||||||
{
|
{
|
||||||
if (already_signal_handled)
|
|
||||||
return;
|
|
||||||
already_signal_handled = true;
|
|
||||||
|
|
||||||
char buf[buf_size];
|
char buf[buf_size];
|
||||||
DB::WriteBufferFromFileDescriptor out(signal_pipe.fds_rw[1], buf_size, buf);
|
DB::WriteBufferFromFileDescriptorDiscardOnFailure out(signal_pipe.fds_rw[1], buf_size, buf);
|
||||||
|
|
||||||
const ucontext_t signal_context = *reinterpret_cast<ucontext_t *>(context);
|
const ucontext_t signal_context = *reinterpret_cast<ucontext_t *>(context);
|
||||||
const StackTrace stack_trace(signal_context);
|
const StackTrace stack_trace(signal_context);
|
||||||
@ -116,10 +110,12 @@ static void faultSignalHandler(int sig, siginfo_t * info, void * context)
|
|||||||
|
|
||||||
out.next();
|
out.next();
|
||||||
|
|
||||||
/// The time that is usually enough for separate thread to print info into log.
|
if (sig != SIGPROF) /// This signal is used for debugging.
|
||||||
::sleep(10);
|
{
|
||||||
|
/// The time that is usually enough for separate thread to print info into log.
|
||||||
call_default_signal_handler(sig);
|
::sleep(10);
|
||||||
|
call_default_signal_handler(sig);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ -192,7 +188,9 @@ public:
|
|||||||
DB::readPODBinary(stack_trace, in);
|
DB::readPODBinary(stack_trace, in);
|
||||||
DB::readBinary(thread_num, in);
|
DB::readBinary(thread_num, in);
|
||||||
|
|
||||||
onFault(sig, info, context, stack_trace, thread_num);
|
/// This allows to receive more signals if failure happens inside onFault function.
|
||||||
|
/// Example: segfault while symbolizing stack trace.
|
||||||
|
std::thread([=] { onFault(sig, info, context, stack_trace, thread_num); }).detach();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -207,14 +205,29 @@ private:
|
|||||||
LOG_FATAL(log, "(version " << VERSION_STRING << VERSION_OFFICIAL << ") (from thread " << thread_num << ") " << message);
|
LOG_FATAL(log, "(version " << VERSION_STRING << VERSION_OFFICIAL << ") (from thread " << thread_num << ") " << message);
|
||||||
}
|
}
|
||||||
|
|
||||||
void onFault(int sig, siginfo_t & info, ucontext_t & context, const StackTrace & stack_trace, ThreadNumber thread_num) const
|
void onFault(int sig, const siginfo_t & info, const ucontext_t & context, const StackTrace & stack_trace, ThreadNumber thread_num) const
|
||||||
{
|
{
|
||||||
LOG_FATAL(log, "########################################");
|
LOG_FATAL(log, "########################################");
|
||||||
LOG_FATAL(log, "(version " << VERSION_STRING << VERSION_OFFICIAL << ") (from thread " << thread_num << ") "
|
LOG_FATAL(log, "(version " << VERSION_STRING << VERSION_OFFICIAL << ") (from thread " << thread_num << ") "
|
||||||
<< "Received signal " << strsignal(sig) << " (" << sig << ")" << ".");
|
<< "Received signal " << strsignal(sig) << " (" << sig << ")" << ".");
|
||||||
|
|
||||||
LOG_FATAL(log, signalToErrorMessage(sig, info, context));
|
LOG_FATAL(log, signalToErrorMessage(sig, info, context));
|
||||||
LOG_FATAL(log, stack_trace.toString());
|
|
||||||
|
if (stack_trace.getSize())
|
||||||
|
{
|
||||||
|
/// Write bare stack trace (addresses) just in case if we will fail to print symbolized stack trace.
|
||||||
|
/// NOTE This still require memory allocations and mutex lock inside logger. BTW we can also print it to stderr using write syscalls.
|
||||||
|
|
||||||
|
std::stringstream bare_stacktrace;
|
||||||
|
bare_stacktrace << "Stack trace:";
|
||||||
|
for (size_t i = stack_trace.getOffset(); i < stack_trace.getSize(); ++i)
|
||||||
|
bare_stacktrace << ' ' << stack_trace.getFrames()[i];
|
||||||
|
|
||||||
|
LOG_FATAL(log, bare_stacktrace.rdbuf());
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Write symbolized stack trace line by line for better grep-ability.
|
||||||
|
stack_trace.toStringEveryLine([&](const std::string & s) { LOG_FATAL(log, s); });
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
@ -703,7 +716,9 @@ void BaseDaemon::initializeTerminationAndSignalProcessing()
|
|||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
add_signal_handler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE}, faultSignalHandler);
|
/// SIGPROF is added for debugging purposes. To output a stack trace of any running thread at anytime.
|
||||||
|
|
||||||
|
add_signal_handler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGPROF}, faultSignalHandler);
|
||||||
add_signal_handler({SIGHUP, SIGUSR1}, closeLogsSignalHandler);
|
add_signal_handler({SIGHUP, SIGUSR1}, closeLogsSignalHandler);
|
||||||
add_signal_handler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler);
|
add_signal_handler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler);
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user