2016-10-27 17:48:12 +00:00
|
|
|
#pragma once
|
|
|
|
|
2024-03-19 16:04:29 +00:00
|
|
|
#include <Interpreters/StorageID.h>
|
2022-01-25 09:58:11 +00:00
|
|
|
#include <Common/SystemLogBase.h>
|
2016-10-27 17:48:12 +00:00
|
|
|
|
2024-03-19 16:04:29 +00:00
|
|
|
#include <boost/noncopyable.hpp>
|
2022-01-10 19:01:41 +00:00
|
|
|
|
2016-10-27 17:48:12 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
/** Allow to store structured log in system table.
|
|
|
|
*
|
|
|
|
* Logging is asynchronous. Data is put into queue from where it will be read by separate thread.
|
|
|
|
* That thread inserts log into a table with no more than specified periodicity.
|
|
|
|
*/
|
|
|
|
|
|
|
|
/** Structure of log, template parameter.
|
|
|
|
* Structure could change on server version update.
|
|
|
|
* If on first write, existing table has different structure,
|
|
|
|
* then it get renamed (put aside) and new table is created.
|
|
|
|
*/
|
|
|
|
/* Example:
|
|
|
|
struct LogElement
|
|
|
|
{
|
|
|
|
/// default constructor must be available
|
|
|
|
/// fields
|
|
|
|
|
|
|
|
static std::string name();
|
2024-01-12 15:39:22 +00:00
|
|
|
static ColumnsDescription getColumnsDescription();
|
|
|
|
/// TODO: Remove this method, we can return aliases directly from getColumnsDescription().
|
2021-06-28 11:42:21 +00:00
|
|
|
static NamesAndAliases getNamesAndAliases();
|
2020-05-21 20:15:18 +00:00
|
|
|
void appendToBlock(MutableColumns & columns) const;
|
2016-10-27 17:48:12 +00:00
|
|
|
};
|
|
|
|
*/
|
|
|
|
|
2020-05-20 20:16:32 +00:00
|
|
|
class QueryLog;
|
|
|
|
class QueryThreadLog;
|
|
|
|
class PartLog;
|
|
|
|
class TextLog;
|
|
|
|
class TraceLog;
|
2020-07-09 04:15:45 +00:00
|
|
|
class CrashLog;
|
2020-05-20 20:16:32 +00:00
|
|
|
class MetricLog;
|
2020-06-10 19:17:30 +00:00
|
|
|
class AsynchronousMetricLog;
|
2020-08-20 20:59:40 +00:00
|
|
|
class OpenTelemetrySpanLog;
|
2021-06-18 13:44:08 +00:00
|
|
|
class QueryViewsLog;
|
2021-07-09 14:05:35 +00:00
|
|
|
class ZooKeeperLog;
|
2021-03-05 14:57:16 +00:00
|
|
|
class SessionLog;
|
2022-01-14 14:03:00 +00:00
|
|
|
class TransactionsInfoLog;
|
2022-02-05 16:33:42 +00:00
|
|
|
class ProcessorsProfileLog;
|
2022-04-30 05:00:40 +00:00
|
|
|
class FilesystemCacheLog;
|
2023-02-07 17:50:31 +00:00
|
|
|
class FilesystemReadPrefetchesLog;
|
2022-10-03 18:52:14 +00:00
|
|
|
class AsynchronousInsertLog;
|
2023-08-10 12:46:04 +00:00
|
|
|
class BackupLog;
|
2023-09-25 17:23:46 +00:00
|
|
|
class S3QueueLog;
|
2023-11-07 10:03:57 +00:00
|
|
|
class BlobStorageLog;
|
2020-04-13 01:33:05 +00:00
|
|
|
|
2018-03-01 16:52:24 +00:00
|
|
|
/// System logs should be destroyed in destructor of the last Context and before tables,
|
2017-06-05 13:59:38 +00:00
|
|
|
/// because SystemLog destruction makes insert query while flushing data into underlying tables
|
|
|
|
struct SystemLogs
|
|
|
|
{
|
2021-04-10 23:33:54 +00:00
|
|
|
SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConfiguration & config);
|
2017-06-05 13:59:38 +00:00
|
|
|
~SystemLogs();
|
|
|
|
|
2019-06-21 17:25:47 +00:00
|
|
|
void shutdown();
|
2023-07-28 07:23:34 +00:00
|
|
|
void handleCrash();
|
2019-06-21 17:25:47 +00:00
|
|
|
|
2019-03-21 19:22:38 +00:00
|
|
|
std::shared_ptr<QueryLog> query_log; /// Used to log queries.
|
|
|
|
std::shared_ptr<QueryThreadLog> query_thread_log; /// Used to log query threads.
|
|
|
|
std::shared_ptr<PartLog> part_log; /// Used to log operations with parts
|
2019-03-23 18:45:22 +00:00
|
|
|
std::shared_ptr<TraceLog> trace_log; /// Used to log traces from query profiler
|
2020-07-09 04:15:45 +00:00
|
|
|
std::shared_ptr<CrashLog> crash_log; /// Used to log server crashes.
|
2019-08-04 15:51:04 +00:00
|
|
|
std::shared_ptr<TextLog> text_log; /// Used to log all text messages.
|
2019-08-13 14:31:46 +00:00
|
|
|
std::shared_ptr<MetricLog> metric_log; /// Used to log all metrics.
|
2023-02-07 17:50:31 +00:00
|
|
|
std::shared_ptr<FilesystemCacheLog> filesystem_cache_log;
|
|
|
|
std::shared_ptr<FilesystemReadPrefetchesLog> filesystem_read_prefetches_log;
|
2023-09-25 17:23:46 +00:00
|
|
|
std::shared_ptr<S3QueueLog> s3_queue_log;
|
2020-06-10 19:17:30 +00:00
|
|
|
/// Metrics from system.asynchronous_metrics.
|
|
|
|
std::shared_ptr<AsynchronousMetricLog> asynchronous_metric_log;
|
2020-10-22 16:47:20 +00:00
|
|
|
/// OpenTelemetry trace spans.
|
|
|
|
std::shared_ptr<OpenTelemetrySpanLog> opentelemetry_span_log;
|
2021-06-18 13:44:08 +00:00
|
|
|
/// Used to log queries of materialized and live views
|
|
|
|
std::shared_ptr<QueryViewsLog> query_views_log;
|
2021-07-09 14:05:35 +00:00
|
|
|
/// Used to log all actions of ZooKeeper client
|
|
|
|
std::shared_ptr<ZooKeeperLog> zookeeper_log;
|
2021-03-05 14:57:16 +00:00
|
|
|
/// Login, LogOut and Login failure events
|
|
|
|
std::shared_ptr<SessionLog> session_log;
|
2022-01-14 14:03:00 +00:00
|
|
|
/// Events related to transactions
|
|
|
|
std::shared_ptr<TransactionsInfoLog> transactions_info_log;
|
2022-02-05 16:33:42 +00:00
|
|
|
/// Used to log processors profiling
|
|
|
|
std::shared_ptr<ProcessorsProfileLog> processors_profile_log;
|
2022-10-03 18:52:14 +00:00
|
|
|
std::shared_ptr<AsynchronousInsertLog> asynchronous_insert_log;
|
2023-08-10 12:46:04 +00:00
|
|
|
/// Backup and restore events
|
|
|
|
std::shared_ptr<BackupLog> backup_log;
|
2023-11-07 10:03:57 +00:00
|
|
|
/// Log blob storage operations
|
|
|
|
std::shared_ptr<BlobStorageLog> blob_storage_log;
|
2020-04-13 01:33:05 +00:00
|
|
|
|
|
|
|
std::vector<ISystemLog *> logs;
|
2017-06-05 13:59:38 +00:00
|
|
|
};
|
|
|
|
|
2023-07-28 07:23:34 +00:00
|
|
|
struct SystemLogSettings
|
|
|
|
{
|
|
|
|
SystemLogQueueSettings queue_settings;
|
|
|
|
|
|
|
|
String engine;
|
|
|
|
};
|
2017-06-05 13:59:38 +00:00
|
|
|
|
2016-10-27 17:48:12 +00:00
|
|
|
template <typename LogElement>
|
2023-07-19 06:15:57 +00:00
|
|
|
class SystemLog : public SystemLogBase<LogElement>, private boost::noncopyable, WithContext
|
2016-10-27 17:48:12 +00:00
|
|
|
{
|
|
|
|
public:
|
2018-09-13 14:59:03 +00:00
|
|
|
using Self = SystemLog;
|
2023-07-19 06:15:57 +00:00
|
|
|
using Base = SystemLogBase<LogElement>;
|
2018-05-31 15:54:08 +00:00
|
|
|
|
2016-10-27 17:48:12 +00:00
|
|
|
/** Parameter: table name where to write log.
|
|
|
|
* If table is not exists, then it get created with specified engine.
|
|
|
|
* If it already exists, then its structure is checked to be compatible with structure of log record.
|
|
|
|
* If it is compatible, then existing table will be used.
|
|
|
|
* If not - then existing table will be renamed to same name but with suffix '_N' at end,
|
|
|
|
* where N - is a minimal number from 1, for that table with corresponding name doesn't exist yet;
|
|
|
|
* and new table get created - as if previous table was not exist.
|
|
|
|
*/
|
2023-07-28 07:23:34 +00:00
|
|
|
SystemLog(ContextPtr context_,
|
2023-08-05 22:58:26 +00:00
|
|
|
const SystemLogSettings & settings_,
|
2023-07-28 07:23:34 +00:00
|
|
|
std::shared_ptr<SystemLogQueue<LogElement>> queue_ = nullptr);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2023-07-18 10:06:44 +00:00
|
|
|
/** Append a record into log.
|
|
|
|
* Writing to table will be done asynchronously and in case of failure, record could be lost.
|
|
|
|
*/
|
|
|
|
|
2022-01-10 19:01:41 +00:00
|
|
|
void shutdown() override;
|
2020-04-13 01:33:05 +00:00
|
|
|
|
2023-07-18 10:06:44 +00:00
|
|
|
void stopFlushThread() override;
|
2023-07-18 14:33:26 +00:00
|
|
|
|
2018-01-23 22:56:46 +00:00
|
|
|
protected:
|
2024-01-23 17:04:50 +00:00
|
|
|
LoggerPtr log;
|
2023-07-18 14:33:26 +00:00
|
|
|
|
2023-07-18 10:06:44 +00:00
|
|
|
using ISystemLog::is_shutdown;
|
|
|
|
using ISystemLog::saving_thread;
|
2023-07-19 08:46:57 +00:00
|
|
|
using ISystemLog::thread_mutex;
|
2023-07-19 06:15:57 +00:00
|
|
|
using Base::queue;
|
2020-01-31 15:45:58 +00:00
|
|
|
|
|
|
|
private:
|
|
|
|
/* Saving thread data */
|
2020-02-17 19:28:25 +00:00
|
|
|
const StorageID table_id;
|
2017-09-17 18:49:43 +00:00
|
|
|
const String storage_def;
|
2023-10-23 20:47:58 +00:00
|
|
|
const String create_query;
|
2021-11-25 18:06:04 +00:00
|
|
|
String old_create_query;
|
2020-01-31 15:45:58 +00:00
|
|
|
bool is_prepared = false;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2016-11-23 20:41:39 +00:00
|
|
|
/** Creates new table if it does not exist.
|
|
|
|
* Renames old table if its structure is not suitable.
|
|
|
|
* This cannot be done in constructor to avoid deadlock while renaming a table under locked Context when SystemLog object is created.
|
|
|
|
*/
|
2020-04-19 23:11:41 +00:00
|
|
|
void prepareTable() override;
|
2019-03-21 19:22:38 +00:00
|
|
|
|
2022-01-25 09:58:11 +00:00
|
|
|
void savingThreadFunction() override;
|
|
|
|
|
2019-06-19 12:28:34 +00:00
|
|
|
/// flushImpl can be executed only in saving_thread.
|
2020-01-31 15:45:58 +00:00
|
|
|
void flushImpl(const std::vector<LogElement> & to_flush, uint64_t to_flush_end);
|
2022-01-25 09:58:11 +00:00
|
|
|
ASTPtr getCreateTableQuery();
|
2016-10-27 17:48:12 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|