2022-01-25 09:58:11 +00:00
|
|
|
#include <Interpreters/AsynchronousMetricLog.h>
|
|
|
|
#include <Interpreters/CrashLog.h>
|
|
|
|
#include <Interpreters/MetricLog.h>
|
|
|
|
#include <Interpreters/OpenTelemetrySpanLog.h>
|
|
|
|
#include <Interpreters/PartLog.h>
|
|
|
|
#include <Interpreters/QueryLog.h>
|
|
|
|
#include <Interpreters/QueryThreadLog.h>
|
|
|
|
#include <Interpreters/QueryViewsLog.h>
|
|
|
|
#include <Interpreters/SessionLog.h>
|
|
|
|
#include <Interpreters/TextLog.h>
|
|
|
|
#include <Interpreters/TraceLog.h>
|
2022-04-30 05:00:40 +00:00
|
|
|
#include <Interpreters/FilesystemCacheLog.h>
|
2023-09-25 17:23:46 +00:00
|
|
|
#include <Interpreters/S3QueueLog.h>
|
2023-02-07 17:50:31 +00:00
|
|
|
#include <Interpreters/FilesystemReadPrefetchesLog.h>
|
2022-02-05 16:33:42 +00:00
|
|
|
#include <Interpreters/ProcessorsProfileLog.h>
|
2022-01-25 09:58:11 +00:00
|
|
|
#include <Interpreters/ZooKeeperLog.h>
|
2022-01-28 18:18:36 +00:00
|
|
|
#include <Interpreters/TransactionsInfoLog.h>
|
2022-10-03 18:52:14 +00:00
|
|
|
#include <Interpreters/AsynchronousInsertLog.h>
|
2023-08-10 12:46:04 +00:00
|
|
|
#include <Interpreters/BackupLog.h>
|
2023-11-07 10:03:57 +00:00
|
|
|
#include <IO/S3/BlobStorageLogWriter.h>
|
2022-01-25 09:58:11 +00:00
|
|
|
|
|
|
|
#include <Common/MemoryTrackerBlockerInThread.h>
|
|
|
|
#include <Common/SystemLogBase.h>
|
2023-04-06 17:00:51 +00:00
|
|
|
#include <Common/ThreadPool.h>
|
2022-01-25 09:58:11 +00:00
|
|
|
|
2022-04-27 15:05:45 +00:00
|
|
|
#include <Common/logger_useful.h>
|
2022-01-25 09:58:11 +00:00
|
|
|
#include <base/scope_guard.h>
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int TIMEOUT_EXCEEDED;
|
|
|
|
}
|
|
|
|
|
2023-04-06 17:00:51 +00:00
|
|
|
ISystemLog::~ISystemLog() = default;
|
|
|
|
|
2022-01-25 09:58:11 +00:00
|
|
|
|
2023-07-13 12:41:17 +00:00
|
|
|
template <typename LogElement>
|
2023-07-28 07:23:34 +00:00
|
|
|
SystemLogQueue<LogElement>::SystemLogQueue(const SystemLogQueueSettings & settings_)
|
|
|
|
: log(&Poco::Logger::get("SystemLogQueue (" + settings_.database + "." +settings_.table + ")"))
|
|
|
|
, settings(settings_)
|
|
|
|
|
2023-07-20 05:41:39 +00:00
|
|
|
{
|
2023-07-28 07:23:34 +00:00
|
|
|
queue.reserve(settings.reserved_size_rows);
|
|
|
|
|
|
|
|
if (settings.turn_off_logger)
|
2023-07-20 05:41:39 +00:00
|
|
|
log->setLevel(0);
|
|
|
|
}
|
2023-07-13 12:41:17 +00:00
|
|
|
|
2023-07-18 13:05:20 +00:00
|
|
|
static thread_local bool recursive_push_call = false;
|
2022-01-25 09:58:11 +00:00
|
|
|
|
|
|
|
template <typename LogElement>
|
2023-07-28 07:23:34 +00:00
|
|
|
void SystemLogQueue<LogElement>::push(LogElement&& element)
|
2022-01-25 09:58:11 +00:00
|
|
|
{
|
|
|
|
/// It is possible that the method will be called recursively.
|
|
|
|
/// Better to drop these events to avoid complications.
|
2023-07-18 13:05:20 +00:00
|
|
|
if (recursive_push_call)
|
2022-01-25 09:58:11 +00:00
|
|
|
return;
|
2023-07-18 13:05:20 +00:00
|
|
|
recursive_push_call = true;
|
|
|
|
SCOPE_EXIT({ recursive_push_call = false; });
|
2023-07-18 10:22:07 +00:00
|
|
|
|
|
|
|
/// Memory can be allocated while resizing on queue.push_back.
|
2022-01-25 09:58:11 +00:00
|
|
|
/// The size of allocation can be in order of a few megabytes.
|
|
|
|
/// But this should not be accounted for query memory usage.
|
|
|
|
/// Otherwise the tests like 01017_uniqCombined_memory_usage.sql will be flacky.
|
2022-07-26 15:22:00 +00:00
|
|
|
MemoryTrackerBlockerInThread temporarily_disable_memory_tracker;
|
2022-01-25 09:58:11 +00:00
|
|
|
|
|
|
|
/// Should not log messages under mutex.
|
2023-07-28 07:23:34 +00:00
|
|
|
bool buffer_size_rows_flush_threshold_exceeded = false;
|
2022-01-25 09:58:11 +00:00
|
|
|
|
|
|
|
{
|
|
|
|
std::unique_lock lock(mutex);
|
|
|
|
|
2023-07-14 10:02:18 +00:00
|
|
|
if (is_shutdown)
|
|
|
|
return;
|
2022-01-25 09:58:11 +00:00
|
|
|
|
2023-07-28 07:23:34 +00:00
|
|
|
if (queue.size() == settings.buffer_size_rows_flush_threshold)
|
2022-01-25 09:58:11 +00:00
|
|
|
{
|
2023-07-28 07:23:34 +00:00
|
|
|
buffer_size_rows_flush_threshold_exceeded = true;
|
2022-01-25 09:58:11 +00:00
|
|
|
|
|
|
|
// The queue more than half full, time to flush.
|
|
|
|
// We only check for strict equality, because messages are added one
|
|
|
|
// by one, under exclusive lock, so we will see each message count.
|
|
|
|
// It is enough to only wake the flushing thread once, after the message
|
|
|
|
// count increases past half available size.
|
|
|
|
const uint64_t queue_end = queue_front_index + queue.size();
|
|
|
|
if (requested_flush_up_to < queue_end)
|
|
|
|
requested_flush_up_to = queue_end;
|
|
|
|
|
|
|
|
flush_event.notify_all();
|
|
|
|
}
|
|
|
|
|
2023-07-28 07:23:34 +00:00
|
|
|
if (queue.size() >= settings.max_size_rows)
|
2022-01-25 09:58:11 +00:00
|
|
|
{
|
|
|
|
// 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.
|
|
|
|
lock.unlock();
|
2023-07-28 07:23:34 +00:00
|
|
|
LOG_ERROR(log, "Queue is full for system log '{}' at {}. max_size_rows {}",
|
|
|
|
demangle(typeid(*this).name()),
|
|
|
|
queue_front_index,
|
|
|
|
settings.max_size_rows);
|
2022-01-25 09:58:11 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2023-07-28 07:23:34 +00:00
|
|
|
queue.push_back(std::move(element));
|
2022-01-25 09:58:11 +00:00
|
|
|
}
|
|
|
|
|
2023-07-28 07:23:34 +00:00
|
|
|
if (buffer_size_rows_flush_threshold_exceeded)
|
|
|
|
LOG_INFO(log, "Queue is half full for system log '{}'. buffer_size_rows_flush_threshold {}",
|
|
|
|
demangle(typeid(*this).name()), settings.buffer_size_rows_flush_threshold);
|
2022-01-25 09:58:11 +00:00
|
|
|
}
|
|
|
|
|
2023-07-18 10:06:44 +00:00
|
|
|
template <typename LogElement>
|
2023-07-28 07:23:34 +00:00
|
|
|
void SystemLogQueue<LogElement>::handleCrash()
|
2023-07-18 13:05:20 +00:00
|
|
|
{
|
2023-07-28 07:23:34 +00:00
|
|
|
if (settings.notify_flush_on_crash)
|
2023-07-28 07:32:53 +00:00
|
|
|
notifyFlush(/* force */ true);
|
2023-07-18 10:06:44 +00:00
|
|
|
}
|
|
|
|
|
2022-01-25 09:58:11 +00:00
|
|
|
template <typename LogElement>
|
2023-07-18 13:05:20 +00:00
|
|
|
void SystemLogQueue<LogElement>::waitFlush(uint64_t expected_flushed_up_to)
|
2022-01-25 09:58:11 +00:00
|
|
|
{
|
|
|
|
// Use an arbitrary timeout to avoid endless waiting. 60s proved to be
|
|
|
|
// too fast for our parallel functional tests, probably because they
|
|
|
|
// heavily load the disk.
|
|
|
|
const int timeout_seconds = 180;
|
2023-07-17 16:33:04 +00:00
|
|
|
std::unique_lock lock(mutex);
|
|
|
|
bool result = flush_event.wait_for(lock, std::chrono::seconds(timeout_seconds), [&]
|
2022-01-25 09:58:11 +00:00
|
|
|
{
|
2023-07-18 13:05:20 +00:00
|
|
|
return flushed_up_to >= expected_flushed_up_to && !is_force_prepare_tables;
|
2022-01-25 09:58:11 +00:00
|
|
|
});
|
|
|
|
|
|
|
|
if (!result)
|
|
|
|
{
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Timeout exceeded ({} s) while flushing system log '{}'.",
|
|
|
|
toString(timeout_seconds), demangle(typeid(*this).name()));
|
2022-01-25 09:58:11 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-07-20 08:55:22 +00:00
|
|
|
template <typename LogElement>
|
|
|
|
uint64_t SystemLogQueue<LogElement>::notifyFlush(bool should_prepare_tables_anyway)
|
|
|
|
{
|
|
|
|
uint64_t this_thread_requested_offset;
|
|
|
|
|
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
if (is_shutdown)
|
|
|
|
return uint64_t(-1);
|
|
|
|
|
|
|
|
this_thread_requested_offset = queue_front_index + queue.size();
|
|
|
|
|
|
|
|
// Publish our flush request, taking care not to overwrite the requests
|
|
|
|
// made by other threads.
|
|
|
|
is_force_prepare_tables |= should_prepare_tables_anyway;
|
|
|
|
requested_flush_up_to = std::max(requested_flush_up_to, this_thread_requested_offset);
|
|
|
|
|
|
|
|
flush_event.notify_all();
|
|
|
|
}
|
|
|
|
|
|
|
|
LOG_DEBUG(log, "Requested flush up to offset {}", this_thread_requested_offset);
|
|
|
|
return this_thread_requested_offset;
|
|
|
|
}
|
|
|
|
|
2023-07-18 10:06:44 +00:00
|
|
|
template <typename LogElement>
|
|
|
|
void SystemLogQueue<LogElement>::confirm(uint64_t to_flush_end)
|
|
|
|
{
|
|
|
|
std::lock_guard lock(mutex);
|
|
|
|
flushed_up_to = to_flush_end;
|
|
|
|
is_force_prepare_tables = false;
|
|
|
|
flush_event.notify_all();
|
|
|
|
}
|
2023-07-04 02:32:08 +00:00
|
|
|
|
|
|
|
template <typename LogElement>
|
2023-07-28 07:23:34 +00:00
|
|
|
typename SystemLogQueue<LogElement>::Index SystemLogQueue<LogElement>::pop(std::vector<LogElement> & output,
|
|
|
|
bool & should_prepare_tables_anyway,
|
|
|
|
bool & exit_this_thread)
|
2022-01-25 09:58:11 +00:00
|
|
|
{
|
2023-07-18 10:06:44 +00:00
|
|
|
std::unique_lock lock(mutex);
|
2023-07-17 16:33:04 +00:00
|
|
|
flush_event.wait_for(lock,
|
2023-07-28 07:23:34 +00:00
|
|
|
std::chrono::milliseconds(settings.flush_interval_milliseconds),
|
2023-07-17 16:33:04 +00:00
|
|
|
[&] ()
|
|
|
|
{
|
|
|
|
return requested_flush_up_to > flushed_up_to || is_shutdown || is_force_prepare_tables;
|
|
|
|
}
|
|
|
|
);
|
2022-01-25 09:58:11 +00:00
|
|
|
|
2023-07-18 10:06:44 +00:00
|
|
|
queue_front_index += queue.size();
|
2023-07-17 16:33:04 +00:00
|
|
|
// Swap with existing array from previous flush, to save memory
|
|
|
|
// allocations.
|
|
|
|
output.resize(0);
|
2023-07-18 10:06:44 +00:00
|
|
|
queue.swap(output);
|
2022-01-25 09:58:11 +00:00
|
|
|
|
2023-07-17 16:33:04 +00:00
|
|
|
should_prepare_tables_anyway = is_force_prepare_tables;
|
2022-01-25 09:58:11 +00:00
|
|
|
|
2023-07-17 16:33:04 +00:00
|
|
|
exit_this_thread = is_shutdown;
|
2023-07-18 13:05:20 +00:00
|
|
|
return queue_front_index;
|
2023-07-17 16:33:04 +00:00
|
|
|
}
|
2022-01-25 09:58:11 +00:00
|
|
|
|
2023-07-18 10:22:07 +00:00
|
|
|
template <typename LogElement>
|
2023-07-18 13:05:20 +00:00
|
|
|
void SystemLogQueue<LogElement>::shutdown()
|
2023-07-18 14:33:26 +00:00
|
|
|
{
|
2023-07-20 05:41:39 +00:00
|
|
|
std::unique_lock lock(mutex);
|
2023-07-18 14:33:26 +00:00
|
|
|
is_shutdown = true;
|
2023-07-18 13:05:20 +00:00
|
|
|
/// Tell thread to shutdown.
|
|
|
|
flush_event.notify_all();
|
2023-07-18 10:22:07 +00:00
|
|
|
}
|
|
|
|
|
2023-07-19 06:15:57 +00:00
|
|
|
template <typename LogElement>
|
|
|
|
SystemLogBase<LogElement>::SystemLogBase(
|
2023-07-28 07:23:34 +00:00
|
|
|
const SystemLogQueueSettings & settings_,
|
2023-07-19 06:15:57 +00:00
|
|
|
std::shared_ptr<SystemLogQueue<LogElement>> queue_)
|
2023-07-28 07:23:34 +00:00
|
|
|
: queue(queue_ ? queue_ : std::make_shared<SystemLogQueue<LogElement>>(settings_))
|
|
|
|
{
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename LogElement>
|
|
|
|
void SystemLogBase<LogElement>::flush(bool force)
|
|
|
|
{
|
|
|
|
uint64_t this_thread_requested_offset = queue->notifyFlush(force);
|
|
|
|
if (this_thread_requested_offset == uint64_t(-1))
|
|
|
|
return;
|
|
|
|
|
|
|
|
queue->waitFlush(this_thread_requested_offset);
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename LogElement>
|
|
|
|
void SystemLogBase<LogElement>::handleCrash()
|
2023-07-19 06:15:57 +00:00
|
|
|
{
|
2023-07-28 07:23:34 +00:00
|
|
|
queue->handleCrash();
|
2023-07-19 06:15:57 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
template <typename LogElement>
|
|
|
|
void SystemLogBase<LogElement>::startup()
|
|
|
|
{
|
2023-07-19 08:46:57 +00:00
|
|
|
std::lock_guard lock(thread_mutex);
|
2023-07-19 06:15:57 +00:00
|
|
|
saving_thread = std::make_unique<ThreadFromGlobalPool>([this] { savingThreadFunction(); });
|
|
|
|
}
|
|
|
|
|
|
|
|
template <typename LogElement>
|
2023-07-28 07:23:34 +00:00
|
|
|
void SystemLogBase<LogElement>::add(LogElement element)
|
2023-07-19 06:15:57 +00:00
|
|
|
{
|
2023-07-28 07:23:34 +00:00
|
|
|
queue->push(std::move(element));
|
2023-07-19 06:15:57 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
template <typename LogElement>
|
|
|
|
void SystemLogBase<LogElement>::notifyFlush(bool force) { queue->notifyFlush(force); }
|
|
|
|
|
|
|
|
#define INSTANTIATE_SYSTEM_LOG_BASE(ELEMENT) template class SystemLogBase<ELEMENT>;
|
2022-01-25 09:58:11 +00:00
|
|
|
SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_BASE)
|
|
|
|
|
2023-07-19 06:15:57 +00:00
|
|
|
#define INSTANTIATE_SYSTEM_LOG_QUEUE(ELEMENT) template class SystemLogQueue<ELEMENT>;
|
|
|
|
SYSTEM_LOG_ELEMENTS(INSTANTIATE_SYSTEM_LOG_QUEUE)
|
|
|
|
|
2022-01-25 09:58:11 +00:00
|
|
|
}
|