diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 2ec4a2f60db..dad4362b3d1 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -380,7 +380,7 @@ do done # 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, 'test text, query_index int, query_id text, version UInt8, time float'); @@ -507,7 +507,7 @@ do file="analyze/tmp/${prefix// /_}.tsv" grep "^$prefix " "analyze/query-run-metrics-for-stats.tsv" > "$file" & printf "%s\0\n" \ - "clickhouse-local --allow_experimental_map_type 1 \ + "clickhouse-local \ --file \"$file\" \ --structure 'test text, query text, run int, version UInt8, metrics Array(float)' \ --query \"$(cat "$script_dir/eqmed.sql")\" \ @@ -526,7 +526,7 @@ 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 -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 -- a denormalized table of them -- statistics for all metrics for all queries. -- 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 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 file('analyze/query-display-names.tsv', TSV, '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} do rm -rf data - clickhouse-local --allow_experimental_map_type 1 --query " + clickhouse-local --query " create view query_profiles as with 0 as left, 1 as right select * from file('analyze/query-profiles.tsv', TSV, @@ -1168,7 +1168,7 @@ build_log_column_definitions rm -rf metrics ||: mkdir metrics -clickhouse-local --allow_experimental_map_type 1 --query " +clickhouse-local --query " create view right_async_metric_log as select * from file('right-async-metric-log.tsv', TSVWithNamesAndTypes, '$(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 client=(clickhouse-client # 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 # 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 --user "${CHPC_DATABASE_USER}" --password "${CHPC_DATABASE_PASSWORD}" @@ -1352,7 +1352,7 @@ EOF } # 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 case "$stage" in diff --git a/docs/zh/operations/system-tables/query_log.md b/docs/zh/operations/system-tables/query_log.md index 5c25d9725a7..b7661b73a50 100644 --- a/docs/zh/operations/system-tables/query_log.md +++ b/docs/zh/operations/system-tables/query_log.md @@ -76,14 +76,14 @@ ClickHouse不会自动从表中删除数据。更多详情请看 [introduction]( - `client_version_patch` ([UInt32](../../sql-reference/data-types/int-uint.md)) — [clickhouse-client](../../interfaces/cli.md) 或其他TCP客户端的Patch component。 - `http_method` (UInt8) — 发起查询的HTTP方法. 可能值: - 0 — TCP接口的查询. - - 1 — `GET` + - 1 — `GET` - 2 — `POST` - `http_user_agent` ([String](../../sql-reference/data-types/string.md)) — The `UserAgent` The UserAgent header passed in the HTTP request。 - `quota_key` ([String](../../sql-reference/data-types/string.md)) — 在[quotas](../../operations/quotas.md) 配置里设置的“quota key” (见 `keyed`). - `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) - `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` ([Array(String)](../../sql-reference/data-types/array.md)) — 客户端运行查询时更改的设置的名称。 要启用对设置的日志记录更改,请将log_query_settings参数设置为1。 - `Settings.Values` ([Array(String)](../../sql-reference/data-types/array.md)) — `Settings.Names` 列中列出的设置的值。 **示例** diff --git a/src/Core/NamesAndAliases.h b/src/Core/NamesAndAliases.h new file mode 100644 index 00000000000..694d4095ace --- /dev/null +++ b/src/Core/NamesAndAliases.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +#include +#include + +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 +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; + +} + +namespace std +{ + template <> struct tuple_size : std::integral_constant {}; + 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; }; +} diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index a72992d4af7..11c625007d9 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -77,6 +77,7 @@ void Settings::dumpToMapColumn(IColumn * column, bool changed_only) value_column.insert(setting.getValueString()); size++; } + offsets.push_back(offsets.back() + size); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ad34bec3ae6..0197bfac7e4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -441,7 +441,7 @@ class IColumn; 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_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_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) \ @@ -596,7 +596,7 @@ struct Settings : public BaseSettings /// Load settings from configuration file, at "path" prefix in configuration. 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); /// Adds program options to set the settings from a command line. diff --git a/src/Interpreters/AsynchronousMetricLog.cpp b/src/Interpreters/AsynchronousMetricLog.cpp index 6c20a04d291..79e2d513d5c 100644 --- a/src/Interpreters/AsynchronousMetricLog.cpp +++ b/src/Interpreters/AsynchronousMetricLog.cpp @@ -11,17 +11,16 @@ namespace DB { -Block AsynchronousMetricLogElement::createBlock() +NamesAndTypesList AsynchronousMetricLogElement::getNamesAndTypes() { - ColumnsWithTypeAndName columns; - - columns.emplace_back(std::make_shared(), "event_date"); - columns.emplace_back(std::make_shared(), "event_time"); - columns.emplace_back(std::make_shared(6), "event_time_microseconds"); - columns.emplace_back(std::make_shared(std::make_shared()), "name"); - columns.emplace_back(std::make_shared(), "value"); - - return Block(columns); + return + { + {"event_date", std::make_shared()}, + {"event_time", std::make_shared()}, + {"event_time_microseconds", std::make_shared(6)}, + {"name", std::make_shared(std::make_shared())}, + {"value", std::make_shared(),} + }; } diff --git a/src/Interpreters/AsynchronousMetricLog.h b/src/Interpreters/AsynchronousMetricLog.h index 30bac3f5a99..6275572935c 100644 --- a/src/Interpreters/AsynchronousMetricLog.h +++ b/src/Interpreters/AsynchronousMetricLog.h @@ -12,8 +12,8 @@ namespace DB { -typedef double AsynchronousMetricValue; -typedef std::unordered_map AsynchronousMetricValues; +using AsynchronousMetricValue = double; +using AsynchronousMetricValues = std::unordered_map; /** AsynchronousMetricLog is a log of metric values measured at regular time interval. */ @@ -27,7 +27,8 @@ struct AsynchronousMetricLogElement double value; static std::string name() { return "AsynchronousMetricLog"; } - static Block createBlock(); + static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases() { return {}; } void appendToBlock(MutableColumns & columns) const; }; diff --git a/src/Interpreters/CrashLog.cpp b/src/Interpreters/CrashLog.cpp index 5067acd4a5c..a9da804f1d2 100644 --- a/src/Interpreters/CrashLog.cpp +++ b/src/Interpreters/CrashLog.cpp @@ -18,21 +18,21 @@ namespace DB std::weak_ptr CrashLog::crash_log; -Block CrashLogElement::createBlock() +NamesAndTypesList CrashLogElement::getNamesAndTypes() { return { - {std::make_shared(), "event_date"}, - {std::make_shared(), "event_time"}, - {std::make_shared(), "timestamp_ns"}, - {std::make_shared(), "signal"}, - {std::make_shared(), "thread_id"}, - {std::make_shared(), "query_id"}, - {std::make_shared(std::make_shared()), "trace"}, - {std::make_shared(std::make_shared()), "trace_full"}, - {std::make_shared(), "version"}, - {std::make_shared(), "revision"}, - {std::make_shared(), "build_id"}, + {"event_date", std::make_shared()}, + {"event_time", std::make_shared()}, + {"timestamp_ns", std::make_shared()}, + {"signal", std::make_shared()}, + {"thread_id", std::make_shared()}, + {"query_id", std::make_shared()}, + {"trace", std::make_shared(std::make_shared())}, + {"trace_full", std::make_shared(std::make_shared())}, + {"version", std::make_shared()}, + {"revision", std::make_shared()}, + {"build_id", std::make_shared()}, }; } @@ -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) { using namespace DB; diff --git a/src/Interpreters/CrashLog.h b/src/Interpreters/CrashLog.h index 9494bdc85c9..ba27c1f513e 100644 --- a/src/Interpreters/CrashLog.h +++ b/src/Interpreters/CrashLog.h @@ -24,7 +24,8 @@ struct CrashLogElement Array trace_full; static std::string name() { return "CrashLog"; } - static Block createBlock(); + static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases() { return {}; } void appendToBlock(MutableColumns & columns) const; }; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6c8bdfd765f..de858bdbdc5 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -302,6 +302,35 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns) return columns_list; } +ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns, const NamesAndAliases & alias_columns) +{ + std::shared_ptr columns_list = std::static_pointer_cast(formatColumns(columns)); + + for (const auto & alias_column : alias_columns) + { + const auto column_declaration = std::make_shared(); + 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) { auto columns_list = std::make_shared(); @@ -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 diff --git a/src/Interpreters/InterpreterCreateQuery.h b/src/Interpreters/InterpreterCreateQuery.h index 45f0bbd7cf8..7bd3ef25746 100644 --- a/src/Interpreters/InterpreterCreateQuery.h +++ b/src/Interpreters/InterpreterCreateQuery.h @@ -1,12 +1,12 @@ #pragma once +#include #include #include #include #include #include #include -#include namespace DB @@ -31,8 +31,8 @@ public: /// List of columns and their types in AST. static ASTPtr formatColumns(const NamesAndTypesList & columns); + static ASTPtr formatColumns(const NamesAndTypesList & columns, const NamesAndAliases & alias_columns); static ASTPtr formatColumns(const ColumnsDescription & columns); - static ASTPtr formatIndices(const IndicesDescription & indices); static ASTPtr formatConstraints(const ConstraintsDescription & constraints); static ASTPtr formatProjections(const ProjectionsDescription & projections); diff --git a/src/Interpreters/MetricLog.cpp b/src/Interpreters/MetricLog.cpp index fd1c120f18c..f8fab6d7a9d 100644 --- a/src/Interpreters/MetricLog.cpp +++ b/src/Interpreters/MetricLog.cpp @@ -8,21 +8,21 @@ 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(), "event_date"); - columns_with_type_and_name.emplace_back(std::make_shared(), "event_time"); - columns_with_type_and_name.emplace_back(std::make_shared(6), "event_time_microseconds"); - columns_with_type_and_name.emplace_back(std::make_shared(), "milliseconds"); + columns_with_type_and_name.emplace_back("event_date", std::make_shared()); + columns_with_type_and_name.emplace_back("event_time", std::make_shared()); + columns_with_type_and_name.emplace_back("event_time_microseconds", std::make_shared(6)); + columns_with_type_and_name.emplace_back("milliseconds", std::make_shared()); 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(), std::move(name)); + columns_with_type_and_name.emplace_back(std::move(name), std::make_shared()); } for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) @@ -30,10 +30,10 @@ Block MetricLogElement::createBlock() std::string name; name += "CurrentMetric_"; name += CurrentMetrics::getName(CurrentMetrics::Metric(i)); - columns_with_type_and_name.emplace_back(std::make_shared(), std::move(name)); + columns_with_type_and_name.emplace_back(std::move(name), std::make_shared()); } - return Block(columns_with_type_and_name); + return columns_with_type_and_name; } diff --git a/src/Interpreters/MetricLog.h b/src/Interpreters/MetricLog.h index f03b682c60a..c43c2872788 100644 --- a/src/Interpreters/MetricLog.h +++ b/src/Interpreters/MetricLog.h @@ -25,7 +25,8 @@ struct MetricLogElement std::vector current_metrics; static std::string name() { return "MetricLog"; } - static Block createBlock(); + static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases() { return {}; } void appendToBlock(MutableColumns & columns) const; }; diff --git a/src/Interpreters/OpenTelemetrySpanLog.cpp b/src/Interpreters/OpenTelemetrySpanLog.cpp index d54c6ba9afc..89cce890555 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.cpp +++ b/src/Interpreters/OpenTelemetrySpanLog.cpp @@ -15,13 +15,13 @@ namespace DB { -Block OpenTelemetrySpanLogElement::createBlock() +NamesAndTypesList OpenTelemetrySpanLogElement::getNamesAndTypes() { return { - {std::make_shared(), "trace_id"}, - {std::make_shared(), "span_id"}, - {std::make_shared(), "parent_span_id"}, - {std::make_shared(), "operation_name"}, + {"trace_id", std::make_shared()}, + {"span_id", std::make_shared()}, + {"parent_span_id", std::make_shared()}, + {"operation_name", std::make_shared()}, // DateTime64 is really unwieldy -- there is no "normal" way to convert // it to an UInt64 count of microseconds, except: // 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 // get duration. // It is much less hassle to just use UInt64 of microseconds. - {std::make_shared(), "start_time_us"}, - {std::make_shared(), "finish_time_us"}, - {std::make_shared(), "finish_date"}, - {std::make_shared(std::make_shared(), std::make_shared()), - "attribute"}, + {"start_time_us", std::make_shared()}, + {"finish_time_us", std::make_shared()}, + {"finish_date", std::make_shared()}, + {"attribute", std::make_shared(std::make_shared(), std::make_shared())}, }; } +NamesAndAliases OpenTelemetrySpanLogElement::getNamesAndAliases() +{ + return + { + {"attribute.names", std::make_shared(std::make_shared()), "mapKeys(attribute)"}, + {"attribute.values", std::make_shared(std::make_shared()), "mapKeys(attribute)"} + }; +} void OpenTelemetrySpanLogElement::appendToBlock(MutableColumns & columns) const { diff --git a/src/Interpreters/OpenTelemetrySpanLog.h b/src/Interpreters/OpenTelemetrySpanLog.h index 861293b4f98..b287301325c 100644 --- a/src/Interpreters/OpenTelemetrySpanLog.h +++ b/src/Interpreters/OpenTelemetrySpanLog.h @@ -27,7 +27,8 @@ struct OpenTelemetrySpanLogElement : public OpenTelemetrySpan : OpenTelemetrySpan(span) {} static std::string name() { return "OpenTelemetrySpanLog"; } - static Block createBlock(); + static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases(); void appendToBlock(MutableColumns & columns) const; }; diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index ad4fb60f00c..f89f836871a 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -16,7 +16,7 @@ namespace DB { -Block PartLogElement::createBlock() +NamesAndTypesList PartLogElement::getNamesAndTypes() { auto event_type_datatype = std::make_shared( DataTypeEnum8::Values @@ -33,35 +33,34 @@ Block PartLogElement::createBlock() ColumnsWithTypeAndName columns_with_type_and_name; return { + {"query_id", std::make_shared()}, + {"event_type", std::move(event_type_datatype)}, + {"event_date", std::make_shared()}, - columns_with_type_and_name.emplace_back(std::make_shared(), "query_id"), - columns_with_type_and_name.emplace_back(std::move(event_type_datatype), "event_type"), - columns_with_type_and_name.emplace_back(std::make_shared(), "event_date"), + {"event_time", std::make_shared()}, + {"event_time_microseconds", std::make_shared(6)}, - columns_with_type_and_name.emplace_back(std::make_shared(), "event_time"), - columns_with_type_and_name.emplace_back(std::make_shared(6), "event_time_microseconds"), + {"duration_ms", std::make_shared()}, - columns_with_type_and_name.emplace_back(std::make_shared(), "duration_ms"), + {"database", std::make_shared()}, + {"table", std::make_shared()}, + {"part_name", std::make_shared()}, + {"partition_id", std::make_shared()}, + {"path_on_disk", std::make_shared()}, - columns_with_type_and_name.emplace_back(std::make_shared(), "database"), - columns_with_type_and_name.emplace_back(std::make_shared(), "table"), - columns_with_type_and_name.emplace_back(std::make_shared(), "part_name"), - columns_with_type_and_name.emplace_back(std::make_shared(), "partition_id"), - columns_with_type_and_name.emplace_back(std::make_shared(), "path_on_disk"), - - columns_with_type_and_name.emplace_back(std::make_shared(), "rows"), - columns_with_type_and_name.emplace_back(std::make_shared(), "size_in_bytes"), // On disk + {"rows", std::make_shared()}, + {"size_in_bytes", std::make_shared()}, // On disk /// Merge-specific info - columns_with_type_and_name.emplace_back(std::make_shared(std::make_shared()), "merged_from"), - columns_with_type_and_name.emplace_back(std::make_shared(), "bytes_uncompressed"), // Result bytes - columns_with_type_and_name.emplace_back(std::make_shared(), "read_rows"), - columns_with_type_and_name.emplace_back(std::make_shared(), "read_bytes"), - columns_with_type_and_name.emplace_back(std::make_shared(), "peak_memory_usage"), + {"merged_from", std::make_shared(std::make_shared())}, + {"bytes_uncompressed", std::make_shared()}, // Result bytes + {"read_rows", std::make_shared()}, + {"read_bytes", std::make_shared()}, + {"peak_memory_usage", std::make_shared()}, /// Is there an error during the execution or commit - columns_with_type_and_name.emplace_back(std::make_shared(), "error"), - columns_with_type_and_name.emplace_back(std::make_shared(), "exception"), + {"error", std::make_shared()}, + {"exception", std::make_shared()}, }; } diff --git a/src/Interpreters/PartLog.h b/src/Interpreters/PartLog.h index edb6ab4a45f..1aec850e3dc 100644 --- a/src/Interpreters/PartLog.h +++ b/src/Interpreters/PartLog.h @@ -52,7 +52,8 @@ struct PartLogElement static std::string name() { return "PartLog"; } - static Block createBlock(); + static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases() { return {}; } void appendToBlock(MutableColumns & columns) const; }; diff --git a/src/Interpreters/ProfileEventsExt.cpp b/src/Interpreters/ProfileEventsExt.cpp index ec3131d39a3..2e8f986ca6c 100644 --- a/src/Interpreters/ProfileEventsExt.cpp +++ b/src/Interpreters/ProfileEventsExt.cpp @@ -36,6 +36,7 @@ void dumpToMapColumn(const Counters & counters, DB::IColumn * column, bool nonze value_column.insert(value); size++; } + offsets.push_back(offsets.back() + size); } diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index e8647e3c192..3f668e5e0ab 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -26,7 +26,7 @@ namespace DB { -Block QueryLogElement::createBlock() +NamesAndTypesList QueryLogElement::getNamesAndTypes() { auto query_status_datatype = std::make_shared( DataTypeEnum8::Values @@ -39,84 +39,94 @@ Block QueryLogElement::createBlock() return { - {std::move(query_status_datatype), "type"}, - {std::make_shared(), "event_date"}, - {std::make_shared(), "event_time"}, - {std::make_shared(6), "event_time_microseconds"}, - {std::make_shared(), "query_start_time"}, - {std::make_shared(6), "query_start_time_microseconds"}, - {std::make_shared(), "query_duration_ms"}, + {"type", std::move(query_status_datatype)}, + {"event_date", std::make_shared()}, + {"event_time", std::make_shared()}, + {"event_time_microseconds", std::make_shared(6)}, + {"query_start_time", std::make_shared()}, + {"query_start_time_microseconds", std::make_shared(6)}, + {"query_duration_ms", std::make_shared()}, - {std::make_shared(), "read_rows"}, - {std::make_shared(), "read_bytes"}, - {std::make_shared(), "written_rows"}, - {std::make_shared(), "written_bytes"}, - {std::make_shared(), "result_rows"}, - {std::make_shared(), "result_bytes"}, - {std::make_shared(), "memory_usage"}, + {"read_rows", std::make_shared()}, + {"read_bytes", std::make_shared()}, + {"written_rows", std::make_shared()}, + {"written_bytes", std::make_shared()}, + {"result_rows", std::make_shared()}, + {"result_bytes", std::make_shared()}, + {"memory_usage", std::make_shared()}, - {std::make_shared(), "current_database"}, - {std::make_shared(), "query"}, - {std::make_shared(), "normalized_query_hash"}, - {std::make_shared(std::make_shared()), "query_kind"}, - {std::make_shared( - std::make_shared(std::make_shared())), "databases"}, - {std::make_shared( - std::make_shared(std::make_shared())), "tables"}, - {std::make_shared( - std::make_shared(std::make_shared())), "columns"}, - {std::make_shared( - std::make_shared(std::make_shared())), "projections"}, - {std::make_shared(), "exception_code"}, - {std::make_shared(), "exception"}, - {std::make_shared(), "stack_trace"}, + {"current_database", std::make_shared()}, + {"query", std::make_shared()}, + {"normalized_query_hash", std::make_shared()}, + {"query_kind", std::make_shared(std::make_shared())}, + {"databases", std::make_shared( + std::make_shared(std::make_shared()))}, + {"tables", std::make_shared( + std::make_shared(std::make_shared()))}, + {"columns", std::make_shared( + std::make_shared(std::make_shared()))}, + {"projections", std::make_shared( + std::make_shared(std::make_shared()))}, + {"exception_code", std::make_shared()}, + {"exception", std::make_shared()}, + {"stack_trace", std::make_shared()}, - {std::make_shared(), "is_initial_query"}, - {std::make_shared(), "user"}, - {std::make_shared(), "query_id"}, - {DataTypeFactory::instance().get("IPv6"), "address"}, - {std::make_shared(), "port"}, - {std::make_shared(), "initial_user"}, - {std::make_shared(), "initial_query_id"}, - {DataTypeFactory::instance().get("IPv6"), "initial_address"}, - {std::make_shared(), "initial_port"}, - {std::make_shared(), "initial_query_start_time"}, - {std::make_shared(6), "initial_query_start_time_microseconds"}, - {std::make_shared(), "interface"}, - {std::make_shared(), "os_user"}, - {std::make_shared(), "client_hostname"}, - {std::make_shared(), "client_name"}, - {std::make_shared(), "client_revision"}, - {std::make_shared(), "client_version_major"}, - {std::make_shared(), "client_version_minor"}, - {std::make_shared(), "client_version_patch"}, - {std::make_shared(), "http_method"}, - {std::make_shared(), "http_user_agent"}, - {std::make_shared(), "http_referer"}, - {std::make_shared(), "forwarded_for"}, - {std::make_shared(), "quota_key"}, + {"is_initial_query", std::make_shared()}, + {"user", std::make_shared()}, + {"query_id", std::make_shared()}, + {"address", DataTypeFactory::instance().get("IPv6")}, + {"port", std::make_shared()}, + {"initial_user", std::make_shared()}, + {"initial_query_id", std::make_shared()}, + {"initial_address", DataTypeFactory::instance().get("IPv6")}, + {"initial_port", std::make_shared()}, + {"initial_query_start_time", std::make_shared()}, + {"initial_query_start_time_microseconds", std::make_shared(6)}, + {"interface", std::make_shared()}, + {"os_user", std::make_shared()}, + {"client_hostname", std::make_shared()}, + {"client_name", std::make_shared()}, + {"client_revision", std::make_shared()}, + {"client_version_major", std::make_shared()}, + {"client_version_minor", std::make_shared()}, + {"client_version_patch", std::make_shared()}, + {"http_method", std::make_shared()}, + {"http_user_agent", std::make_shared()}, + {"http_referer", std::make_shared()}, + {"forwarded_for", std::make_shared()}, + {"quota_key", std::make_shared()}, - {std::make_shared(), "revision"}, + {"revision", std::make_shared()}, - {std::make_shared(), "log_comment"}, + {"log_comment", std::make_shared()}, - {std::make_shared(std::make_shared()), "thread_ids"}, - {std::make_shared(std::make_shared(), std::make_shared()), "ProfileEvents"}, - {std::make_shared(std::make_shared(), std::make_shared()), "Settings"}, + {"thread_ids", std::make_shared(std::make_shared())}, + {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, + {"Settings", std::make_shared(std::make_shared(), std::make_shared())}, - {std::make_shared(std::make_shared()), "used_aggregate_functions"}, - {std::make_shared(std::make_shared()), "used_aggregate_function_combinators"}, - {std::make_shared(std::make_shared()), "used_database_engines"}, - {std::make_shared(std::make_shared()), "used_data_type_families"}, - {std::make_shared(std::make_shared()), "used_dictionaries"}, - {std::make_shared(std::make_shared()), "used_formats"}, - {std::make_shared(std::make_shared()), "used_functions"}, - {std::make_shared(std::make_shared()), "used_storages"}, - {std::make_shared(std::make_shared()), "used_table_functions"} + {"used_aggregate_functions", std::make_shared(std::make_shared())}, + {"used_aggregate_function_combinators", std::make_shared(std::make_shared())}, + {"used_database_engines", std::make_shared(std::make_shared())}, + {"used_data_type_families", std::make_shared(std::make_shared())}, + {"used_dictionaries", std::make_shared(std::make_shared())}, + {"used_formats", std::make_shared(std::make_shared())}, + {"used_functions", std::make_shared(std::make_shared())}, + {"used_storages", std::make_shared(std::make_shared())}, + {"used_table_functions", std::make_shared(std::make_shared())} }; } +NamesAndAliases QueryLogElement::getNamesAndAliases() +{ + return + { + {"ProfileEvents.Names", {std::make_shared(std::make_shared())}, "mapKeys(ProfileEvents)"}, + {"ProfileEvents.Values", {std::make_shared(std::make_shared())}, "mapValues(ProfileEvents)"}, + {"Settings.Names", {std::make_shared(std::make_shared())}, "mapKeys(Settings)" }, + {"Settings.Values", {std::make_shared(std::make_shared())}, "mapValues(Settings)"} + }; +} void QueryLogElement::appendToBlock(MutableColumns & columns) const { diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 684a635a920..0aa02104306 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -1,9 +1,9 @@ #pragma once +#include #include #include - namespace ProfileEvents { class Counters; @@ -83,7 +83,8 @@ struct QueryLogElement static std::string name() { return "QueryLog"; } - static Block createBlock(); + static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases(); void appendToBlock(MutableColumns & columns) const; static void appendClientInfo(const ClientInfo & client_info, MutableColumns & columns, size_t & i); diff --git a/src/Interpreters/QueryThreadLog.cpp b/src/Interpreters/QueryThreadLog.cpp index dfdb905f229..7ca3c10045e 100644 --- a/src/Interpreters/QueryThreadLog.cpp +++ b/src/Interpreters/QueryThreadLog.cpp @@ -19,58 +19,68 @@ namespace DB { -Block QueryThreadLogElement::createBlock() + +NamesAndTypesList QueryThreadLogElement::getNamesAndTypes() { return { - {std::make_shared(), "event_date"}, - {std::make_shared(), "event_time"}, - {std::make_shared(6), "event_time_microseconds"}, - {std::make_shared(), "query_start_time"}, - {std::make_shared(6), "query_start_time_microseconds"}, - {std::make_shared(), "query_duration_ms"}, + {"event_date", std::make_shared()}, + {"event_time", std::make_shared()}, + {"event_time_microseconds", std::make_shared(6)}, + {"query_start_time", std::make_shared()}, + {"query_start_time_microseconds", std::make_shared(6)}, + {"query_duration_ms", std::make_shared()}, - {std::make_shared(), "read_rows"}, - {std::make_shared(), "read_bytes"}, - {std::make_shared(), "written_rows"}, - {std::make_shared(), "written_bytes"}, - {std::make_shared(), "memory_usage"}, - {std::make_shared(), "peak_memory_usage"}, + {"read_rows", std::make_shared()}, + {"read_bytes", std::make_shared()}, + {"written_rows", std::make_shared()}, + {"written_bytes", std::make_shared()}, + {"memory_usage", std::make_shared()}, + {"peak_memory_usage", std::make_shared()}, - {std::make_shared(), "thread_name"}, - {std::make_shared(), "thread_id"}, - {std::make_shared(), "master_thread_id"}, - {std::make_shared(), "current_database"}, - {std::make_shared(), "query"}, - {std::make_shared(), "normalized_query_hash"}, + {"thread_name", std::make_shared()}, + {"thread_id", std::make_shared()}, + {"master_thread_id", std::make_shared()}, + {"current_database", std::make_shared()}, + {"query", std::make_shared()}, + {"normalized_query_hash", std::make_shared()}, - {std::make_shared(), "is_initial_query"}, - {std::make_shared(), "user"}, - {std::make_shared(), "query_id"}, - {DataTypeFactory::instance().get("IPv6"), "address"}, - {std::make_shared(), "port"}, - {std::make_shared(), "initial_user"}, - {std::make_shared(), "initial_query_id"}, - {DataTypeFactory::instance().get("IPv6"), "initial_address"}, - {std::make_shared(), "initial_port"}, - {std::make_shared(), "initial_query_start_time"}, - {std::make_shared(6), "initial_query_start_time_microseconds"}, - {std::make_shared(), "interface"}, - {std::make_shared(), "os_user"}, - {std::make_shared(), "client_hostname"}, - {std::make_shared(), "client_name"}, - {std::make_shared(), "client_revision"}, - {std::make_shared(), "client_version_major"}, - {std::make_shared(), "client_version_minor"}, - {std::make_shared(), "client_version_patch"}, - {std::make_shared(), "http_method"}, - {std::make_shared(), "http_user_agent"}, - {std::make_shared(), "http_referer"}, - {std::make_shared(), "forwarded_for"}, - {std::make_shared(), "quota_key"}, + {"is_initial_query", std::make_shared()}, + {"user", std::make_shared()}, + {"query_id", std::make_shared()}, + {"address", DataTypeFactory::instance().get("IPv6")}, + {"port", std::make_shared()}, + {"initial_user", std::make_shared()}, + {"initial_query_id", std::make_shared()}, + {"initial_address", DataTypeFactory::instance().get("IPv6")}, + {"initial_port", std::make_shared()}, + {"initial_query_start_time", std::make_shared()}, + {"initial_query_start_time_microseconds", std::make_shared(6)}, + {"interface", std::make_shared()}, + {"os_user", std::make_shared()}, + {"client_hostname", std::make_shared()}, + {"client_name", std::make_shared()}, + {"client_revision", std::make_shared()}, + {"client_version_major", std::make_shared()}, + {"client_version_minor", std::make_shared()}, + {"client_version_patch", std::make_shared()}, + {"http_method", std::make_shared()}, + {"http_user_agent", std::make_shared()}, + {"http_referer", std::make_shared()}, + {"forwarded_for", std::make_shared()}, + {"quota_key", std::make_shared()}, - {std::make_shared(), "revision"}, + {"revision", std::make_shared()}, - {std::make_shared(std::make_shared(), std::make_shared()), "ProfileEvents"}, + {"ProfileEvents", std::make_shared(std::make_shared(), std::make_shared())}, + }; +} + +NamesAndAliases QueryThreadLogElement::getNamesAndAliases() +{ + return + { + {"ProfileEvents.Names", {std::make_shared(std::make_shared())}, "mapKeys(ProfileEvents)"}, + {"ProfileEvents.Values", {std::make_shared(std::make_shared())}, "mapValues(ProfileEvents)"} }; } diff --git a/src/Interpreters/QueryThreadLog.h b/src/Interpreters/QueryThreadLog.h index 6771e8ba88c..57e93edbaf7 100644 --- a/src/Interpreters/QueryThreadLog.h +++ b/src/Interpreters/QueryThreadLog.h @@ -49,7 +49,8 @@ struct QueryThreadLogElement static std::string name() { return "QueryThreadLog"; } - static Block createBlock(); + static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases(); void appendToBlock(MutableColumns & columns) const; }; diff --git a/src/Interpreters/SystemLog.h b/src/Interpreters/SystemLog.h index 9e57e308dbe..ee3116362e5 100644 --- a/src/Interpreters/SystemLog.h +++ b/src/Interpreters/SystemLog.h @@ -52,7 +52,8 @@ namespace DB /// fields static std::string name(); - static Block createBlock(); + static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases(); void appendToBlock(MutableColumns & columns) const; }; */ @@ -451,10 +452,18 @@ void SystemLog::flushImpl(const std::vector & to_flush, /// is called from single thread. 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(); for (const auto & elem : to_flush) elem.appendToBlock(columns); + block.setColumns(std::move(columns)); /// We write to table indirectly, using InterpreterInsertQuery. @@ -500,11 +509,14 @@ void SystemLog::prepareTable() if (table) { - auto metadata_snapshot = table->getInMemoryMetadataPtr(); - const Block expected = LogElement::createBlock(); - const Block actual = metadata_snapshot->getSampleBlockNonMaterialized(); + auto metadata_columns = table->getInMemoryMetadataPtr()->getColumns(); + auto old_query = InterpreterCreateQuery::formatColumns(metadata_columns); - 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. int suffix = 0; @@ -575,10 +587,10 @@ ASTPtr SystemLog::getCreateTableQuery() create->database = table_id.database_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(); - 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); ParserStorage storage_parser; diff --git a/src/Interpreters/TextLog.cpp b/src/Interpreters/TextLog.cpp index f5a0ce51d49..baf98b6771d 100644 --- a/src/Interpreters/TextLog.cpp +++ b/src/Interpreters/TextLog.cpp @@ -14,7 +14,7 @@ namespace DB { -Block TextLogElement::createBlock() +NamesAndTypesList TextLogElement::getNamesAndTypes() { auto priority_datatype = std::make_shared( DataTypeEnum8::Values @@ -31,23 +31,23 @@ Block TextLogElement::createBlock() return { - {std::make_shared(), "event_date"}, - {std::make_shared(), "event_time"}, - {std::make_shared(6), "event_time_microseconds"}, - {std::make_shared(), "microseconds"}, + {"event_date", std::make_shared()}, + {"event_time", std::make_shared()}, + {"event_time_microseconds", std::make_shared(6)}, + {"microseconds", std::make_shared()}, - {std::make_shared(std::make_shared()), "thread_name"}, - {std::make_shared(), "thread_id"}, + {"thread_name", std::make_shared(std::make_shared())}, + {"thread_id", std::make_shared()}, - {std::move(priority_datatype), "level"}, - {std::make_shared(), "query_id"}, - {std::make_shared(std::make_shared()), "logger_name"}, - {std::make_shared(), "message"}, + {"level", std::move(priority_datatype)}, + {"query_id", std::make_shared()}, + {"logger_name", std::make_shared(std::make_shared())}, + {"message", std::make_shared()}, - {std::make_shared(), "revision"}, + {"revision", std::make_shared()}, - {std::make_shared(std::make_shared()), "source_file"}, - {std::make_shared(), "source_line"} + {"source_file", std::make_shared(std::make_shared())}, + {"source_line", std::make_shared()} }; } diff --git a/src/Interpreters/TextLog.h b/src/Interpreters/TextLog.h index 0133d5e4eb6..d2ddd23d1e9 100644 --- a/src/Interpreters/TextLog.h +++ b/src/Interpreters/TextLog.h @@ -25,7 +25,8 @@ struct TextLogElement UInt64 source_line{}; static std::string name() { return "TextLog"; } - static Block createBlock(); + static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases() { return {}; } void appendToBlock(MutableColumns & columns) const; }; diff --git a/src/Interpreters/TraceLog.cpp b/src/Interpreters/TraceLog.cpp index fe7512f2f00..dac27aebe58 100644 --- a/src/Interpreters/TraceLog.cpp +++ b/src/Interpreters/TraceLog.cpp @@ -21,20 +21,20 @@ const TraceDataType::Values TraceLogElement::trace_values = {"MemorySample", static_cast(TraceType::MemorySample)}, }; -Block TraceLogElement::createBlock() +NamesAndTypesList TraceLogElement::getNamesAndTypes() { return { - {std::make_shared(), "event_date"}, - {std::make_shared(), "event_time"}, - {std::make_shared(6), "event_time_microseconds"}, - {std::make_shared(), "timestamp_ns"}, - {std::make_shared(), "revision"}, - {std::make_shared(trace_values), "trace_type"}, - {std::make_shared(), "thread_id"}, - {std::make_shared(), "query_id"}, - {std::make_shared(std::make_shared()), "trace"}, - {std::make_shared(), "size"}, + {"event_date", std::make_shared()}, + {"event_time", std::make_shared()}, + {"event_time_microseconds", std::make_shared(6)}, + {"timestamp_ns", std::make_shared()}, + {"revision", std::make_shared()}, + {"trace_type", std::make_shared(trace_values)}, + {"thread_id", std::make_shared()}, + {"query_id", std::make_shared()}, + {"trace", std::make_shared(std::make_shared())}, + {"size", std::make_shared()}, }; } diff --git a/src/Interpreters/TraceLog.h b/src/Interpreters/TraceLog.h index 9ee43bf32cc..85400560a7b 100644 --- a/src/Interpreters/TraceLog.h +++ b/src/Interpreters/TraceLog.h @@ -27,7 +27,8 @@ struct TraceLogElement Int64 size{}; /// Allocation size in bytes for TraceType::Memory static std::string name() { return "TraceLog"; } - static Block createBlock(); + static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases() { return {}; } void appendToBlock(MutableColumns & columns) const; }; diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 4a904c96432..179204a1a0b 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -145,6 +145,25 @@ ColumnsDescription::ColumnsDescription(NamesAndTypesList ordinary) 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 ".". /// For example "fruits.bananas" diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index 7fff22abf71..f1887d772ca 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -54,7 +55,9 @@ class ColumnsDescription { public: ColumnsDescription() = default; - explicit ColumnsDescription(NamesAndTypesList ordinary_); + explicit ColumnsDescription(NamesAndTypesList ordinary); + + explicit ColumnsDescription(NamesAndTypesList ordinary, NamesAndAliases aliases); /// `after_column` can be a Nested column name; void add(ColumnDescription column, const String & after_column = String(), bool first = false); diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 0fe9ca94972..37089ac8e3d 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -1,4 +1,6 @@ #pragma once + +#include #include #include #include @@ -29,6 +31,7 @@ class IStorageSystemOneBlock : public IStorage protected: virtual void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const = 0; + public: #if defined(ARCADIA_BUILD) IStorageSystemOneBlock(const String & name_) : IStorageSystemOneBlock(StorageID{"system", name_}) {} @@ -37,7 +40,7 @@ public: IStorageSystemOneBlock(const StorageID & table_id_) : IStorage(table_id_) { StorageInMemoryMetadata metadata_; - metadata_.setColumns(ColumnsDescription(Self::getNamesAndTypes())); + metadata_.setColumns(ColumnsDescription(Self::getNamesAndTypes(), Self::getNamesAndAliases())); setInMemoryMetadata(metadata_); } @@ -62,6 +65,8 @@ public: return Pipe(std::make_shared(sample_block, std::move(chunk))); } + + static NamesAndAliases getNamesAndAliases() { return {}; } }; } diff --git a/src/Storages/System/StorageSystemProcesses.cpp b/src/Storages/System/StorageSystemProcesses.cpp index e99cd9ddf3e..e2685af7718 100644 --- a/src/Storages/System/StorageSystemProcesses.cpp +++ b/src/Storages/System/StorageSystemProcesses.cpp @@ -68,6 +68,14 @@ NamesAndTypesList StorageSystemProcesses::getNamesAndTypes() }; } +NamesAndAliases StorageSystemProcesses::getNamesAndAliases() +{ + return + { + {"ProfileEvents.Names", {std::make_shared(std::make_shared())}, "mapKeys(ProfileEvents)"}, + {"ProfileEvents.Values", {std::make_shared(std::make_shared())}, "mapValues(ProfileEvents)"} + }; +} void StorageSystemProcesses::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const { diff --git a/src/Storages/System/StorageSystemProcesses.h b/src/Storages/System/StorageSystemProcesses.h index e63e3cc3e9d..9daf079800f 100644 --- a/src/Storages/System/StorageSystemProcesses.h +++ b/src/Storages/System/StorageSystemProcesses.h @@ -20,6 +20,8 @@ public: static NamesAndTypesList getNamesAndTypes(); + static NamesAndAliases getNamesAndAliases(); + protected: using IStorageSystemOneBlock::IStorageSystemOneBlock; diff --git a/tests/queries/0_stateless/01343_min_bytes_to_use_mmap_io.sql b/tests/queries/0_stateless/01343_min_bytes_to_use_mmap_io.sql index 1eabf27692d..cbcb5c643fe 100644 --- a/tests/queries/0_stateless/01343_min_bytes_to_use_mmap_io.sql +++ b/tests/queries/0_stateless/01343_min_bytes_to_use_mmap_io.sql @@ -6,6 +6,6 @@ SET min_bytes_to_use_mmap_io = 1; SELECT * FROM test_01343; 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; diff --git a/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql b/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql index 2c52c633c8a..3d148527270 100644 --- a/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql +++ b/tests/queries/0_stateless/01344_min_bytes_to_use_mmap_io_index.sql @@ -6,7 +6,7 @@ SET min_bytes_to_use_mmap_io = 1; SELECT * FROM test_01344 WHERE x = 'Hello, world'; SYSTEM FLUSH LOGS; -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; +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; DROP TABLE test_01344; diff --git a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql index d01e5ef5a87..5f4855c6119 100644 --- a/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql +++ b/tests/queries/0_stateless/01360_materialized_view_with_join_on_query_log.sql @@ -9,8 +9,6 @@ SELECT 1; SYSTEM FLUSH LOGS; -SET allow_experimental_map_type = 1; - -- NOTE: can be rewritten using log_queries_min_query_duration_ms CREATE MATERIALIZED VIEW slow_log Engine=Memory AS