mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-09-20 08:40:50 +00:00
Add more logs to debug trace_log overflow
This commit is contained in:
parent
97c2e549be
commit
a54773f947
@ -542,7 +542,7 @@ case "$stage" in
|
||||
# to collect the logs. Prefer not to restart, because addresses might change
|
||||
# and we won't be able to process trace_log data. Start in a subshell, so that
|
||||
# it doesn't interfere with the watchdog through `wait`.
|
||||
( time get_profiles || restart || get_profiles ||: )
|
||||
( time get_profiles || restart || get_profiles ||: ) 2>> profile-errors.log
|
||||
|
||||
# Kill the whole process group, because somehow when the subshell is killed,
|
||||
# the sleep inside remains alive and orphaned.
|
||||
|
@ -165,9 +165,9 @@ private:
|
||||
// Queue is bounded. But its size is quite large to not block in all normal cases.
|
||||
std::vector<LogElement> queue;
|
||||
// An always-incrementing index of the first message currently in the queue.
|
||||
// We use it to give a global sequential index to every message, so that we can wait
|
||||
// until a particular message is flushed. This is used to implement synchronous log
|
||||
// flushing for SYSTEM FLUSH LOGS.
|
||||
// We use it to give a global sequential index to every message, so that we
|
||||
// can wait until a particular message is flushed. This is used to implement
|
||||
// synchronous log flushing for SYSTEM FLUSH LOGS.
|
||||
uint64_t queue_front_index = 0;
|
||||
bool is_shutdown = false;
|
||||
std::condition_variable flush_event;
|
||||
@ -175,6 +175,8 @@ private:
|
||||
uint64_t requested_flush_before = 0;
|
||||
// Flushed log up to this index, exclusive
|
||||
uint64_t flushed_before = 0;
|
||||
// Logged overflow message at this queue front index
|
||||
uint64_t logged_queue_full_at_index = -1;
|
||||
|
||||
void savingThreadFunction();
|
||||
|
||||
@ -244,9 +246,22 @@ void SystemLog<LogElement>::add(const LogElement & element)
|
||||
|
||||
if (queue.size() >= DBMS_SYSTEM_LOG_QUEUE_SIZE)
|
||||
{
|
||||
// TextLog sets its logger level to 0, so this log is a noop and there
|
||||
// is no recursive logging.
|
||||
LOG_ERROR(log, "Queue is full for system log '" + demangle(typeid(*this).name()) + "'.");
|
||||
// Ignore all further entries until the queue is flushed.
|
||||
// Log a message about that. Don't spam it -- this might be especially
|
||||
// problematic in case of trace log. Remember what the front index of the
|
||||
// queue was when we last logged the message. If it changed, it means the
|
||||
// queue was flushed, and we can log again.
|
||||
if (queue_front_index != logged_queue_full_at_index)
|
||||
{
|
||||
logged_queue_full_at_index = queue_front_index;
|
||||
|
||||
// TextLog sets its logger level to 0, so this log is a noop and
|
||||
// there is no recursive logging.
|
||||
LOG_ERROR(log, "Queue is full for system log '"
|
||||
<< demangle(typeid(*this).name()) << "'"
|
||||
<< " at " << queue_front_index);
|
||||
}
|
||||
|
||||
return;
|
||||
}
|
||||
|
||||
@ -325,9 +340,15 @@ void SystemLog<LogElement>::savingThreadFunction()
|
||||
uint64_t to_flush_end = 0;
|
||||
|
||||
{
|
||||
LOG_TRACE(log, "Sleeping");
|
||||
std::unique_lock lock(mutex);
|
||||
flush_event.wait_for(lock, std::chrono::milliseconds(flush_interval_milliseconds),
|
||||
[&] () { return requested_flush_before > flushed_before || is_shutdown; });
|
||||
const bool predicate = flush_event.wait_for(lock,
|
||||
std::chrono::milliseconds(flush_interval_milliseconds),
|
||||
[&] () {
|
||||
return requested_flush_before > flushed_before
|
||||
|| is_shutdown;
|
||||
}
|
||||
);
|
||||
|
||||
queue_front_index += queue.size();
|
||||
to_flush_end = queue_front_index;
|
||||
@ -337,6 +358,13 @@ void SystemLog<LogElement>::savingThreadFunction()
|
||||
queue.swap(to_flush);
|
||||
|
||||
exit_this_thread = is_shutdown;
|
||||
|
||||
LOG_TRACE(log, "Woke up"
|
||||
<< (predicate ? " by condition" : " by timeout ("
|
||||
+ toString(flush_interval_milliseconds) + " ms)")
|
||||
<< ", " << to_flush.size() << " elements to flush"
|
||||
<< " up to " << to_flush_end
|
||||
<< (is_shutdown ? ", shutdown requested" : ""));
|
||||
}
|
||||
|
||||
if (to_flush.empty())
|
||||
@ -351,6 +379,7 @@ void SystemLog<LogElement>::savingThreadFunction()
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
LOG_TRACE(log, "Terminating");
|
||||
}
|
||||
|
||||
|
||||
@ -359,11 +388,13 @@ void SystemLog<LogElement>::flushImpl(const std::vector<LogElement> & to_flush,
|
||||
{
|
||||
try
|
||||
{
|
||||
LOG_TRACE(log, "Flushing system log");
|
||||
LOG_TRACE(log, "Flushing system log, "
|
||||
<< to_flush.size() << " entries to flush");
|
||||
|
||||
/// We check for existence of the table and create it as needed at every flush.
|
||||
/// This is done to allow user to drop the table at any moment (new empty table will be created automatically).
|
||||
/// BTW, flush method is called from single thread.
|
||||
/// We check for existence of the table and create it as needed at every
|
||||
/// flush. This is done to allow user to drop the table at any moment
|
||||
/// (new empty table will be created automatically). BTW, flush method
|
||||
/// is called from single thread.
|
||||
prepareTable();
|
||||
|
||||
Block block = LogElement::createBlock();
|
||||
@ -389,9 +420,13 @@ void SystemLog<LogElement>::flushImpl(const std::vector<LogElement> & to_flush,
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
|
||||
std::unique_lock lock(mutex);
|
||||
flushed_before = to_flush_end;
|
||||
flush_event.notify_all();
|
||||
{
|
||||
std::unique_lock lock(mutex);
|
||||
flushed_before = to_flush_end;
|
||||
flush_event.notify_all();
|
||||
}
|
||||
|
||||
LOG_TRACE(log, "Flushed system log");
|
||||
}
|
||||
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user