mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
done
This commit is contained in:
parent
60504bc2c8
commit
bbe20b3f9f
@ -38,10 +38,6 @@
|
||||
#include <Interpreters/Compiler.h>
|
||||
#include <Interpreters/SettingsConstraints.h>
|
||||
#include <Interpreters/SystemLog.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Interpreters/QueryThreadLog.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/TraceLog.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DDLWorker.h>
|
||||
#include <Common/DNSResolver.h>
|
||||
@ -1701,6 +1697,7 @@ std::shared_ptr<PartLog> Context::getPartLog(const String & part_database)
|
||||
return shared->system_logs->part_log;
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<TraceLog> Context::getTraceLog()
|
||||
{
|
||||
auto lock = getLock();
|
||||
@ -1711,6 +1708,7 @@ std::shared_ptr<TraceLog> Context::getTraceLog()
|
||||
return shared->system_logs->trace_log;
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<TextLog> Context::getTextLog()
|
||||
{
|
||||
auto lock = getLock();
|
||||
@ -1722,6 +1720,17 @@ std::shared_ptr<TextLog> Context::getTextLog()
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<MetricLog> Context::getMetricLog()
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (!shared->system_logs || !shared->system_logs->metric_log)
|
||||
return {};
|
||||
|
||||
return shared->system_logs->metric_log;
|
||||
}
|
||||
|
||||
|
||||
CompressionCodecPtr Context::chooseCompressionCodec(size_t part_size, double part_size_ratio) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
@ -64,6 +64,7 @@ class QueryThreadLog;
|
||||
class PartLog;
|
||||
class TextLog;
|
||||
class TraceLog;
|
||||
class MetricLog;
|
||||
struct MergeTreeSettings;
|
||||
class IDatabase;
|
||||
class DDLGuard;
|
||||
@ -434,6 +435,7 @@ public:
|
||||
std::shared_ptr<QueryThreadLog> getQueryThreadLog();
|
||||
std::shared_ptr<TraceLog> getTraceLog();
|
||||
std::shared_ptr<TextLog> getTextLog();
|
||||
std::shared_ptr<MetricLog> getMetricLog();
|
||||
|
||||
/// Returns an object used to log opertaions with parts if it possible.
|
||||
/// Provide table name to make required cheks.
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <Interpreters/QueryThreadLog.h>
|
||||
#include <Interpreters/TraceLog.h>
|
||||
#include <Interpreters/TextLog.h>
|
||||
#include <Interpreters/MetricLog.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Storages/StorageDistributed.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
@ -232,7 +233,8 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
[&] () { if (auto part_log = context.getPartLog("")) part_log->flush(); },
|
||||
[&] () { if (auto query_thread_log = context.getQueryThreadLog()) query_thread_log->flush(); },
|
||||
[&] () { if (auto trace_log = context.getTraceLog()) trace_log->flush(); },
|
||||
[&] () { if (auto text_log = context.getTextLog()) text_log->flush(); }
|
||||
[&] () { if (auto text_log = context.getTextLog()) text_log->flush(); },
|
||||
[&] () { if (auto metric_log = context.getMetricLog()) metric_log->flush(); }
|
||||
);
|
||||
break;
|
||||
case Type::STOP_LISTEN_QUERIES:
|
||||
|
68
dbms/src/Interpreters/MetricLog.cpp
Normal file
68
dbms/src/Interpreters/MetricLog.cpp
Normal file
@ -0,0 +1,68 @@
|
||||
#include <Interpreters/MetricLog.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
Block MetricLogElement::createBlock()
|
||||
{
|
||||
ColumnsWithTypeAndName columns_with_type_and_name;
|
||||
|
||||
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeDate>(), "event_date");
|
||||
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeDateTime>(), "event_time");
|
||||
|
||||
//ProfileEvents
|
||||
for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i)
|
||||
{
|
||||
std::string name;
|
||||
name += "ProfileEvent_";
|
||||
name += ProfileEvents::getName(ProfileEvents::Event(i));
|
||||
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), name);
|
||||
}
|
||||
|
||||
//CurrentMetrics
|
||||
for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i)
|
||||
{
|
||||
|
||||
std::string name;
|
||||
name += "CurrentMetric_";
|
||||
name += CurrentMetrics::getName(ProfileEvents::Event(i));
|
||||
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeInt64>(), name);
|
||||
|
||||
}
|
||||
|
||||
return Block(columns_with_type_and_name);
|
||||
}
|
||||
|
||||
void MetricLogElement::appendToBlock(Block & block) const
|
||||
{
|
||||
MutableColumns columns = block.mutateColumns();
|
||||
|
||||
size_t iter = 0;
|
||||
|
||||
columns[iter++]->insert(DateLUT::instance().toDayNum(event_time));
|
||||
columns[iter++]->insert(event_time);
|
||||
|
||||
//ProfileEvents
|
||||
for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i)
|
||||
{
|
||||
UInt64 value = ProfileEvents::global_counters[i];
|
||||
|
||||
columns[iter++]->insert(value);
|
||||
|
||||
}
|
||||
|
||||
//CurrentMetrics
|
||||
for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i)
|
||||
{
|
||||
UInt64 value = CurrentMetrics::values[i];
|
||||
|
||||
columns[iter++]->insert(value);
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
23
dbms/src/Interpreters/MetricLog.h
Normal file
23
dbms/src/Interpreters/MetricLog.h
Normal file
@ -0,0 +1,23 @@
|
||||
#pragma once
|
||||
#include <Interpreters/SystemLog.h>
|
||||
#include <Interpreters/AsynchronousMetrics.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::Message;
|
||||
|
||||
struct MetricLogElement
|
||||
{
|
||||
time_t event_time{};
|
||||
static std::string name() { return "MetricLog"; }
|
||||
static Block createBlock();
|
||||
void appendToBlock(Block & block) const;
|
||||
};
|
||||
|
||||
class MetricLog : public SystemLog<MetricLogElement>
|
||||
{
|
||||
using SystemLog<MetricLogElement>::SystemLog;
|
||||
};
|
||||
|
||||
}
|
@ -4,6 +4,7 @@
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/TextLog.h>
|
||||
#include <Interpreters/TraceLog.h>
|
||||
#include <Interpreters/MetricLog.h>
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
@ -48,7 +49,10 @@ SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfi
|
||||
part_log = createSystemLog<PartLog>(global_context, "system", "part_log", config, "part_log");
|
||||
trace_log = createSystemLog<TraceLog>(global_context, "system", "trace_log", config, "trace_log");
|
||||
text_log = createSystemLog<TextLog>(global_context, "system", "text_log", config, "text_log");
|
||||
metric_log = createSystemLog<MetricLog>(global_context, "system", "metric_log", config, "metric_log");
|
||||
|
||||
if (metric_log)
|
||||
metric_flush_thread = ThreadFromGlobalPool([this] { metricThreadFunction(); });
|
||||
part_log_database = config.getString("part_log.database", "system");
|
||||
}
|
||||
|
||||
@ -70,6 +74,41 @@ void SystemLogs::shutdown()
|
||||
trace_log->shutdown();
|
||||
if (text_log)
|
||||
text_log->shutdown();
|
||||
if (metric_log)
|
||||
{
|
||||
bool old_val = false;
|
||||
if (!is_shutdown_metric_thread.compare_exchange_strong(old_val, true))
|
||||
return;
|
||||
metric_flush_thread.join();
|
||||
metric_log->shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
void SystemLogs::metricThreadFunction()
|
||||
{
|
||||
const size_t flush_interval_milliseconds = 1000;
|
||||
while (true)
|
||||
{
|
||||
try
|
||||
{
|
||||
const auto prev_timepoint = std::chrono::system_clock::now();
|
||||
|
||||
if (is_shutdown_metric_thread)
|
||||
break;
|
||||
|
||||
MetricLogElement elem;
|
||||
elem.event_time = std::time(nullptr);
|
||||
metric_log->add(elem);
|
||||
|
||||
const auto next_timepoint = prev_timepoint + std::chrono::milliseconds(flush_interval_milliseconds);
|
||||
std::this_thread::sleep_until(next_timepoint);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -61,6 +61,7 @@ class QueryThreadLog;
|
||||
class PartLog;
|
||||
class TextLog;
|
||||
class TraceLog;
|
||||
class MetricLog;
|
||||
|
||||
/// System logs should be destroyed in destructor of the last Context and before tables,
|
||||
/// because SystemLog destruction makes insert query while flushing data into underlying tables
|
||||
@ -76,6 +77,11 @@ struct SystemLogs
|
||||
std::shared_ptr<PartLog> part_log; /// Used to log operations with parts
|
||||
std::shared_ptr<TraceLog> trace_log; /// Used to log traces from query profiler
|
||||
std::shared_ptr<TextLog> text_log; /// Used to log all text messages.
|
||||
std::shared_ptr<MetricLog> metric_log; /// Used to log all metrics.
|
||||
|
||||
ThreadFromGlobalPool metric_flush_thread;
|
||||
void metricThreadFunction();
|
||||
std::atomic<bool> is_shutdown_metric_thread{false};
|
||||
|
||||
String part_log_database;
|
||||
};
|
||||
@ -115,6 +121,8 @@ public:
|
||||
/// Stop the background flush thread before destructor. No more data will be written.
|
||||
void shutdown();
|
||||
|
||||
size_t getFlushInterval() { return flush_interval_milliseconds; }
|
||||
|
||||
protected:
|
||||
Context & context;
|
||||
const String database_name;
|
||||
@ -179,7 +187,7 @@ SystemLog<LogElement>::SystemLog(Context & context_,
|
||||
flush_interval_milliseconds(flush_interval_milliseconds_)
|
||||
{
|
||||
log = &Logger::get("SystemLog (" + database_name + "." + table_name + ")");
|
||||
|
||||
qq
|
||||
data.reserve(DBMS_SYSTEM_LOG_QUEUE_SIZE);
|
||||
saving_thread = ThreadFromGlobalPool([this] { threadFunction(); });
|
||||
}
|
||||
|
@ -27,9 +27,7 @@ MergeTreeSequentialBlockInputStream::MergeTreeSequentialBlockInputStream(
|
||||
std::stringstream message;
|
||||
message << "Reading " << data_part->getMarksCount() << " marks from part " << data_part->name
|
||||
<< ", total " << data_part->rows_count
|
||||
<< " rows starting from the beginning of the part, columns: ";
|
||||
for (size_t i = 0, size = columns_to_read.size(); i < size; ++i)
|
||||
message << (i == 0 ? "" : ", ") << columns_to_read[i];
|
||||
<< " rows starting from the beginning of the part";
|
||||
|
||||
LOG_TRACE(log, message.rdbuf());
|
||||
}
|
||||
|
@ -20,6 +20,7 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \
|
||||
ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \
|
||||
|
@ -46,6 +46,7 @@ ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/con
|
||||
ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \
|
||||
|
@ -40,6 +40,7 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \
|
||||
ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \
|
||||
|
@ -48,6 +48,7 @@ ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/con
|
||||
ln -s /usr/share/clickhouse-test/config/listen.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \
|
||||
ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \
|
||||
|
Loading…
Reference in New Issue
Block a user