ClickHouse/dbms/Interpreters/SystemLog.cpp

114 lines
3.9 KiB
C++
Raw Normal View History

#include <Interpreters/SystemLog.h>
#include <Interpreters/QueryLog.h>
#include <Interpreters/QueryThreadLog.h>
#include <Interpreters/PartLog.h>
2019-07-22 13:54:08 +00:00
#include <Interpreters/TextLog.h>
2019-02-03 21:30:45 +00:00
#include <Interpreters/TraceLog.h>
2019-08-13 14:31:46 +00:00
#include <Interpreters/MetricLog.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <common/logger_useful.h>
namespace DB
{
2020-01-23 19:21:57 +00:00
2020-01-23 18:28:37 +00:00
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
namespace
{
constexpr size_t DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS = 7500;
/// Creates a system log with MergeTree engine using parameters from config
template <typename TSystemLog>
2019-03-21 19:22:38 +00:00
std::shared_ptr<TSystemLog> createSystemLog(
Context & context,
const String & default_database_name,
const String & default_table_name,
const Poco::Util::AbstractConfiguration & config,
const String & config_prefix)
{
if (!config.has(config_prefix))
return {};
String database = config.getString(config_prefix + ".database", default_database_name);
String table = config.getString(config_prefix + ".table", default_table_name);
if (database != default_database_name)
{
/// System tables must be loaded before other tables, but loading order is undefined for all databases except `system`
LOG_ERROR(&Logger::get("SystemLog"), "Custom database name for a system table specified in config. "
"Table `" << table << "` will be created in `system` database "
"instead of `" << database << "`");
database = default_database_name;
}
String engine;
2020-01-23 18:28:37 +00:00
if (config.has(config_prefix + ".engine"))
{
if (config.has(config_prefix + ".partition_by"))
throw Exception("If 'engine' is specified for system table, PARTITION BY parameters should be specified directly inside 'engine' and 'partition_by' setting doesn't make sense", ErrorCodes::BAD_ARGUMENTS);
engine = config.getString(config_prefix + ".engine");
}
else
{
String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)");
engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time)"
2020-03-28 01:20:18 +00:00
"SETTINGS min_bytes_for_wide_part = '10M'"; /// Use polymorphic parts for log tables by default
}
size_t flush_interval_milliseconds = config.getUInt64(config_prefix + ".flush_interval_milliseconds", DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS);
2019-03-21 19:22:38 +00:00
return std::make_shared<TSystemLog>(context, database, table, engine, flush_interval_milliseconds);
}
2019-08-13 16:17:18 +00:00
}
SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfiguration & config)
{
query_log = createSystemLog<QueryLog>(global_context, "system", "query_log", config, "query_log");
query_thread_log = createSystemLog<QueryThreadLog>(global_context, "system", "query_thread_log", config, "query_thread_log");
part_log = createSystemLog<PartLog>(global_context, "system", "part_log", config, "part_log");
trace_log = createSystemLog<TraceLog>(global_context, "system", "trace_log", config, "trace_log");
2019-08-04 15:51:04 +00:00
text_log = createSystemLog<TextLog>(global_context, "system", "text_log", config, "text_log");
2019-08-13 14:31:46 +00:00
metric_log = createSystemLog<MetricLog>(global_context, "system", "metric_log", config, "metric_log");
2018-12-14 16:12:12 +00:00
2019-08-13 14:31:46 +00:00
if (metric_log)
2019-08-13 16:47:12 +00:00
{
2019-08-14 12:54:41 +00:00
size_t collect_interval_milliseconds = config.getUInt64("metric_log.collect_interval_milliseconds");
2019-08-13 16:47:12 +00:00
metric_log->startCollectMetric(collect_interval_milliseconds);
}
}
2019-03-21 19:22:38 +00:00
SystemLogs::~SystemLogs()
2019-06-21 17:25:47 +00:00
{
shutdown();
}
void SystemLogs::shutdown()
2019-03-21 19:22:38 +00:00
{
if (query_log)
query_log->shutdown();
if (query_thread_log)
query_thread_log->shutdown();
if (part_log)
part_log->shutdown();
if (trace_log)
trace_log->shutdown();
2019-08-04 15:51:04 +00:00
if (text_log)
text_log->shutdown();
2019-08-13 14:31:46 +00:00
if (metric_log)
{
2019-08-13 16:47:12 +00:00
metric_log->stopCollectMetric();
2019-08-13 14:31:46 +00:00
metric_log->shutdown();
}
}
}