mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Rename system.query_log_metric to system.query_metric_log
Seems I had at the very beginning a brainfart and started using query_log_metric instead of query_metric_log and it went on and on :facepalm
This commit is contained in:
parent
d735019ccd
commit
a561899da0
@ -2198,36 +2198,36 @@ If the table does not exist, ClickHouse will create it. If the structure of the
|
||||
</query_log>
|
||||
```
|
||||
|
||||
# query_log_metric {#query_log_metric}
|
||||
# query_metric_log {#query_metric_log}
|
||||
|
||||
It is disabled by default.
|
||||
|
||||
**Enabling**
|
||||
|
||||
To manually turn on metrics history collection [`system.query_log_metric`](../../operations/system-tables/query_log_metric.md), create `/etc/clickhouse-server/config.d/query_log_metric.xml` with the following content:
|
||||
To manually turn on metrics history collection [`system.query_metric_log`](../../operations/system-tables/query_metric_log.md), create `/etc/clickhouse-server/config.d/query_metric_log.xml` with the following content:
|
||||
|
||||
``` xml
|
||||
<clickhouse>
|
||||
<query_log_metric>
|
||||
<query_metric_log>
|
||||
<database>system</database>
|
||||
<table>query_log_metric</table>
|
||||
<table>query_metric_log</table>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
|
||||
<max_size_rows>1048576</max_size_rows>
|
||||
<reserved_size_rows>8192</reserved_size_rows>
|
||||
<buffer_size_rows_flush_threshold>524288</buffer_size_rows_flush_threshold>
|
||||
<flush_on_crash>false</flush_on_crash>
|
||||
</query_log_metric>
|
||||
</query_metric_log>
|
||||
</clickhouse>
|
||||
```
|
||||
|
||||
**Disabling**
|
||||
|
||||
To disable `query_log_metric` setting, you should create the following file `/etc/clickhouse-server/config.d/disable_query_log_metric.xml` with the following content:
|
||||
To disable `query_metric_log` setting, you should create the following file `/etc/clickhouse-server/config.d/disable_query_metric_log.xml` with the following content:
|
||||
|
||||
``` xml
|
||||
<clickhouse>
|
||||
<query_log_metric remove="1" />
|
||||
<query_metric_log remove="1" />
|
||||
</clickhouse>
|
||||
```
|
||||
|
||||
|
@ -1820,10 +1820,10 @@ Possible values:
|
||||
|
||||
Default value: 0 (no restriction).
|
||||
|
||||
## query_log_metric_interval (#query_log_metric_interval)
|
||||
## query_metric_log_interval (#query_metric_log_interval)
|
||||
|
||||
The interval in milliseconds at which the [query_log_metric](../../operations/system-tables/query_log_metric.md) for individual queries is collected.
|
||||
If set to 0, it will take the `collect_interval_milliseconds` from the [query_log_metric setting](../../operations/server-configuration-parameters/settings.md#query_log_metric).
|
||||
The interval in milliseconds at which the [query_metric_log](../../operations/system-tables/query_metric_log.md) for individual queries is collected.
|
||||
If set to 0, it will take the `collect_interval_milliseconds` from the [query_metric_log setting](../../operations/server-configuration-parameters/settings.md#query_metric_log).
|
||||
|
||||
Default value: 0
|
||||
|
||||
|
@ -1,11 +1,11 @@
|
||||
---
|
||||
slug: /en/operations/system-tables/query_log_metric
|
||||
slug: /en/operations/system-tables/query_metric_log
|
||||
---
|
||||
# query_log_metric
|
||||
# query_metric_log
|
||||
|
||||
Contains history of memory and metric values from table `system.events` for individual queries, periodically flushed to disk.
|
||||
|
||||
Once a query starts, data is collected at periodic intervals of `query_log_metric_interval` milliseconds (which is set to 1000
|
||||
Once a query starts, data is collected at periodic intervals of `query_metric_log_interval` milliseconds (which is set to 1000
|
||||
by default) and when the query finishes.
|
||||
|
||||
Columns:
|
||||
@ -18,7 +18,7 @@ Columns:
|
||||
**Example**
|
||||
|
||||
``` sql
|
||||
SELECT * FROM system.query_log_metric LIMIT 1 FORMAT Vertical;
|
||||
SELECT * FROM system.query_metric_log LIMIT 1 FORMAT Vertical;
|
||||
```
|
||||
|
||||
``` text
|
||||
@ -40,8 +40,8 @@ ProfileEvent_FailedSelectQuery: 0
|
||||
|
||||
**See also**
|
||||
|
||||
- [query_log_metric setting](../../operations/server-configuration-parameters/settings.md#query_log_metric) — Enabling and disabling the setting.
|
||||
- [query_log_metric_interval](../../operations/settings/settings.md#query_log_metric_interval)
|
||||
- [query_metric_log setting](../../operations/server-configuration-parameters/settings.md#query_metric_log) — Enabling and disabling the setting.
|
||||
- [query_metric_log_interval](../../operations/settings/settings.md#query_metric_log_interval)
|
||||
- [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md) — Contains periodically calculated metrics.
|
||||
- [system.events](../../operations/system-tables/events.md#system_tables-events) — Contains a number of events that occurred.
|
||||
- [system.metrics](../../operations/system-tables/metrics.md) — Contains instantly calculated metrics.
|
@ -1182,18 +1182,18 @@
|
||||
<flush_on_crash>false</flush_on_crash>
|
||||
</error_log>
|
||||
|
||||
<!-- Query log metric contains rows Contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk
|
||||
<!-- Query metric log contains rows Contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk
|
||||
every "collect_interval_milliseconds" interval-->
|
||||
<query_log_metric>
|
||||
<query_metric_log>
|
||||
<database>system</database>
|
||||
<table>query_log_metric</table>
|
||||
<table>query_metric_log</table>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
<max_size_rows>1048576</max_size_rows>
|
||||
<reserved_size_rows>8192</reserved_size_rows>
|
||||
<buffer_size_rows_flush_threshold>524288</buffer_size_rows_flush_threshold>
|
||||
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
|
||||
<flush_on_crash>false</flush_on_crash>
|
||||
</query_log_metric>
|
||||
</query_metric_log>
|
||||
|
||||
<!--
|
||||
Asynchronous metric log contains values of metrics from
|
||||
|
@ -743,10 +743,10 @@ error_log:
|
||||
flush_interval_milliseconds: 7500
|
||||
collect_interval_milliseconds: 1000
|
||||
|
||||
# Query log metric contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk.
|
||||
query_log_metric:
|
||||
# Query metric log contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk.
|
||||
query_metric_log:
|
||||
database: system
|
||||
table: query_log_metric
|
||||
table: query_metric_log
|
||||
flush_interval_milliseconds: 7500
|
||||
collect_interval_milliseconds: 1000
|
||||
|
||||
|
@ -4,7 +4,7 @@
|
||||
#include <Interpreters/MetricLog.h>
|
||||
#include <Interpreters/OpenTelemetrySpanLog.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/QueryLogMetric.h>
|
||||
#include <Interpreters/QueryMetricLog.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Interpreters/QueryThreadLog.h>
|
||||
#include <Interpreters/QueryViewsLog.h>
|
||||
|
@ -510,7 +510,7 @@ class IColumn;
|
||||
M(Bool, log_query_threads, false, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \
|
||||
M(Bool, log_query_views, true, "Log query dependent views into system.query_views_log table. This setting have effect only when 'log_queries' is true.", 0) \
|
||||
M(String, log_comment, "", "Log comment into system.query_log table and server log. It can be set to arbitrary string no longer than max_query_size.", 0) \
|
||||
M(UInt64, query_log_metric_interval, 0, "Periodic interval in milliseconds to collect query log metrics.", 0) \
|
||||
M(UInt64, query_metric_log_interval, 0, "Periodic interval in milliseconds to collect query metric logs.", 0) \
|
||||
M(LogsLevel, send_logs_level, LogsLevel::fatal, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \
|
||||
M(String, send_logs_source_regexp, "", "Send server text logs with specified regexp to match log source name. Empty means all sources.", 0) \
|
||||
M(Bool, enable_optimize_predicate_expression, true, "If it is set to true, optimize predicates to subqueries.", 0) \
|
||||
|
@ -81,7 +81,7 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
|
||||
{"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."},
|
||||
{"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."},
|
||||
{"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."},
|
||||
{"query_log_metric_interval", 0, 0, "New setting."},
|
||||
{"query_metric_log_interval", 0, 0, "New setting."},
|
||||
}},
|
||||
{"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"},
|
||||
{"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"},
|
||||
|
@ -4097,14 +4097,14 @@ std::shared_ptr<QueryLog> Context::getQueryLog() const
|
||||
return shared->system_logs->query_log;
|
||||
}
|
||||
|
||||
std::shared_ptr<QueryLogMetric> Context::getQueryLogMetric() const
|
||||
std::shared_ptr<QueryMetricLog> Context::getQueryMetricLog() const
|
||||
{
|
||||
SharedLockGuard lock(shared->mutex);
|
||||
|
||||
if (!shared->system_logs)
|
||||
return {};
|
||||
|
||||
return shared->system_logs->query_log_metric;
|
||||
return shared->system_logs->query_metric_log;
|
||||
}
|
||||
|
||||
std::shared_ptr<QueryThreadLog> Context::getQueryThreadLog() const
|
||||
|
@ -93,7 +93,7 @@ class Clusters;
|
||||
class QueryCache;
|
||||
class ISystemLog;
|
||||
class QueryLog;
|
||||
class QueryLogMetric;
|
||||
class QueryMetricLog;
|
||||
class QueryThreadLog;
|
||||
class QueryViewsLog;
|
||||
class PartLog;
|
||||
@ -1151,7 +1151,7 @@ public:
|
||||
std::shared_ptr<AsynchronousInsertLog> getAsynchronousInsertLog() const;
|
||||
std::shared_ptr<BackupLog> getBackupLog() const;
|
||||
std::shared_ptr<BlobStorageLog> getBlobStorageLog() const;
|
||||
std::shared_ptr<QueryLogMetric> getQueryLogMetric() const;
|
||||
std::shared_ptr<QueryMetricLog> getQueryMetricLog() const;
|
||||
|
||||
std::vector<ISystemLog *> getSystemLogs() const;
|
||||
|
||||
|
@ -2,7 +2,7 @@
|
||||
#include <Interpreters/ErrorLog.h>
|
||||
#include <Interpreters/MetricLog.h>
|
||||
#include <Interpreters/PeriodicLog.h>
|
||||
#include <Interpreters/QueryLogMetric.h>
|
||||
#include <Interpreters/QueryMetricLog.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -9,7 +9,7 @@
|
||||
#define SYSTEM_PERIODIC_LOG_ELEMENTS(M) \
|
||||
M(ErrorLogElement) \
|
||||
M(MetricLogElement) \
|
||||
M(QueryLogMetricElement)
|
||||
M(QueryMetricLogElement)
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -10,7 +10,7 @@
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/QueryLogMetric.h>
|
||||
#include <Interpreters/QueryMetricLog.h>
|
||||
#include <Interpreters/PeriodicLog.h>
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
@ -30,7 +30,7 @@ namespace DB
|
||||
|
||||
const auto memory_metrics = std::array{CurrentMetrics::MemoryTracking, CurrentMetrics::MergesMutationsMemoryTracking};
|
||||
|
||||
ColumnsDescription QueryLogMetricElement::getColumnsDescription()
|
||||
ColumnsDescription QueryMetricLogElement::getColumnsDescription()
|
||||
{
|
||||
ColumnsDescription result;
|
||||
ParserCodec codec_parser;
|
||||
@ -72,7 +72,7 @@ ColumnsDescription QueryLogMetricElement::getColumnsDescription()
|
||||
return result;
|
||||
}
|
||||
|
||||
void QueryLogMetricElement::appendToBlock(MutableColumns & columns) const
|
||||
void QueryMetricLogElement::appendToBlock(MutableColumns & columns) const
|
||||
{
|
||||
size_t column_idx = 0;
|
||||
|
||||
@ -88,9 +88,9 @@ void QueryLogMetricElement::appendToBlock(MutableColumns & columns) const
|
||||
columns[column_idx++]->insert(profile_events[i]);
|
||||
}
|
||||
|
||||
void QueryLogMetric::startQuery(const String & query_id, TimePoint query_start_time, UInt64 interval_milliseconds)
|
||||
void QueryMetricLog::startQuery(const String & query_id, TimePoint query_start_time, UInt64 interval_milliseconds)
|
||||
{
|
||||
QueryLogMetricStatus status;
|
||||
QueryMetricLogStatus status;
|
||||
status.query_id = query_id;
|
||||
status.interval_milliseconds = interval_milliseconds;
|
||||
status.next_collect_time = query_start_time + std::chrono::milliseconds(interval_milliseconds);
|
||||
@ -111,16 +111,16 @@ void QueryLogMetric::startQuery(const String & query_id, TimePoint query_start_t
|
||||
}
|
||||
}
|
||||
|
||||
void QueryLogMetric::finishQuery(const String & query_id)
|
||||
void QueryMetricLog::finishQuery(const String & query_id)
|
||||
{
|
||||
std::lock_guard lock(queries_mutex);
|
||||
auto & queries_by_id = queries.get<ByQueryId>();
|
||||
queries_by_id.erase(query_id);
|
||||
}
|
||||
|
||||
void QueryLogMetric::threadFunction()
|
||||
void QueryMetricLog::threadFunction()
|
||||
{
|
||||
setThreadName("QueryLogMetric");
|
||||
setThreadName("QueryMetricLog");
|
||||
auto desired_timepoint = std::chrono::system_clock::now();
|
||||
while (!is_shutdown_metric_thread)
|
||||
{
|
||||
@ -152,18 +152,18 @@ void QueryLogMetric::threadFunction()
|
||||
}
|
||||
}
|
||||
|
||||
QueryLogMetricElement QueryLogMetric::createLogMetricElement(const String & query_id, std::shared_ptr<ProfileEvents::Counters::Snapshot> profile_counters, PeriodicLog<QueryLogMetricElement>::TimePoint current_time)
|
||||
QueryMetricLogElement QueryMetricLog::createLogMetricElement(const String & query_id, std::shared_ptr<ProfileEvents::Counters::Snapshot> profile_counters, PeriodicLog<QueryMetricLogElement>::TimePoint current_time)
|
||||
{
|
||||
auto query_status_it = queries.find(query_id);
|
||||
|
||||
QueryLogMetricElement elem;
|
||||
QueryMetricLogElement elem;
|
||||
elem.event_time = timeInSeconds(current_time);
|
||||
elem.event_time_microseconds = timeInMicroseconds(current_time);
|
||||
elem.query_id = query_status_it->query_id;
|
||||
elem.memory = CurrentMetrics::values[CurrentMetrics::MemoryTracking];
|
||||
elem.background_memory = CurrentMetrics::values[CurrentMetrics::MergesMutationsMemoryTracking];
|
||||
|
||||
// We copy the QueryLogMetricStatus and update the queries in a final step because updating the multi-index set
|
||||
// We copy the QueryMetricLogStatus and update the queries in a final step because updating the multi-index set
|
||||
// for every profile event doesn't seem a good idea.
|
||||
auto new_query_status = *query_status_it;
|
||||
new_query_status.next_collect_time += std::chrono::milliseconds(new_query_status.interval_milliseconds);
|
||||
@ -175,12 +175,12 @@ QueryLogMetricElement QueryLogMetric::createLogMetricElement(const String & quer
|
||||
new_query_status.last_profile_events[i] = new_value;
|
||||
}
|
||||
|
||||
queries.modify(query_status_it, [&](QueryLogMetricStatus & query_status) { query_status = std::move(new_query_status); });
|
||||
queries.modify(query_status_it, [&](QueryMetricLogStatus & query_status) { query_status = std::move(new_query_status); });
|
||||
|
||||
return elem;
|
||||
}
|
||||
|
||||
void QueryLogMetric::stepFunction(TimePoint current_time)
|
||||
void QueryMetricLog::stepFunction(TimePoint current_time)
|
||||
{
|
||||
static const auto & process_list = context->getProcessList();
|
||||
|
@ -20,10 +20,10 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** QueryLogMetricElement is a log of query metric values measured at regular time interval.
|
||||
/** QueryMetricLogElement is a log of query metric values measured at regular time interval.
|
||||
*/
|
||||
|
||||
struct QueryLogMetricElement
|
||||
struct QueryMetricLogElement
|
||||
{
|
||||
time_t event_time{};
|
||||
Decimal64 event_time_microseconds{};
|
||||
@ -32,38 +32,38 @@ struct QueryLogMetricElement
|
||||
Int64 background_memory{};
|
||||
std::vector<ProfileEvents::Count> profile_events = std::vector<ProfileEvents::Count>(ProfileEvents::end());
|
||||
|
||||
static std::string name() { return "QueryLogMetric"; }
|
||||
static std::string name() { return "QueryMetricLog"; }
|
||||
static ColumnsDescription getColumnsDescription();
|
||||
static NamesAndAliases getNamesAndAliases() { return {}; }
|
||||
void appendToBlock(MutableColumns & columns) const;
|
||||
};
|
||||
|
||||
struct QueryLogMetricStatus
|
||||
struct QueryMetricLogStatus
|
||||
{
|
||||
String query_id;
|
||||
UInt64 interval_milliseconds;
|
||||
std::chrono::system_clock::time_point next_collect_time;
|
||||
std::vector<ProfileEvents::Count> last_profile_events = std::vector<ProfileEvents::Count>(ProfileEvents::end());
|
||||
|
||||
bool operator<(const QueryLogMetricStatus & other) const
|
||||
bool operator<(const QueryMetricLogStatus & other) const
|
||||
{
|
||||
return next_collect_time < other.next_collect_time;
|
||||
}
|
||||
};
|
||||
|
||||
class QueryLogMetric : public PeriodicLog<QueryLogMetricElement>
|
||||
class QueryMetricLog : public PeriodicLog<QueryMetricLogElement>
|
||||
{
|
||||
using PeriodicLog<QueryLogMetricElement>::PeriodicLog;
|
||||
using PeriodicLog<QueryMetricLogElement>::PeriodicLog;
|
||||
|
||||
public:
|
||||
struct ByQueryId{};
|
||||
struct ByNextCollectTime{};
|
||||
|
||||
using QuerySet = boost::multi_index_container<
|
||||
QueryLogMetricStatus,
|
||||
QueryMetricLogStatus,
|
||||
boost::multi_index::indexed_by<
|
||||
boost::multi_index::hashed_unique<boost::multi_index::tag<ByQueryId>, boost::multi_index::member<QueryLogMetricStatus, String, &QueryLogMetricStatus::query_id>>,
|
||||
boost::multi_index::ordered_non_unique<boost::multi_index::tag<ByNextCollectTime>, boost::multi_index::member<QueryLogMetricStatus, std::chrono::system_clock::time_point, &QueryLogMetricStatus::next_collect_time>>>>;
|
||||
boost::multi_index::hashed_unique<boost::multi_index::tag<ByQueryId>, boost::multi_index::member<QueryMetricLogStatus, String, &QueryMetricLogStatus::query_id>>,
|
||||
boost::multi_index::ordered_non_unique<boost::multi_index::tag<ByNextCollectTime>, boost::multi_index::member<QueryMetricLogStatus, std::chrono::system_clock::time_point, &QueryMetricLogStatus::next_collect_time>>>>;
|
||||
|
||||
// Both startQuery and finishQuery are called from the thread that executes the query
|
||||
void startQuery(const String & query_id, TimePoint query_start_time, UInt64 interval_milliseconds);
|
||||
@ -74,7 +74,7 @@ protected:
|
||||
void threadFunction() override;
|
||||
|
||||
private:
|
||||
QueryLogMetricElement createLogMetricElement(const String & query_id, std::shared_ptr<ProfileEvents::Counters::Snapshot> profile_counters, PeriodicLog<QueryLogMetricElement>::TimePoint current_time);
|
||||
QueryMetricLogElement createLogMetricElement(const String & query_id, std::shared_ptr<ProfileEvents::Counters::Snapshot> profile_counters, PeriodicLog<QueryMetricLogElement>::TimePoint current_time);
|
||||
|
||||
std::mutex queries_mutex;
|
||||
QuerySet queries;
|
@ -24,7 +24,7 @@
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/ProcessorsProfileLog.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Interpreters/QueryLogMetric.h>
|
||||
#include <Interpreters/QueryMetricLog.h>
|
||||
#include <Interpreters/QueryThreadLog.h>
|
||||
#include <Interpreters/QueryViewsLog.h>
|
||||
#include <Interpreters/ObjectStorageQueueLog.h>
|
||||
@ -291,7 +291,7 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf
|
||||
text_log = createSystemLog<TextLog>(global_context, "system", "text_log", config, "text_log", "Contains logging entries which are normally written to a log file or to stdout.");
|
||||
metric_log = createSystemLog<MetricLog>(global_context, "system", "metric_log", config, "metric_log", "Contains history of metrics values from tables system.metrics and system.events, periodically flushed to disk.");
|
||||
error_log = createSystemLog<ErrorLog>(global_context, "system", "error_log", config, "error_log", "Contains history of error values from table system.errors, periodically flushed to disk.");
|
||||
query_log_metric = createSystemLog<QueryLogMetric>(global_context, "system", "query_log_metric", config, "query_log_metric", "Contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk.");
|
||||
query_metric_log = createSystemLog<QueryMetricLog>(global_context, "system", "query_metric_log", config, "query_metric_log", "Contains history of memory and metric values from table system.events for individual queries, periodically flushed to disk.");
|
||||
filesystem_cache_log = createSystemLog<FilesystemCacheLog>(global_context, "system", "filesystem_cache_log", config, "filesystem_cache_log", "Contains a history of all events occurred with filesystem cache for objects on a remote filesystem.");
|
||||
filesystem_read_prefetches_log = createSystemLog<FilesystemReadPrefetchesLog>(
|
||||
global_context, "system", "filesystem_read_prefetches_log", config, "filesystem_read_prefetches_log", "Contains a history of all prefetches done during reading from MergeTables backed by a remote filesystem.");
|
||||
@ -315,8 +315,8 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf
|
||||
|
||||
if (query_log)
|
||||
logs.emplace_back(query_log.get());
|
||||
if (query_log_metric)
|
||||
logs.emplace_back(query_log_metric.get());
|
||||
if (query_metric_log)
|
||||
logs.emplace_back(query_metric_log.get());
|
||||
if (query_thread_log)
|
||||
logs.emplace_back(query_thread_log.get());
|
||||
if (part_log)
|
||||
@ -392,8 +392,8 @@ SystemLogs::SystemLogs(ContextPtr global_context, const Poco::Util::AbstractConf
|
||||
error_log->startCollect(global_context, collect_interval_milliseconds);
|
||||
}
|
||||
|
||||
if (query_log_metric)
|
||||
query_log_metric->startCollect(global_context, 0);
|
||||
if (query_metric_log)
|
||||
query_metric_log->startCollect(global_context, 0);
|
||||
|
||||
if (crash_log)
|
||||
{
|
||||
|
@ -42,7 +42,7 @@ class TraceLog;
|
||||
class CrashLog;
|
||||
class ErrorLog;
|
||||
class MetricLog;
|
||||
class QueryLogMetric;
|
||||
class QueryMetricLog;
|
||||
class AsynchronousMetricLog;
|
||||
class OpenTelemetrySpanLog;
|
||||
class QueryViewsLog;
|
||||
@ -75,7 +75,7 @@ struct SystemLogs
|
||||
std::shared_ptr<TextLog> text_log; /// Used to log all text messages.
|
||||
std::shared_ptr<MetricLog> metric_log; /// Used to log all metrics.
|
||||
std::shared_ptr<ErrorLog> error_log; /// Used to log errors.
|
||||
std::shared_ptr<QueryLogMetric> query_log_metric; /// Used to log all metrics for individual queries.
|
||||
std::shared_ptr<QueryMetricLog> query_metric_log; /// Used to log all metrics for individual queries.
|
||||
std::shared_ptr<FilesystemCacheLog> filesystem_cache_log;
|
||||
std::shared_ptr<FilesystemReadPrefetchesLog> filesystem_read_prefetches_log;
|
||||
std::shared_ptr<ObjectStorageQueueLog> s3_queue_log;
|
||||
|
@ -59,7 +59,7 @@
|
||||
#include <Interpreters/ProcessList.h>
|
||||
#include <Interpreters/ProcessorsProfileLog.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Interpreters/QueryLogMetric.h>
|
||||
#include <Interpreters/QueryMetricLog.h>
|
||||
#include <Interpreters/ReplaceQueryParameterVisitor.h>
|
||||
#include <Interpreters/SelectIntersectExceptQueryVisitor.h>
|
||||
#include <Interpreters/SelectQueryOptions.h>
|
||||
@ -374,12 +374,12 @@ QueryLogElement logQueryStart(
|
||||
}
|
||||
}
|
||||
|
||||
if (auto query_log_metric = context->getQueryLogMetric(); query_log_metric && !internal)
|
||||
if (auto query_metric_log = context->getQueryMetricLog(); query_metric_log && !internal)
|
||||
{
|
||||
auto interval_milliseconds = context->getSettingsRef().query_log_metric_interval;
|
||||
auto interval_milliseconds = context->getSettingsRef().query_metric_log_interval;
|
||||
if (interval_milliseconds == 0)
|
||||
interval_milliseconds = context->getConfigRef().getUInt64("query_log_metric.collect_interval_milliseconds", 1000);
|
||||
query_log_metric->startQuery(elem.client_info.current_query_id, query_start_time, interval_milliseconds);
|
||||
interval_milliseconds = context->getConfigRef().getUInt64("query_metric_log.collect_interval_milliseconds", 1000);
|
||||
query_metric_log->startQuery(elem.client_info.current_query_id, query_start_time, interval_milliseconds);
|
||||
}
|
||||
|
||||
return elem;
|
||||
@ -514,8 +514,8 @@ void logQueryFinish(
|
||||
query_span->finish();
|
||||
}
|
||||
|
||||
if (auto query_log_metric = context->getQueryLogMetric(); query_log_metric && !internal)
|
||||
query_log_metric->finishQuery(elem.client_info.current_query_id);
|
||||
if (auto query_metric_log = context->getQueryMetricLog(); query_metric_log && !internal)
|
||||
query_metric_log->finishQuery(elem.client_info.current_query_id);
|
||||
}
|
||||
|
||||
void logQueryException(
|
||||
@ -585,8 +585,8 @@ void logQueryException(
|
||||
query_span->finish();
|
||||
}
|
||||
|
||||
if (auto query_log_metric = context->getQueryLogMetric(); query_log_metric && !internal)
|
||||
query_log_metric->finishQuery(elem.client_info.current_query_id);
|
||||
if (auto query_metric_log = context->getQueryMetricLog(); query_metric_log && !internal)
|
||||
query_metric_log->finishQuery(elem.client_info.current_query_id);
|
||||
}
|
||||
|
||||
void logExceptionBeforeStart(
|
||||
@ -684,8 +684,8 @@ void logExceptionBeforeStart(
|
||||
}
|
||||
}
|
||||
|
||||
if (auto query_log_metric = context->getQueryLogMetric(); query_log_metric)
|
||||
query_log_metric->finishQuery(elem.client_info.current_query_id);
|
||||
if (auto query_metric_log = context->getQueryMetricLog(); query_metric_log)
|
||||
query_metric_log->finishQuery(elem.client_info.current_query_id);
|
||||
}
|
||||
|
||||
void validateAnalyzerSettings(ASTPtr ast, bool context_value)
|
||||
|
@ -545,7 +545,7 @@ void QueryPipeline::setProgressCallback(const ProgressCallback & callback)
|
||||
{
|
||||
// Performance counters need to be updated from the same thread the query is being executed
|
||||
// on because most info is taken using getrusage with RUSAGE_THREAD. Ideally, we would only
|
||||
// update the counters once we're close to the interval at which the query log metric data
|
||||
// update the counters once we're close to the interval at which the query metric log data
|
||||
// needs to be collected. However, since the progress callback is called not very
|
||||
// frequently, we'd rather update them as needed. Using the
|
||||
// updatePerformanceCountersIfNeeded instead of just updatePerformanceCounters we make sure
|
||||
@ -553,7 +553,7 @@ void QueryPipeline::setProgressCallback(const ProgressCallback & callback)
|
||||
auto context = CurrentThread::getQueryContext();
|
||||
if (context)
|
||||
{
|
||||
if (auto query_log_metric = context->getQueryLogMetric())
|
||||
if (auto query_metric_log = context->getQueryMetricLog())
|
||||
CurrentThread::updatePerformanceCountersIfNeeded();
|
||||
}
|
||||
|
||||
|
@ -3,7 +3,7 @@
|
||||
|
||||
<query_thread_log remove="remove"/>
|
||||
<query_log remove="remove" />
|
||||
<query_log_metric remove="remove" />
|
||||
<query_metric_log remove="remove" />
|
||||
<query_views_log remove="remove" />
|
||||
<metric_log remove="remove"/>
|
||||
<error_log remove="remove"/>
|
||||
|
@ -1468,7 +1468,7 @@ def test_backup_all(exclude_system_log_tables):
|
||||
# See the list of log tables in src/Interpreters/SystemLog.cpp
|
||||
log_tables = [
|
||||
"query_log",
|
||||
"query_log_metric",
|
||||
"query_metric_log",
|
||||
"query_thread_log",
|
||||
"part_log",
|
||||
"trace_log",
|
||||
|
@ -1,8 +1,8 @@
|
||||
<clickhouse>
|
||||
<query_log_metric>
|
||||
<query_metric_log>
|
||||
<database>system</database>
|
||||
<table>query_log_metric</table>
|
||||
<table>query_metric_log</table>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
|
||||
</query_log_metric>
|
||||
</query_metric_log>
|
||||
</clickhouse>
|
@ -18,8 +18,8 @@ def test_xml_full_conf():
|
||||
"configs/config.d/more_clusters.xml",
|
||||
"configs/config.d/part_log.xml",
|
||||
"configs/config.d/path.xml",
|
||||
"configs/config.d/query_log_metric.xml",
|
||||
"configs/config.d/query_masking_rules.xml",
|
||||
"configs/config.d/query_metric_log.xml",
|
||||
"configs/config.d/tcp_with_proxy.xml",
|
||||
"configs/config.d/text_log.xml",
|
||||
"configs/config.d/zookeeper.xml",
|
||||
|
@ -1,6 +1,6 @@
|
||||
query_log_metric:
|
||||
query_metric_log:
|
||||
database: system
|
||||
table: query_log_metric
|
||||
table: query_metric_log
|
||||
flush_interval_milliseconds: 7500
|
||||
collect_interval_milliseconds: 1000
|
||||
|
@ -18,8 +18,8 @@ def test_xml_main_conf():
|
||||
"configs/config.d/more_clusters.yaml",
|
||||
"configs/config.d/part_log.yaml",
|
||||
"configs/config.d/path.yaml",
|
||||
"configs/config.d/query_metric_log.yaml",
|
||||
"configs/config.d/query_masking_rules.yaml",
|
||||
"configs/config.d/query_metric_log.yaml",
|
||||
"configs/config.d/tcp_with_proxy.yaml",
|
||||
"configs/config.d/test_cluster_with_incorrect_pw.yaml",
|
||||
"configs/config.d/text_log.yaml",
|
||||
|
@ -1,8 +1,8 @@
|
||||
<clickhouse>
|
||||
<query_log_metric>
|
||||
<query_metric_log>
|
||||
<database>system</database>
|
||||
<table>query_log_metric</table>
|
||||
<table>query_metric_log</table>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
|
||||
</query_log_metric>
|
||||
</query_metric_log>
|
||||
</clickhouse>
|
@ -20,8 +20,8 @@ def test_extra_yaml_mix():
|
||||
"configs/config.d/more_clusters.yaml",
|
||||
"configs/config.d/part_log.xml",
|
||||
"configs/config.d/path.yaml",
|
||||
"configs/config.d/query_log_metric.xml",
|
||||
"configs/config.d/query_masking_rules.xml",
|
||||
"configs/config.d/query_metric_log.xml",
|
||||
"configs/config.d/tcp_with_proxy.yaml",
|
||||
"configs/config.d/test_cluster_with_incorrect_pw.xml",
|
||||
"configs/config.d/text_log.yaml",
|
||||
|
@ -1,6 +1,6 @@
|
||||
query_log_metric:
|
||||
query_metric_log:
|
||||
database: system
|
||||
table: query_log_metric
|
||||
table: query_metric_log
|
||||
flush_interval_milliseconds: 7500
|
||||
collect_interval_milliseconds: 1000
|
||||
|
@ -19,8 +19,8 @@ def test_yaml_full_conf():
|
||||
"configs/config.d/more_clusters.yaml",
|
||||
"configs/config.d/part_log.yaml",
|
||||
"configs/config.d/path.yaml",
|
||||
"configs/config.d/query_log_metric.yaml",
|
||||
"configs/config.d/query_masking_rules.yaml",
|
||||
"configs/config.d/query_metric_log.yaml",
|
||||
"configs/config.d/tcp_with_proxy.yaml",
|
||||
"configs/config.d/test_cluster_with_incorrect_pw.yaml",
|
||||
"configs/config.d/text_log.yaml",
|
||||
|
@ -1,8 +1,8 @@
|
||||
<clickhouse>
|
||||
<query_log_metric>
|
||||
<query_metric_log>
|
||||
<database>system</database>
|
||||
<table>query_log_metric</table>
|
||||
<table>query_metric_log</table>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
|
||||
</query_log_metric>
|
||||
</query_metric_log>
|
||||
</clickhouse>
|
@ -19,8 +19,8 @@ def test_yaml_main_conf():
|
||||
"configs/config.d/more_clusters.xml",
|
||||
"configs/config.d/part_log.xml",
|
||||
"configs/config.d/path.xml",
|
||||
"configs/config.d/query_log_metric.xml",
|
||||
"configs/config.d/query_masking_rules.xml",
|
||||
"configs/config.d/query_metric_log.xml",
|
||||
"configs/config.d/tcp_with_proxy.xml",
|
||||
"configs/config.d/test_cluster_with_incorrect_pw.xml",
|
||||
"configs/config.d/text_log.xml",
|
||||
|
@ -4,7 +4,7 @@
|
||||
|
||||
<query_thread_log remove="remove"/>
|
||||
<query_log remove="remove" />
|
||||
<query_log_metric remove="remove" />
|
||||
<query_metric_log remove="remove" />
|
||||
<query_views_log remove="remove" />
|
||||
<metric_log remove="remove"/>
|
||||
<error_log remove="remove"/>
|
||||
|
@ -5,7 +5,7 @@
|
||||
|
||||
<query_thread_log remove="remove"/>
|
||||
<query_log remove="remove" />
|
||||
<query_log_metric remove="remove" />
|
||||
<query_metric_log remove="remove" />
|
||||
<query_views_log remove="remove" />
|
||||
<metric_log remove="remove"/>
|
||||
<error_log remove="remove"/>
|
||||
|
@ -17,7 +17,7 @@ system_logs = [
|
||||
("system.text_log", 0),
|
||||
# enabled by default
|
||||
("system.query_log", 1),
|
||||
("system.query_log_metric", 1),
|
||||
("system.query_metric_log", 1),
|
||||
("system.query_thread_log", 1),
|
||||
("system.part_log", 1),
|
||||
("system.trace_log", 1),
|
||||
|
@ -26,7 +26,7 @@ def test_system_logs_recreate():
|
||||
system_logs = [
|
||||
# enabled by default
|
||||
"query_log",
|
||||
"query_log_metric",
|
||||
"query_metric_log",
|
||||
"query_thread_log",
|
||||
"part_log",
|
||||
"trace_log",
|
||||
|
@ -8,9 +8,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
readonly query_prefix=$CLICKHOUSE_DATABASE
|
||||
|
||||
$CLICKHOUSE_CLIENT --query-id="${query_prefix}_1000" -q "SELECT sleep(3) + sleep(2) FORMAT Null" &
|
||||
$CLICKHOUSE_CLIENT --query-id="${query_prefix}_1234" -q "SELECT sleep(3) + sleep(2) SETTINGS query_log_metric_interval=1234 FORMAT Null" &
|
||||
$CLICKHOUSE_CLIENT --query-id="${query_prefix}_123" -q "SELECT sleep(3) + sleep(2) SETTINGS query_log_metric_interval=123 FORMAT Null" &
|
||||
$CLICKHOUSE_CLIENT --query-id="${query_prefix}_47" -q "SELECT sleep(3) + sleep(2) SETTINGS query_log_metric_interval=47 FORMAT Null" &
|
||||
$CLICKHOUSE_CLIENT --query-id="${query_prefix}_1234" -q "SELECT sleep(3) + sleep(2) SETTINGS query_metric_log_interval=1234 FORMAT Null" &
|
||||
$CLICKHOUSE_CLIENT --query-id="${query_prefix}_123" -q "SELECT sleep(3) + sleep(2) SETTINGS query_metric_log_interval=123 FORMAT Null" &
|
||||
$CLICKHOUSE_CLIENT --query-id="${query_prefix}_47" -q "SELECT sleep(3) + sleep(2) SETTINGS query_metric_log_interval=47 FORMAT Null" &
|
||||
|
||||
wait
|
||||
|
||||
@ -25,7 +25,7 @@ function check_log()
|
||||
event_time_microseconds,
|
||||
first_value(event_time_microseconds) OVER (ORDER BY event_time_microseconds ROWS BETWEEN 1 PRECEDING AND 0 FOLLOWING) as prev,
|
||||
dateDiff('ms', prev, event_time_microseconds) AS diff
|
||||
FROM system.query_log_metric
|
||||
FROM system.query_metric_log
|
||||
WHERE query_id = '${query_prefix}_${interval}'
|
||||
ORDER BY event_time_microseconds
|
||||
OFFSET 1
|
Loading…
Reference in New Issue
Block a user