This commit is contained in:
Nikita Mikhaylov 2019-08-13 17:31:46 +03:00
parent 60504bc2c8
commit bbe20b3f9f
12 changed files with 162 additions and 9 deletions

View File

@ -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();

View File

@ -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.

View File

@ -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:

View 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);
}
}
}

View 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;
};
}

View File

@ -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__);
}
}
}
}

View File

@ -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(); });
}

View File

@ -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());
}

View File

@ -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/; \

View File

@ -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/; \

View File

@ -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/; \

View File

@ -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/; \