Merge pull request #6529 from yandex/better-signals

Better debug-ability of fault signals.
This commit is contained in:
alexey-milovidov 2019-08-18 16:44:54 +03:00 committed by GitHub
commit f36d20915b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 123 additions and 64 deletions

View File

@ -172,7 +172,7 @@
M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \
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.") \
namespace ProfileEvents

View File

@ -11,12 +11,11 @@
#include <Common/Exception.h>
#include <Common/thread_local_rng.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>
namespace ProfileEvents
{
extern const Event QueryProfilerCannotWriteTrace;
extern const Event QueryProfilerSignalOverruns;
}
@ -27,36 +26,6 @@ extern LazyPipe trace_pipe;
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.
/// 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;
@ -90,7 +59,7 @@ namespace
sizeof(TimerType) + // timer type
sizeof(UInt32); // thread_number
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();
query_id.size = std::min(query_id.size, QUERY_ID_MAX_LEN);

View File

@ -151,6 +151,12 @@ std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext
}
break;
}
case SIGPROF:
{
error << "This is a signal used for debugging purposes by the user.";
break;
}
}
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)
return "<Empty trace>";
return callback("<Empty trace>");
const DB::SymbolIndex & symbol_index = DB::SymbolIndex::instance();
std::unordered_map<std::string, DB::Dwarf> dwarfs;
@ -281,12 +287,23 @@ static std::string toStringImpl(const StackTrace::Frames & frames, size_t offset
else
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();
}
void StackTrace::toStringEveryLine(std::function<void(const std::string &)> callback) const
{
toStringEveryLineImpl(frames, offset, size, std::move(callback));
}
std::string StackTrace::toString() const
{
/// Calculation of stack trace text is extremely slow.

View File

@ -4,6 +4,7 @@
#include <vector>
#include <array>
#include <optional>
#include <functional>
#include <signal.h>
#ifdef __APPLE__
@ -39,6 +40,8 @@ public:
const Frames & getFrames() const;
std::string toString() const;
void toStringEveryLine(std::function<void(const std::string &)> callback) const;
protected:
void tryCapture();

View File

@ -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;
}
}
}

View 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 {}
};
}

View File

@ -180,19 +180,22 @@ void AsynchronousMetrics::update()
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);
try
if (!status.is_readonly)
{
time_t absolute_delay = 0;
time_t relative_delay = 0;
table_replicated_merge_tree->getReplicaDelays(absolute_delay, relative_delay);
try
{
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_relative_delay, relative_delay);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__,
"Cannot get replica delay for table: " + backQuoteIfNeed(db.first) + "." + backQuoteIfNeed(iterator->name()));
calculateMax(max_absolute_delay, absolute_delay);
calculateMax(max_relative_delay, relative_delay);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__,
"Cannot get replica delay for table: " + backQuoteIfNeed(db.first) + "." + backQuoteIfNeed(iterator->name()));
}
}
calculateMax(max_part_count_for_partition, table_replicated_merge_tree->getMaxPartsCountForPartition());

View File

@ -38,7 +38,7 @@
#include <Poco/DirectoryIterator.h>
#include <Common/Exception.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/ReadHelpers.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.
*/
static void faultSignalHandler(int sig, siginfo_t * info, void * context)
{
if (already_signal_handled)
return;
already_signal_handled = true;
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 StackTrace stack_trace(signal_context);
@ -116,10 +110,12 @@ static void faultSignalHandler(int sig, siginfo_t * info, void * context)
out.next();
/// The time that is usually enough for separate thread to print info into log.
::sleep(10);
call_default_signal_handler(sig);
if (sig != SIGPROF) /// This signal is used for debugging.
{
/// The time that is usually enough for separate thread to print info into log.
::sleep(10);
call_default_signal_handler(sig);
}
}
@ -192,7 +188,9 @@ public:
DB::readPODBinary(stack_trace, 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);
}
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, "(version " << VERSION_STRING << VERSION_OFFICIAL << ") (from thread " << thread_num << ") "
<< "Received signal " << strsignal(sig) << " (" << sig << ")" << ".");
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({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler);