Update using Map datatyle in system log tables before merge

This commit is contained in:
Maksim Kita 2021-06-28 14:42:21 +03:00
parent f96383c382
commit 352e1f27ff
35 changed files with 409 additions and 263 deletions

View File

@ -380,7 +380,7 @@ do
done done
# for each query run, prepare array of metrics from query log # for each query run, prepare array of metrics from query log
clickhouse-local --allow_experimental_map_type 1 --query " clickhouse-local --query "
create view query_runs as select * from file('analyze/query-runs.tsv', TSV, create view query_runs as select * from file('analyze/query-runs.tsv', TSV,
'test text, query_index int, query_id text, version UInt8, time float'); 'test text, query_index int, query_id text, version UInt8, time float');
@ -507,7 +507,7 @@ do
file="analyze/tmp/${prefix// /_}.tsv" file="analyze/tmp/${prefix// /_}.tsv"
grep "^$prefix " "analyze/query-run-metrics-for-stats.tsv" > "$file" & grep "^$prefix " "analyze/query-run-metrics-for-stats.tsv" > "$file" &
printf "%s\0\n" \ printf "%s\0\n" \
"clickhouse-local --allow_experimental_map_type 1 \ "clickhouse-local \
--file \"$file\" \ --file \"$file\" \
--structure 'test text, query text, run int, version UInt8, metrics Array(float)' \ --structure 'test text, query text, run int, version UInt8, metrics Array(float)' \
--query \"$(cat "$script_dir/eqmed.sql")\" \ --query \"$(cat "$script_dir/eqmed.sql")\" \
@ -526,7 +526,7 @@ numactl --show
numactl --cpunodebind=all --membind=all numactl --show numactl --cpunodebind=all --membind=all numactl --show
numactl --cpunodebind=all --membind=all parallel --joblog analyze/parallel-log.txt --null < analyze/commands.txt 2>> analyze/errors.log numactl --cpunodebind=all --membind=all parallel --joblog analyze/parallel-log.txt --null < analyze/commands.txt 2>> analyze/errors.log
clickhouse-local --allow_experimental_map_type 1 --query " clickhouse-local --query "
-- Join the metric names back to the metric statistics we've calculated, and make -- Join the metric names back to the metric statistics we've calculated, and make
-- a denormalized table of them -- statistics for all metrics for all queries. -- a denormalized table of them -- statistics for all metrics for all queries.
-- The WITH, ARRAY JOIN and CROSS JOIN do not like each other: -- The WITH, ARRAY JOIN and CROSS JOIN do not like each other:
@ -627,7 +627,7 @@ build_log_column_definitions
cat analyze/errors.log >> report/errors.log ||: cat analyze/errors.log >> report/errors.log ||:
cat profile-errors.log >> report/errors.log ||: cat profile-errors.log >> report/errors.log ||:
clickhouse-local --allow_experimental_map_type 1 --query " clickhouse-local --query "
create view query_display_names as select * from create view query_display_names as select * from
file('analyze/query-display-names.tsv', TSV, file('analyze/query-display-names.tsv', TSV,
'test text, query_index int, query_display_name text') 'test text, query_index int, query_display_name text')
@ -974,7 +974,7 @@ create table all_query_metrics_tsv engine File(TSV, 'report/all-query-metrics.ts
for version in {right,left} for version in {right,left}
do do
rm -rf data rm -rf data
clickhouse-local --allow_experimental_map_type 1 --query " clickhouse-local --query "
create view query_profiles as create view query_profiles as
with 0 as left, 1 as right with 0 as left, 1 as right
select * from file('analyze/query-profiles.tsv', TSV, select * from file('analyze/query-profiles.tsv', TSV,
@ -1168,7 +1168,7 @@ build_log_column_definitions
rm -rf metrics ||: rm -rf metrics ||:
mkdir metrics mkdir metrics
clickhouse-local --allow_experimental_map_type 1 --query " clickhouse-local --query "
create view right_async_metric_log as create view right_async_metric_log as
select * from file('right-async-metric-log.tsv', TSVWithNamesAndTypes, select * from file('right-async-metric-log.tsv', TSVWithNamesAndTypes,
'$(cat right-async-metric-log.tsv.columns)') '$(cat right-async-metric-log.tsv.columns)')
@ -1285,10 +1285,10 @@ create table ci_checks engine File(TSVWithNamesAndTypes, 'ci-checks.tsv')
set +x # Don't show password in the log set +x # Don't show password in the log
client=(clickhouse-client client=(clickhouse-client
# Surprisingly, clickhouse-client doesn't understand --host 127.0.0.1:9000 # Surprisingly, clickhouse-client doesn't understand --host 127.0.0.1:9000
# so I have to extract host and port with clickhouse-local --allow_experimental_map_type 1. I tried to use # so I have to extract host and port with clickhouse-local. I tried to use
# Poco URI parser to support this in the client, but it's broken and can't # Poco URI parser to support this in the client, but it's broken and can't
# parse host:port. # parse host:port.
$(clickhouse-local --allow_experimental_map_type 1 --query "with '${CHPC_DATABASE_URL}' as url select '--host ' || domain(url) || ' --port ' || toString(port(url)) format TSV") $(clickhouse-local --query "with '${CHPC_DATABASE_URL}' as url select '--host ' || domain(url) || ' --port ' || toString(port(url)) format TSV")
--secure --secure
--user "${CHPC_DATABASE_USER}" --user "${CHPC_DATABASE_USER}"
--password "${CHPC_DATABASE_PASSWORD}" --password "${CHPC_DATABASE_PASSWORD}"
@ -1352,7 +1352,7 @@ EOF
} }
# Check that local and client are in PATH # Check that local and client are in PATH
clickhouse-local --allow_experimental_map_type 1 --version > /dev/null clickhouse-local --version > /dev/null
clickhouse-client --version > /dev/null clickhouse-client --version > /dev/null
case "$stage" in case "$stage" in

View File

@ -83,7 +83,7 @@ ClickHouse不会自动从表中删除数据。更多详情请看 [introduction](
- `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse revision. - `revision` ([UInt32](../../sql-reference/data-types/int-uint.md)) — ClickHouse revision.
- `ProfileEvents` ([Map(String, UInt64))](../../sql-reference/data-types/array.md)) — Counters that measure different metrics. The description of them could be found in the table [系统。活动](../../operations/system-tables/events.md#system_tables-events) - `ProfileEvents` ([Map(String, UInt64))](../../sql-reference/data-types/array.md)) — Counters that measure different metrics. The description of them could be found in the table [系统。活动](../../operations/system-tables/events.md#system_tables-events)
- `Settings` ([Map(String, String)](../../sql-reference/data-types/array.md)) — Names of settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` 参数为1。 - `Settings` ([Map(String, String)](../../sql-reference/data-types/array.md)) — Names of settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` 参数为1。
- `thread_ids` ([Array(UInt32)](../../sql-reference/data-types/array.md)) — 参与查询的线程数. - `thread_ids` ([Array(UInt64)](../../sql-reference/data-types/array.md)) — 参与查询的线程数.
- `Settings.Names` ([ArrayString)](../../sql-reference/data-types/array.md)) — 客户端运行查询时更改的设置的名称。 要启用对设置的日志记录更改请将log_query_settings参数设置为1。 - `Settings.Names` ([ArrayString)](../../sql-reference/data-types/array.md)) — 客户端运行查询时更改的设置的名称。 要启用对设置的日志记录更改请将log_query_settings参数设置为1。
- `Settings.Values` ([ArrayString)](../../sql-reference/data-types/array.md)) — `Settings.Names` 列中列出的设置的值。 - `Settings.Values` ([ArrayString)](../../sql-reference/data-types/array.md)) — `Settings.Names` 列中列出的设置的值。
**示例** **示例**

View File

@ -0,0 +1,53 @@
#pragma once
#include <map>
#include <list>
#include <optional>
#include <string>
#include <set>
#include <initializer_list>
#include <DataTypes/IDataType.h>
#include <Core/Names.h>
namespace DB
{
class NameAndAliasPair
{
public:
NameAndAliasPair(const String & name_, const DataTypePtr & type_, const String & expression_)
: name(name_)
, type(type_)
, expression(expression_)
{}
String name;
DataTypePtr type;
String expression;
};
/// This needed to use structured bindings for NameAndTypePair
/// const auto & [name, type] = name_and_type
template <int I>
decltype(auto) get(const NameAndAliasPair & name_and_alias)
{
if constexpr (I == 0)
return name_and_alias.name;
else if constexpr (I == 1)
return name_and_alias.type;
else if constexpr (I == 2)
return name_and_alias.expression;
}
using NamesAndAliases = std::vector<NameAndAliasPair>;
}
namespace std
{
template <> struct tuple_size<DB::NameAndAliasPair> : std::integral_constant<size_t, 2> {};
template <> struct tuple_element<0, DB::NameAndAliasPair> { using type = DB::String; };
template <> struct tuple_element<1, DB::NameAndAliasPair> { using type = DB::DataTypePtr; };
template <> struct tuple_element<2, DB::NameAndAliasPair> { using type = DB::String; };
}

View File

@ -77,6 +77,7 @@ void Settings::dumpToMapColumn(IColumn * column, bool changed_only)
value_column.insert(setting.getValueString()); value_column.insert(setting.getValueString());
size++; size++;
} }
offsets.push_back(offsets.back() + size); offsets.push_back(offsets.back() + size);
} }

View File

@ -441,7 +441,7 @@ class IColumn;
M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \ M(Bool, asterisk_include_alias_columns, false, "Include ALIAS columns for wildcard query", 0) \
M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \ M(Bool, optimize_skip_merged_partitions, false, "Skip partitions with one part with level > 0 in optimize final", 0) \
M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \
M(Bool, allow_experimental_map_type, false, "Allow data type Map", 0) \ M(Bool, allow_experimental_map_type, true, "Obsolete setting, does nothing.", 0) \
M(Bool, allow_experimental_window_functions, false, "Allow experimental window functions", 0) \ M(Bool, allow_experimental_window_functions, false, "Allow experimental window functions", 0) \
M(Bool, allow_experimental_projection_optimization, false, "Enable projection optimization when processing SELECT queries", 0) \ M(Bool, allow_experimental_projection_optimization, false, "Enable projection optimization when processing SELECT queries", 0) \
M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \ M(Bool, force_optimize_projection, false, "If projection optimization is enabled, SELECT queries need to use projection", 0) \
@ -596,7 +596,7 @@ struct Settings : public BaseSettings<SettingsTraits>
/// Load settings from configuration file, at "path" prefix in configuration. /// Load settings from configuration file, at "path" prefix in configuration.
void loadSettingsFromConfig(const String & path, const Poco::Util::AbstractConfiguration & config); void loadSettingsFromConfig(const String & path, const Poco::Util::AbstractConfiguration & config);
/// Dumps profile events to two columns of type Array(String) /// Dumps profile events to column of type Map(String, String)
void dumpToMapColumn(IColumn * column, bool changed_only = true); void dumpToMapColumn(IColumn * column, bool changed_only = true);
/// Adds program options to set the settings from a command line. /// Adds program options to set the settings from a command line.

View File

@ -11,17 +11,16 @@
namespace DB namespace DB
{ {
Block AsynchronousMetricLogElement::createBlock() NamesAndTypesList AsynchronousMetricLogElement::getNamesAndTypes()
{ {
ColumnsWithTypeAndName columns; return
{
columns.emplace_back(std::make_shared<DataTypeDate>(), "event_date"); {"event_date", std::make_shared<DataTypeDate>()},
columns.emplace_back(std::make_shared<DataTypeDateTime>(), "event_time"); {"event_time", std::make_shared<DataTypeDateTime>()},
columns.emplace_back(std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"); {"event_time_microseconds", std::make_shared<DataTypeDateTime64>(6)},
columns.emplace_back(std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), "name"); {"name", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
columns.emplace_back(std::make_shared<DataTypeFloat64>(), "value"); {"value", std::make_shared<DataTypeFloat64>(),}
};
return Block(columns);
} }

View File

@ -12,8 +12,8 @@
namespace DB namespace DB
{ {
typedef double AsynchronousMetricValue; using AsynchronousMetricValue = double;
typedef std::unordered_map<std::string, AsynchronousMetricValue> AsynchronousMetricValues; using AsynchronousMetricValues = std::unordered_map<std::string, AsynchronousMetricValue>;
/** AsynchronousMetricLog is a log of metric values measured at regular time interval. /** AsynchronousMetricLog is a log of metric values measured at regular time interval.
*/ */
@ -27,7 +27,8 @@ struct AsynchronousMetricLogElement
double value; double value;
static std::string name() { return "AsynchronousMetricLog"; } static std::string name() { return "AsynchronousMetricLog"; }
static Block createBlock(); static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases() { return {}; }
void appendToBlock(MutableColumns & columns) const; void appendToBlock(MutableColumns & columns) const;
}; };

View File

@ -18,21 +18,21 @@ namespace DB
std::weak_ptr<CrashLog> CrashLog::crash_log; std::weak_ptr<CrashLog> CrashLog::crash_log;
Block CrashLogElement::createBlock() NamesAndTypesList CrashLogElement::getNamesAndTypes()
{ {
return return
{ {
{std::make_shared<DataTypeDate>(), "event_date"}, {"event_date", std::make_shared<DataTypeDate>()},
{std::make_shared<DataTypeDateTime>(), "event_time"}, {"event_time", std::make_shared<DataTypeDateTime>()},
{std::make_shared<DataTypeUInt64>(), "timestamp_ns"}, {"timestamp_ns", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeInt32>(), "signal"}, {"signal", std::make_shared<DataTypeInt32>()},
{std::make_shared<DataTypeUInt64>(), "thread_id"}, {"thread_id", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeString>(), "query_id"}, {"query_id", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>()), "trace"}, {"trace", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "trace_full"}, {"trace_full", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{std::make_shared<DataTypeString>(), "version"}, {"version", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeUInt32>(), "revision"}, {"revision", std::make_shared<DataTypeUInt32>()},
{std::make_shared<DataTypeString>(), "build_id"}, {"build_id", std::make_shared<DataTypeString>()},
}; };
} }
@ -60,7 +60,6 @@ void CrashLogElement::appendToBlock(MutableColumns & columns) const
} }
void collectCrashLog(Int32 signal, UInt64 thread_id, const String & query_id, const StackTrace & stack_trace) void collectCrashLog(Int32 signal, UInt64 thread_id, const String & query_id, const StackTrace & stack_trace)
{ {
using namespace DB; using namespace DB;

View File

@ -24,7 +24,8 @@ struct CrashLogElement
Array trace_full; Array trace_full;
static std::string name() { return "CrashLog"; } static std::string name() { return "CrashLog"; }
static Block createBlock(); static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases() { return {}; }
void appendToBlock(MutableColumns & columns) const; void appendToBlock(MutableColumns & columns) const;
}; };

View File

@ -302,6 +302,35 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
return columns_list; return columns_list;
} }
ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns, const NamesAndAliases & alias_columns)
{
std::shared_ptr<ASTExpressionList> columns_list = std::static_pointer_cast<ASTExpressionList>(formatColumns(columns));
for (const auto & alias_column : alias_columns)
{
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
column_declaration->name = alias_column.name;
ParserDataType type_parser;
String type_name = alias_column.type->getName();
const char * type_pos = type_name.data();
const char * type_end = type_pos + type_name.size();
column_declaration->type = parseQuery(type_parser, type_pos, type_end, "data type", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
column_declaration->default_specifier = "ALIAS";
const auto & alias = alias_column.expression;
const char * alias_pos = alias.data();
const char * alias_end = alias_pos + alias.size();
ParserExpression expression_parser;
column_declaration->default_expression = parseQuery(expression_parser, alias_pos, alias_end, "expression", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
columns_list->children.emplace_back(column_declaration);
}
return columns_list;
}
ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
{ {
auto columns_list = std::make_shared<ASTExpressionList>(); auto columns_list = std::make_shared<ASTExpressionList>();
@ -646,24 +675,6 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat
} }
} }
} }
// enable allow_experimental_map_type for system tables
if (create.database != "system" && !create.attach && !settings.allow_experimental_map_type)
{
for (const auto & name_and_type_pair : properties.columns.getAllPhysical())
{
WhichDataType which(*name_and_type_pair.type);
if (which.isMap())
{
const auto & type_name = name_and_type_pair.type->getName();
String message = "Cannot create table with column '" + name_and_type_pair.name + "' which type is '"
+ type_name + "' because experimental Map type is not allowed. "
+ "Set 'allow_experimental_map_type = 1' setting to enable";
throw Exception(message, ErrorCodes::ILLEGAL_COLUMN);
}
}
}
} }
void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const

View File

@ -1,12 +1,12 @@
#pragma once #pragma once
#include <Core/NamesAndAliases.h>
#include <Access/AccessRightsElement.h> #include <Access/AccessRightsElement.h>
#include <Interpreters/IInterpreter.h> #include <Interpreters/IInterpreter.h>
#include <Storages/ColumnsDescription.h> #include <Storages/ColumnsDescription.h>
#include <Storages/ConstraintsDescription.h> #include <Storages/ConstraintsDescription.h>
#include <Storages/IStorage_fwd.h> #include <Storages/IStorage_fwd.h>
#include <Storages/StorageInMemoryMetadata.h> #include <Storages/StorageInMemoryMetadata.h>
#include <Common/ThreadPool.h>
namespace DB namespace DB
@ -31,8 +31,8 @@ public:
/// List of columns and their types in AST. /// List of columns and their types in AST.
static ASTPtr formatColumns(const NamesAndTypesList & columns); static ASTPtr formatColumns(const NamesAndTypesList & columns);
static ASTPtr formatColumns(const NamesAndTypesList & columns, const NamesAndAliases & alias_columns);
static ASTPtr formatColumns(const ColumnsDescription & columns); static ASTPtr formatColumns(const ColumnsDescription & columns);
static ASTPtr formatIndices(const IndicesDescription & indices); static ASTPtr formatIndices(const IndicesDescription & indices);
static ASTPtr formatConstraints(const ConstraintsDescription & constraints); static ASTPtr formatConstraints(const ConstraintsDescription & constraints);
static ASTPtr formatProjections(const ProjectionsDescription & projections); static ASTPtr formatProjections(const ProjectionsDescription & projections);

View File

@ -8,21 +8,21 @@
namespace DB namespace DB
{ {
Block MetricLogElement::createBlock() NamesAndTypesList MetricLogElement::getNamesAndTypes()
{ {
ColumnsWithTypeAndName columns_with_type_and_name; NamesAndTypesList 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("event_date", std::make_shared<DataTypeDate>());
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeDateTime>(), "event_time"); columns_with_type_and_name.emplace_back("event_time", std::make_shared<DataTypeDateTime>());
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"); columns_with_type_and_name.emplace_back("event_time_microseconds", std::make_shared<DataTypeDateTime64>(6));
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "milliseconds"); columns_with_type_and_name.emplace_back("milliseconds", std::make_shared<DataTypeUInt64>());
for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i)
{ {
std::string name; std::string name;
name += "ProfileEvent_"; name += "ProfileEvent_";
name += ProfileEvents::getName(ProfileEvents::Event(i)); name += ProfileEvents::getName(ProfileEvents::Event(i));
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), std::move(name)); columns_with_type_and_name.emplace_back(std::move(name), std::make_shared<DataTypeUInt64>());
} }
for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i)
@ -30,10 +30,10 @@ Block MetricLogElement::createBlock()
std::string name; std::string name;
name += "CurrentMetric_"; name += "CurrentMetric_";
name += CurrentMetrics::getName(CurrentMetrics::Metric(i)); name += CurrentMetrics::getName(CurrentMetrics::Metric(i));
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeInt64>(), std::move(name)); columns_with_type_and_name.emplace_back(std::move(name), std::make_shared<DataTypeInt64>());
} }
return Block(columns_with_type_and_name); return columns_with_type_and_name;
} }

View File

@ -25,7 +25,8 @@ struct MetricLogElement
std::vector<CurrentMetrics::Metric> current_metrics; std::vector<CurrentMetrics::Metric> current_metrics;
static std::string name() { return "MetricLog"; } static std::string name() { return "MetricLog"; }
static Block createBlock(); static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases() { return {}; }
void appendToBlock(MutableColumns & columns) const; void appendToBlock(MutableColumns & columns) const;
}; };

View File

@ -15,13 +15,13 @@
namespace DB namespace DB
{ {
Block OpenTelemetrySpanLogElement::createBlock() NamesAndTypesList OpenTelemetrySpanLogElement::getNamesAndTypes()
{ {
return { return {
{std::make_shared<DataTypeUUID>(), "trace_id"}, {"trace_id", std::make_shared<DataTypeUUID>()},
{std::make_shared<DataTypeUInt64>(), "span_id"}, {"span_id", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt64>(), "parent_span_id"}, {"parent_span_id", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeString>(), "operation_name"}, {"operation_name", std::make_shared<DataTypeString>()},
// DateTime64 is really unwieldy -- there is no "normal" way to convert // DateTime64 is really unwieldy -- there is no "normal" way to convert
// it to an UInt64 count of microseconds, except: // it to an UInt64 count of microseconds, except:
// 1) reinterpretAsUInt64(reinterpretAsFixedString(date)), which just // 1) reinterpretAsUInt64(reinterpretAsFixedString(date)), which just
@ -32,14 +32,21 @@ Block OpenTelemetrySpanLogElement::createBlock()
// Also subtraction of two DateTime64 points doesn't work, so you can't // Also subtraction of two DateTime64 points doesn't work, so you can't
// get duration. // get duration.
// It is much less hassle to just use UInt64 of microseconds. // It is much less hassle to just use UInt64 of microseconds.
{std::make_shared<DataTypeUInt64>(), "start_time_us"}, {"start_time_us", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt64>(), "finish_time_us"}, {"finish_time_us", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeDate>(), "finish_date"}, {"finish_date", std::make_shared<DataTypeDate>()},
{std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), std::make_shared<DataTypeString>()), {"attribute", std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), std::make_shared<DataTypeString>())},
"attribute"},
}; };
} }
NamesAndAliases OpenTelemetrySpanLogElement::getNamesAndAliases()
{
return
{
{"attribute.names", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "mapKeys(attribute)"},
{"attribute.values", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "mapKeys(attribute)"}
};
}
void OpenTelemetrySpanLogElement::appendToBlock(MutableColumns & columns) const void OpenTelemetrySpanLogElement::appendToBlock(MutableColumns & columns) const
{ {

View File

@ -27,7 +27,8 @@ struct OpenTelemetrySpanLogElement : public OpenTelemetrySpan
: OpenTelemetrySpan(span) {} : OpenTelemetrySpan(span) {}
static std::string name() { return "OpenTelemetrySpanLog"; } static std::string name() { return "OpenTelemetrySpanLog"; }
static Block createBlock(); static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases();
void appendToBlock(MutableColumns & columns) const; void appendToBlock(MutableColumns & columns) const;
}; };

View File

@ -16,7 +16,7 @@
namespace DB namespace DB
{ {
Block PartLogElement::createBlock() NamesAndTypesList PartLogElement::getNamesAndTypes()
{ {
auto event_type_datatype = std::make_shared<DataTypeEnum8>( auto event_type_datatype = std::make_shared<DataTypeEnum8>(
DataTypeEnum8::Values DataTypeEnum8::Values
@ -33,35 +33,34 @@ Block PartLogElement::createBlock()
ColumnsWithTypeAndName columns_with_type_and_name; ColumnsWithTypeAndName columns_with_type_and_name;
return { return {
{"query_id", std::make_shared<DataTypeString>()},
{"event_type", std::move(event_type_datatype)},
{"event_date", std::make_shared<DataTypeDate>()},
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeString>(), "query_id"), {"event_time", std::make_shared<DataTypeDateTime>()},
columns_with_type_and_name.emplace_back(std::move(event_type_datatype), "event_type"), {"event_time_microseconds", std::make_shared<DataTypeDateTime64>(6)},
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"), {"duration_ms", std::make_shared<DataTypeUInt64>()},
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "duration_ms"), {"database", std::make_shared<DataTypeString>()},
{"table", std::make_shared<DataTypeString>()},
{"part_name", std::make_shared<DataTypeString>()},
{"partition_id", std::make_shared<DataTypeString>()},
{"path_on_disk", std::make_shared<DataTypeString>()},
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeString>(), "database"), {"rows", std::make_shared<DataTypeUInt64>()},
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeString>(), "table"), {"size_in_bytes", std::make_shared<DataTypeUInt64>()}, // On disk
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeString>(), "part_name"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeString>(), "partition_id"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeString>(), "path_on_disk"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "rows"),
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "size_in_bytes"), // On disk
/// Merge-specific info /// Merge-specific info
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "merged_from"), {"merged_from", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "bytes_uncompressed"), // Result bytes {"bytes_uncompressed", std::make_shared<DataTypeUInt64>()}, // Result bytes
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "read_rows"), {"read_rows", std::make_shared<DataTypeUInt64>()},
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "read_bytes"), {"read_bytes", std::make_shared<DataTypeUInt64>()},
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "peak_memory_usage"), {"peak_memory_usage", std::make_shared<DataTypeUInt64>()},
/// Is there an error during the execution or commit /// Is there an error during the execution or commit
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt16>(), "error"), {"error", std::make_shared<DataTypeUInt16>()},
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeString>(), "exception"), {"exception", std::make_shared<DataTypeString>()},
}; };
} }

View File

@ -52,7 +52,8 @@ struct PartLogElement
static std::string name() { return "PartLog"; } static std::string name() { return "PartLog"; }
static Block createBlock(); static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases() { return {}; }
void appendToBlock(MutableColumns & columns) const; void appendToBlock(MutableColumns & columns) const;
}; };

View File

@ -36,6 +36,7 @@ void dumpToMapColumn(const Counters & counters, DB::IColumn * column, bool nonze
value_column.insert(value); value_column.insert(value);
size++; size++;
} }
offsets.push_back(offsets.back() + size); offsets.push_back(offsets.back() + size);
} }

View File

@ -26,7 +26,7 @@
namespace DB namespace DB
{ {
Block QueryLogElement::createBlock() NamesAndTypesList QueryLogElement::getNamesAndTypes()
{ {
auto query_status_datatype = std::make_shared<DataTypeEnum8>( auto query_status_datatype = std::make_shared<DataTypeEnum8>(
DataTypeEnum8::Values DataTypeEnum8::Values
@ -39,84 +39,94 @@ Block QueryLogElement::createBlock()
return return
{ {
{std::move(query_status_datatype), "type"}, {"type", std::move(query_status_datatype)},
{std::make_shared<DataTypeDate>(), "event_date"}, {"event_date", std::make_shared<DataTypeDate>()},
{std::make_shared<DataTypeDateTime>(), "event_time"}, {"event_time", std::make_shared<DataTypeDateTime>()},
{std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"}, {"event_time_microseconds", std::make_shared<DataTypeDateTime64>(6)},
{std::make_shared<DataTypeDateTime>(), "query_start_time"}, {"query_start_time", std::make_shared<DataTypeDateTime>()},
{std::make_shared<DataTypeDateTime64>(6), "query_start_time_microseconds"}, {"query_start_time_microseconds", std::make_shared<DataTypeDateTime64>(6)},
{std::make_shared<DataTypeUInt64>(), "query_duration_ms"}, {"query_duration_ms", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt64>(), "read_rows"}, {"read_rows", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt64>(), "read_bytes"}, {"read_bytes", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt64>(), "written_rows"}, {"written_rows", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt64>(), "written_bytes"}, {"written_bytes", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt64>(), "result_rows"}, {"result_rows", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt64>(), "result_bytes"}, {"result_bytes", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt64>(), "memory_usage"}, {"memory_usage", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeString>(), "current_database"}, {"current_database", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "query"}, {"query", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeUInt64>(), "normalized_query_hash"}, {"normalized_query_hash", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), "query_kind"}, {"query_kind", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
{std::make_shared<DataTypeArray>( {"databases", std::make_shared<DataTypeArray>(
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())), "databases"}, std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()))},
{std::make_shared<DataTypeArray>( {"tables", std::make_shared<DataTypeArray>(
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())), "tables"}, std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()))},
{std::make_shared<DataTypeArray>( {"columns", std::make_shared<DataTypeArray>(
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())), "columns"}, std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()))},
{std::make_shared<DataTypeArray>( {"projections", std::make_shared<DataTypeArray>(
std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())), "projections"}, std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()))},
{std::make_shared<DataTypeInt32>(), "exception_code"}, {"exception_code", std::make_shared<DataTypeInt32>()},
{std::make_shared<DataTypeString>(), "exception"}, {"exception", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "stack_trace"}, {"stack_trace", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeUInt8>(), "is_initial_query"}, {"is_initial_query", std::make_shared<DataTypeUInt8>()},
{std::make_shared<DataTypeString>(), "user"}, {"user", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "query_id"}, {"query_id", std::make_shared<DataTypeString>()},
{DataTypeFactory::instance().get("IPv6"), "address"}, {"address", DataTypeFactory::instance().get("IPv6")},
{std::make_shared<DataTypeUInt16>(), "port"}, {"port", std::make_shared<DataTypeUInt16>()},
{std::make_shared<DataTypeString>(), "initial_user"}, {"initial_user", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "initial_query_id"}, {"initial_query_id", std::make_shared<DataTypeString>()},
{DataTypeFactory::instance().get("IPv6"), "initial_address"}, {"initial_address", DataTypeFactory::instance().get("IPv6")},
{std::make_shared<DataTypeUInt16>(), "initial_port"}, {"initial_port", std::make_shared<DataTypeUInt16>()},
{std::make_shared<DataTypeDateTime>(), "initial_query_start_time"}, {"initial_query_start_time", std::make_shared<DataTypeDateTime>()},
{std::make_shared<DataTypeDateTime64>(6), "initial_query_start_time_microseconds"}, {"initial_query_start_time_microseconds", std::make_shared<DataTypeDateTime64>(6)},
{std::make_shared<DataTypeUInt8>(), "interface"}, {"interface", std::make_shared<DataTypeUInt8>()},
{std::make_shared<DataTypeString>(), "os_user"}, {"os_user", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "client_hostname"}, {"client_hostname", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "client_name"}, {"client_name", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeUInt32>(), "client_revision"}, {"client_revision", std::make_shared<DataTypeUInt32>()},
{std::make_shared<DataTypeUInt32>(), "client_version_major"}, {"client_version_major", std::make_shared<DataTypeUInt32>()},
{std::make_shared<DataTypeUInt32>(), "client_version_minor"}, {"client_version_minor", std::make_shared<DataTypeUInt32>()},
{std::make_shared<DataTypeUInt32>(), "client_version_patch"}, {"client_version_patch", std::make_shared<DataTypeUInt32>()},
{std::make_shared<DataTypeUInt8>(), "http_method"}, {"http_method", std::make_shared<DataTypeUInt8>()},
{std::make_shared<DataTypeString>(), "http_user_agent"}, {"http_user_agent", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "http_referer"}, {"http_referer", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "forwarded_for"}, {"forwarded_for", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "quota_key"}, {"quota_key", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeUInt32>(), "revision"}, {"revision", std::make_shared<DataTypeUInt32>()},
{std::make_shared<DataTypeString>(), "log_comment"}, {"log_comment", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>()), "thread_ids"}, {"thread_ids", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())},
{std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), std::make_shared<DataTypeUInt64>()), "ProfileEvents"}, {"ProfileEvents", std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), std::make_shared<DataTypeUInt64>())},
{std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), std::make_shared<DataTypeString>()), "Settings"}, {"Settings", std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), std::make_shared<DataTypeString>())},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "used_aggregate_functions"}, {"used_aggregate_functions", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "used_aggregate_function_combinators"}, {"used_aggregate_function_combinators", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "used_database_engines"}, {"used_database_engines", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "used_data_type_families"}, {"used_data_type_families", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "used_dictionaries"}, {"used_dictionaries", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "used_formats"}, {"used_formats", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "used_functions"}, {"used_functions", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "used_storages"}, {"used_storages", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "used_table_functions"} {"used_table_functions", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())}
}; };
} }
NamesAndAliases QueryLogElement::getNamesAndAliases()
{
return
{
{"ProfileEvents.Names", {std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())}, "mapKeys(ProfileEvents)"},
{"ProfileEvents.Values", {std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())}, "mapValues(ProfileEvents)"},
{"Settings.Names", {std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())}, "mapKeys(Settings)" },
{"Settings.Values", {std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())}, "mapValues(Settings)"}
};
}
void QueryLogElement::appendToBlock(MutableColumns & columns) const void QueryLogElement::appendToBlock(MutableColumns & columns) const
{ {

View File

@ -1,9 +1,9 @@
#pragma once #pragma once
#include <Core/NamesAndAliases.h>
#include <Interpreters/SystemLog.h> #include <Interpreters/SystemLog.h>
#include <Interpreters/ClientInfo.h> #include <Interpreters/ClientInfo.h>
namespace ProfileEvents namespace ProfileEvents
{ {
class Counters; class Counters;
@ -83,7 +83,8 @@ struct QueryLogElement
static std::string name() { return "QueryLog"; } static std::string name() { return "QueryLog"; }
static Block createBlock(); static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases();
void appendToBlock(MutableColumns & columns) const; void appendToBlock(MutableColumns & columns) const;
static void appendClientInfo(const ClientInfo & client_info, MutableColumns & columns, size_t & i); static void appendClientInfo(const ClientInfo & client_info, MutableColumns & columns, size_t & i);

View File

@ -19,58 +19,68 @@
namespace DB namespace DB
{ {
Block QueryThreadLogElement::createBlock()
NamesAndTypesList QueryThreadLogElement::getNamesAndTypes()
{ {
return { return {
{std::make_shared<DataTypeDate>(), "event_date"}, {"event_date", std::make_shared<DataTypeDate>()},
{std::make_shared<DataTypeDateTime>(), "event_time"}, {"event_time", std::make_shared<DataTypeDateTime>()},
{std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"}, {"event_time_microseconds", std::make_shared<DataTypeDateTime64>(6)},
{std::make_shared<DataTypeDateTime>(), "query_start_time"}, {"query_start_time", std::make_shared<DataTypeDateTime>()},
{std::make_shared<DataTypeDateTime64>(6), "query_start_time_microseconds"}, {"query_start_time_microseconds", std::make_shared<DataTypeDateTime64>(6)},
{std::make_shared<DataTypeUInt64>(), "query_duration_ms"}, {"query_duration_ms", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt64>(), "read_rows"}, {"read_rows", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt64>(), "read_bytes"}, {"read_bytes", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt64>(), "written_rows"}, {"written_rows", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt64>(), "written_bytes"}, {"written_bytes", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeInt64>(), "memory_usage"}, {"memory_usage", std::make_shared<DataTypeInt64>()},
{std::make_shared<DataTypeInt64>(), "peak_memory_usage"}, {"peak_memory_usage", std::make_shared<DataTypeInt64>()},
{std::make_shared<DataTypeString>(), "thread_name"}, {"thread_name", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeUInt64>(), "thread_id"}, {"thread_id", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt64>(), "master_thread_id"}, {"master_thread_id", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeString>(), "current_database"}, {"current_database", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "query"}, {"query", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeUInt64>(), "normalized_query_hash"}, {"normalized_query_hash", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt8>(), "is_initial_query"}, {"is_initial_query", std::make_shared<DataTypeUInt8>()},
{std::make_shared<DataTypeString>(), "user"}, {"user", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "query_id"}, {"query_id", std::make_shared<DataTypeString>()},
{DataTypeFactory::instance().get("IPv6"), "address"}, {"address", DataTypeFactory::instance().get("IPv6")},
{std::make_shared<DataTypeUInt16>(), "port"}, {"port", std::make_shared<DataTypeUInt16>()},
{std::make_shared<DataTypeString>(), "initial_user"}, {"initial_user", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "initial_query_id"}, {"initial_query_id", std::make_shared<DataTypeString>()},
{DataTypeFactory::instance().get("IPv6"), "initial_address"}, {"initial_address", DataTypeFactory::instance().get("IPv6")},
{std::make_shared<DataTypeUInt16>(), "initial_port"}, {"initial_port", std::make_shared<DataTypeUInt16>()},
{std::make_shared<DataTypeDateTime>(), "initial_query_start_time"}, {"initial_query_start_time", std::make_shared<DataTypeDateTime>()},
{std::make_shared<DataTypeDateTime64>(6), "initial_query_start_time_microseconds"}, {"initial_query_start_time_microseconds", std::make_shared<DataTypeDateTime64>(6)},
{std::make_shared<DataTypeUInt8>(), "interface"}, {"interface", std::make_shared<DataTypeUInt8>()},
{std::make_shared<DataTypeString>(), "os_user"}, {"os_user", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "client_hostname"}, {"client_hostname", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "client_name"}, {"client_name", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeUInt32>(), "client_revision"}, {"client_revision", std::make_shared<DataTypeUInt32>()},
{std::make_shared<DataTypeUInt32>(), "client_version_major"}, {"client_version_major", std::make_shared<DataTypeUInt32>()},
{std::make_shared<DataTypeUInt32>(), "client_version_minor"}, {"client_version_minor", std::make_shared<DataTypeUInt32>()},
{std::make_shared<DataTypeUInt32>(), "client_version_patch"}, {"client_version_patch", std::make_shared<DataTypeUInt32>()},
{std::make_shared<DataTypeUInt8>(), "http_method"}, {"http_method", std::make_shared<DataTypeUInt8>()},
{std::make_shared<DataTypeString>(), "http_user_agent"}, {"http_user_agent", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "http_referer"}, {"http_referer", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "forwarded_for"}, {"forwarded_for", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeString>(), "quota_key"}, {"quota_key", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeUInt32>(), "revision"}, {"revision", std::make_shared<DataTypeUInt32>()},
{std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), std::make_shared<DataTypeUInt64>()), "ProfileEvents"}, {"ProfileEvents", std::make_shared<DataTypeMap>(std::make_shared<DataTypeString>(), std::make_shared<DataTypeUInt64>())},
};
}
NamesAndAliases QueryThreadLogElement::getNamesAndAliases()
{
return
{
{"ProfileEvents.Names", {std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())}, "mapKeys(ProfileEvents)"},
{"ProfileEvents.Values", {std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())}, "mapValues(ProfileEvents)"}
}; };
} }

View File

@ -49,7 +49,8 @@ struct QueryThreadLogElement
static std::string name() { return "QueryThreadLog"; } static std::string name() { return "QueryThreadLog"; }
static Block createBlock(); static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases();
void appendToBlock(MutableColumns & columns) const; void appendToBlock(MutableColumns & columns) const;
}; };

View File

@ -52,7 +52,8 @@ namespace DB
/// fields /// fields
static std::string name(); static std::string name();
static Block createBlock(); static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases();
void appendToBlock(MutableColumns & columns) const; void appendToBlock(MutableColumns & columns) const;
}; };
*/ */
@ -451,10 +452,18 @@ void SystemLog<LogElement>::flushImpl(const std::vector<LogElement> & to_flush,
/// is called from single thread. /// is called from single thread.
prepareTable(); prepareTable();
Block block = LogElement::createBlock(); ColumnsWithTypeAndName log_element_columns;
auto log_element_names_and_types = LogElement::getNamesAndTypes();
for (auto name_and_type : log_element_names_and_types)
log_element_columns.emplace_back(name_and_type.type, name_and_type.name);
Block block(std::move(log_element_columns));
MutableColumns columns = block.mutateColumns(); MutableColumns columns = block.mutateColumns();
for (const auto & elem : to_flush) for (const auto & elem : to_flush)
elem.appendToBlock(columns); elem.appendToBlock(columns);
block.setColumns(std::move(columns)); block.setColumns(std::move(columns));
/// We write to table indirectly, using InterpreterInsertQuery. /// We write to table indirectly, using InterpreterInsertQuery.
@ -500,11 +509,14 @@ void SystemLog<LogElement>::prepareTable()
if (table) if (table)
{ {
auto metadata_snapshot = table->getInMemoryMetadataPtr(); auto metadata_columns = table->getInMemoryMetadataPtr()->getColumns();
const Block expected = LogElement::createBlock(); auto old_query = InterpreterCreateQuery::formatColumns(metadata_columns);
const Block actual = metadata_snapshot->getSampleBlockNonMaterialized();
if (!blocksHaveEqualStructure(actual, expected)) auto ordinary_columns = LogElement::getNamesAndTypes();
auto alias_columns = LogElement::getNamesAndAliases();
auto current_query = InterpreterCreateQuery::formatColumns(ordinary_columns, alias_columns);
if (old_query->getTreeHash() != current_query->getTreeHash())
{ {
/// Rename the existing table. /// Rename the existing table.
int suffix = 0; int suffix = 0;
@ -575,10 +587,10 @@ ASTPtr SystemLog<LogElement>::getCreateTableQuery()
create->database = table_id.database_name; create->database = table_id.database_name;
create->table = table_id.table_name; create->table = table_id.table_name;
Block sample = LogElement::createBlock(); auto ordinary_columns = LogElement::getNamesAndTypes();
auto alias_columns = LogElement::getNamesAndAliases();
auto new_columns_list = std::make_shared<ASTColumns>(); auto new_columns_list = std::make_shared<ASTColumns>();
new_columns_list->set(new_columns_list->columns, InterpreterCreateQuery::formatColumns(sample.getNamesAndTypesList())); new_columns_list->set(new_columns_list->columns, InterpreterCreateQuery::formatColumns(ordinary_columns, alias_columns));
create->set(create->columns_list, new_columns_list); create->set(create->columns_list, new_columns_list);
ParserStorage storage_parser; ParserStorage storage_parser;

View File

@ -14,7 +14,7 @@
namespace DB namespace DB
{ {
Block TextLogElement::createBlock() NamesAndTypesList TextLogElement::getNamesAndTypes()
{ {
auto priority_datatype = std::make_shared<DataTypeEnum8>( auto priority_datatype = std::make_shared<DataTypeEnum8>(
DataTypeEnum8::Values DataTypeEnum8::Values
@ -31,23 +31,23 @@ Block TextLogElement::createBlock()
return return
{ {
{std::make_shared<DataTypeDate>(), "event_date"}, {"event_date", std::make_shared<DataTypeDate>()},
{std::make_shared<DataTypeDateTime>(), "event_time"}, {"event_time", std::make_shared<DataTypeDateTime>()},
{std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"}, {"event_time_microseconds", std::make_shared<DataTypeDateTime64>(6)},
{std::make_shared<DataTypeUInt32>(), "microseconds"}, {"microseconds", std::make_shared<DataTypeUInt32>()},
{std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), "thread_name"}, {"thread_name", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
{std::make_shared<DataTypeUInt64>(), "thread_id"}, {"thread_id", std::make_shared<DataTypeUInt64>()},
{std::move(priority_datatype), "level"}, {"level", std::move(priority_datatype)},
{std::make_shared<DataTypeString>(), "query_id"}, {"query_id", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), "logger_name"}, {"logger_name", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
{std::make_shared<DataTypeString>(), "message"}, {"message", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeUInt32>(), "revision"}, {"revision", std::make_shared<DataTypeUInt32>()},
{std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), "source_file"}, {"source_file", std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>())},
{std::make_shared<DataTypeUInt64>(), "source_line"} {"source_line", std::make_shared<DataTypeUInt64>()}
}; };
} }

View File

@ -25,7 +25,8 @@ struct TextLogElement
UInt64 source_line{}; UInt64 source_line{};
static std::string name() { return "TextLog"; } static std::string name() { return "TextLog"; }
static Block createBlock(); static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases() { return {}; }
void appendToBlock(MutableColumns & columns) const; void appendToBlock(MutableColumns & columns) const;
}; };

View File

@ -21,20 +21,20 @@ const TraceDataType::Values TraceLogElement::trace_values =
{"MemorySample", static_cast<UInt8>(TraceType::MemorySample)}, {"MemorySample", static_cast<UInt8>(TraceType::MemorySample)},
}; };
Block TraceLogElement::createBlock() NamesAndTypesList TraceLogElement::getNamesAndTypes()
{ {
return return
{ {
{std::make_shared<DataTypeDate>(), "event_date"}, {"event_date", std::make_shared<DataTypeDate>()},
{std::make_shared<DataTypeDateTime>(), "event_time"}, {"event_time", std::make_shared<DataTypeDateTime>()},
{std::make_shared<DataTypeDateTime64>(6), "event_time_microseconds"}, {"event_time_microseconds", std::make_shared<DataTypeDateTime64>(6)},
{std::make_shared<DataTypeUInt64>(), "timestamp_ns"}, {"timestamp_ns", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeUInt32>(), "revision"}, {"revision", std::make_shared<DataTypeUInt32>()},
{std::make_shared<TraceDataType>(trace_values), "trace_type"}, {"trace_type", std::make_shared<TraceDataType>(trace_values)},
{std::make_shared<DataTypeUInt64>(), "thread_id"}, {"thread_id", std::make_shared<DataTypeUInt64>()},
{std::make_shared<DataTypeString>(), "query_id"}, {"query_id", std::make_shared<DataTypeString>()},
{std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>()), "trace"}, {"trace", std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())},
{std::make_shared<DataTypeInt64>(), "size"}, {"size", std::make_shared<DataTypeInt64>()},
}; };
} }

View File

@ -27,7 +27,8 @@ struct TraceLogElement
Int64 size{}; /// Allocation size in bytes for TraceType::Memory Int64 size{}; /// Allocation size in bytes for TraceType::Memory
static std::string name() { return "TraceLog"; } static std::string name() { return "TraceLog"; }
static Block createBlock(); static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases() { return {}; }
void appendToBlock(MutableColumns & columns) const; void appendToBlock(MutableColumns & columns) const;
}; };

View File

@ -145,6 +145,25 @@ ColumnsDescription::ColumnsDescription(NamesAndTypesList ordinary)
add(ColumnDescription(std::move(elem.name), std::move(elem.type))); add(ColumnDescription(std::move(elem.name), std::move(elem.type)));
} }
ColumnsDescription::ColumnsDescription(NamesAndTypesList ordinary, NamesAndAliases aliases)
{
for (auto & elem : ordinary)
add(ColumnDescription(std::move(elem.name), std::move(elem.type)));
for (auto & alias : aliases)
{
ColumnDescription description(std::move(alias.name), std::move(alias.type));
description.default_desc.kind = ColumnDefaultKind::Alias;
const char * alias_expression_pos = alias.expression.data();
const char * alias_expression_end = alias_expression_pos + alias.expression.size();
ParserExpression expression_parser;
description.default_desc.expression = parseQuery(expression_parser, alias_expression_pos, alias_expression_end, "expression", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
add(std::move(description));
}
}
/// We are trying to find first column from end with name `column_name` or with a name beginning with `column_name` and ".". /// We are trying to find first column from end with name `column_name` or with a name beginning with `column_name` and ".".
/// For example "fruits.bananas" /// For example "fruits.bananas"

View File

@ -4,6 +4,7 @@
#include <Core/Block.h> #include <Core/Block.h>
#include <Core/Names.h> #include <Core/Names.h>
#include <Core/NamesAndTypes.h> #include <Core/NamesAndTypes.h>
#include <Core/NamesAndAliases.h>
#include <Interpreters/Context_fwd.h> #include <Interpreters/Context_fwd.h>
#include <Storages/ColumnCodec.h> #include <Storages/ColumnCodec.h>
#include <Storages/ColumnDefault.h> #include <Storages/ColumnDefault.h>
@ -54,7 +55,9 @@ class ColumnsDescription
{ {
public: public:
ColumnsDescription() = default; ColumnsDescription() = default;
explicit ColumnsDescription(NamesAndTypesList ordinary_); explicit ColumnsDescription(NamesAndTypesList ordinary);
explicit ColumnsDescription(NamesAndTypesList ordinary, NamesAndAliases aliases);
/// `after_column` can be a Nested column name; /// `after_column` can be a Nested column name;
void add(ColumnDescription column, const String & after_column = String(), bool first = false); void add(ColumnDescription column, const String & after_column = String(), bool first = false);

View File

@ -1,4 +1,6 @@
#pragma once #pragma once
#include <Core/NamesAndAliases.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
#include <Storages/ColumnsDescription.h> #include <Storages/ColumnsDescription.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
@ -29,6 +31,7 @@ class IStorageSystemOneBlock : public IStorage
protected: protected:
virtual void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const = 0; virtual void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const = 0;
public: public:
#if defined(ARCADIA_BUILD) #if defined(ARCADIA_BUILD)
IStorageSystemOneBlock(const String & name_) : IStorageSystemOneBlock(StorageID{"system", name_}) {} IStorageSystemOneBlock(const String & name_) : IStorageSystemOneBlock(StorageID{"system", name_}) {}
@ -37,7 +40,7 @@ public:
IStorageSystemOneBlock(const StorageID & table_id_) : IStorage(table_id_) IStorageSystemOneBlock(const StorageID & table_id_) : IStorage(table_id_)
{ {
StorageInMemoryMetadata metadata_; StorageInMemoryMetadata metadata_;
metadata_.setColumns(ColumnsDescription(Self::getNamesAndTypes())); metadata_.setColumns(ColumnsDescription(Self::getNamesAndTypes(), Self::getNamesAndAliases()));
setInMemoryMetadata(metadata_); setInMemoryMetadata(metadata_);
} }
@ -62,6 +65,8 @@ public:
return Pipe(std::make_shared<SourceFromSingleChunk>(sample_block, std::move(chunk))); return Pipe(std::make_shared<SourceFromSingleChunk>(sample_block, std::move(chunk)));
} }
static NamesAndAliases getNamesAndAliases() { return {}; }
}; };
} }

View File

@ -68,6 +68,14 @@ NamesAndTypesList StorageSystemProcesses::getNamesAndTypes()
}; };
} }
NamesAndAliases StorageSystemProcesses::getNamesAndAliases()
{
return
{
{"ProfileEvents.Names", {std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>())}, "mapKeys(ProfileEvents)"},
{"ProfileEvents.Values", {std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>())}, "mapValues(ProfileEvents)"}
};
}
void StorageSystemProcesses::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const void StorageSystemProcesses::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{ {

View File

@ -20,6 +20,8 @@ public:
static NamesAndTypesList getNamesAndTypes(); static NamesAndTypesList getNamesAndTypes();
static NamesAndAliases getNamesAndAliases();
protected: protected:
using IStorageSystemOneBlock::IStorageSystemOneBlock; using IStorageSystemOneBlock::IStorageSystemOneBlock;

View File

@ -6,6 +6,6 @@ SET min_bytes_to_use_mmap_io = 1;
SELECT * FROM test_01343; SELECT * FROM test_01343;
SYSTEM FLUSH LOGS; SYSTEM FLUSH LOGS;
SELECT ProfileEvents['CreatedReadBufferMMap'] as value FROM system.query_log WHERE current_database = currentDatabase() and event_date >= yesterday() AND event_time >= now() - 300 AND query LIKE 'SELECT * FROM test_01343%' AND type = 2 ORDER BY event_time DESC LIMIT 1; SELECT ProfileEvents['CreatedReadBufferMMap'] AS value FROM system.query_log WHERE current_database = currentDatabase() AND event_date >= yesterday() AND event_time >= now() - 300 AND query LIKE 'SELECT * FROM test_01343%' AND type = 2 ORDER BY event_time DESC LIMIT 1;
DROP TABLE test_01343; DROP TABLE test_01343;

View File

@ -7,6 +7,6 @@ SELECT * FROM test_01344 WHERE x = 'Hello, world';
SYSTEM FLUSH LOGS; SYSTEM FLUSH LOGS;
SELECT ProfileEvents['CreatedReadBufferMMap'] as value FROM system.query_log SELECT ProfileEvents['CreatedReadBufferMMap'] as value FROM system.query_log
WHERE current_database = currentDatabase() and event_date >= yesterday() AND query LIKE 'SELECT * FROM test_01344 WHERE x = ''Hello, world''%' AND type = 2 ORDER BY event_time DESC LIMIT 1; WHERE current_database = currentDatabase() AND event_date >= yesterday() AND query LIKE 'SELECT * FROM test_01344 WHERE x = ''Hello, world''%' AND type = 2 ORDER BY event_time DESC LIMIT 1;
DROP TABLE test_01344; DROP TABLE test_01344;

View File

@ -9,8 +9,6 @@ SELECT 1;
SYSTEM FLUSH LOGS; SYSTEM FLUSH LOGS;
SET allow_experimental_map_type = 1;
-- NOTE: can be rewritten using log_queries_min_query_duration_ms -- NOTE: can be rewritten using log_queries_min_query_duration_ms
CREATE MATERIALIZED VIEW slow_log Engine=Memory AS CREATE MATERIALIZED VIEW slow_log Engine=Memory AS