From bb29c3b7b4a1631653e8068221887313e348429b Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 14 May 2024 18:16:01 +0200 Subject: [PATCH] address part of comments --- src/Core/Settings.h | 4 +- src/Interpreters/InterpreterAlterQuery.cpp | 4 +- src/Interpreters/InterpreterCreateQuery.cpp | 4 +- src/Interpreters/InterpreterExplainQuery.cpp | 4 +- src/Parsers/ExpressionElementParsers.h | 4 +- src/Storages/AlterCommands.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 6 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 6 +- .../MergeTree/MergeTreeWhereOptimizer.h | 2 +- src/Storages/Statistics/Statistics.cpp | 12 +-- src/Storages/Statistics/Statistics.h | 10 +-- src/Storages/Statistics/TDigestStatistics.cpp | 7 +- src/Storages/Statistics/TDigestStatistics.h | 4 +- src/Storages/Statistics/UniqStatistics.cpp | 23 ++--- src/Storages/Statistics/UniqStatistics.h | 12 +-- src/Storages/StatisticsDescription.cpp | 84 +++++++++---------- src/Storages/StatisticsDescription.h | 5 +- 18 files changed, 97 insertions(+), 98 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 375bdb1c516..e270f6642a2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -151,8 +151,8 @@ class IColumn; M(Bool, enable_multiple_prewhere_read_steps, true, "Move more conditions from WHERE to PREWHERE and do reads from disk and filtering in multiple steps if there are multiple conditions combined with AND", 0) \ M(Bool, move_primary_key_columns_to_end_of_prewhere, true, "Move PREWHERE conditions containing primary key columns to the end of AND chain. It is likely that these conditions are taken into account during primary key analysis and thus will not contribute a lot to PREWHERE filtering.", 0) \ \ - M(Bool, allow_statistic_optimize, false, "Allows using statistic to optimize queries", 0) \ - M(Bool, allow_experimental_statistic, false, "Allows using statistic", 0) \ + M(Bool, allow_statistics_optimize, false, "Allows using statistics to optimize queries", 0) \ + M(Bool, allow_experimental_statistics, false, "Allows using statistics", 0) \ \ M(UInt64, alter_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) ALIAS(replication_alter_partitions_sync) \ M(Int64, replication_wait_for_inactive_replica_timeout, 120, "Wait for inactive replica to execute ALTER/OPTIMIZE. Time in seconds, 0 - do not wait, negative - wait for unlimited time.", 0) \ diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 1e0706f728d..d2017bc3766 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -175,11 +175,11 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) else throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong parameter type in ALTER query"); - if (!getContext()->getSettings().allow_experimental_statistic && ( + if (!getContext()->getSettings().allow_experimental_statistics && ( command_ast->type == ASTAlterCommand::ADD_STATISTICS || command_ast->type == ASTAlterCommand::DROP_STATISTICS || command_ast->type == ASTAlterCommand::MATERIALIZE_STATISTICS)) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Alter table with statistic is now disabled. Turn on allow_experimental_statistic"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Alter table with statistics is now disabled. Turn on allow_experimental_statistics"); } if (typeid_cast(database.get())) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 28441843ab1..475490ec35f 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -686,8 +686,8 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( column.stats.column_name = column.name; /// We assign column name here for better exception error message. if (col_decl.stat_type) { - if (!skip_checks && !context_->getSettingsRef().allow_experimental_statistic) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Create table with statistic is now disabled. Turn on allow_experimental_statistic"); + if (!skip_checks && !context_->getSettingsRef().allow_experimental_statistics) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Create table with statistics is now disabled. Turn on allow_experimental_statistics"); column.stats = ColumnStatisticsDescription::getStatisticFromColumnDeclaration(col_decl); column.stats.data_type = column.type; } diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 458be843b59..3a06e1b2301 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -67,8 +67,8 @@ namespace static void visit(ASTSelectQuery & select, ASTPtr & node, Data & data) { - /// we need to read statistic when `allow_statistic_optimize` is enabled. - bool only_analyze = !data.getContext()->getSettings().allow_statistic_optimize; + /// we need to read statistic when `allow_statistics_optimize` is enabled. + bool only_analyze = !data.getContext()->getSettings().allow_statistics_optimize; InterpreterSelectQuery interpreter( node, data.getContext(), SelectQueryOptions(QueryProcessingStage::FetchColumns).analyze(only_analyze).modify()); diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index becbd724a25..a28f40a00e3 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -201,11 +201,11 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; -/// STATISTIC(tdigest(200)) +/// STATISTICS(tdigest(200)) class ParserStatisticsType : public IParserBase { protected: - const char * getName() const override { return "statistic"; } + const char * getName() const override { return "statistics"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index e768a3f362a..bf00fae933b 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -705,7 +705,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) for (const auto & stats : stats_vec) { metadata.columns.modify(stats.column_name, - [&](ColumnDescription & column) { column.stats.merge(stats, column, if_not_exists); }); + [&](ColumnDescription & column) { column.stats.merge(stats, column.name, column.type, if_not_exists); }); } } else if (type == DROP_STATISTICS) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index ce2b8f9efd7..ae9d32fb5a2 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -666,7 +666,7 @@ ColumnsStatistics IMergeTreeDataPart::loadStatistics() const ColumnsStatistics result; for (auto & stat : total_statistics) { - String file_name = stat->getFileName() + STAT_FILE_SUFFIX; + String file_name = stat->getFileName() + STATS_FILE_SUFFIX; String file_path = fs::path(getDataPartStorage().getRelativePath()) / file_name; if (!metadata_manager->exists(file_name)) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 91f16d69a3d..12b361392e0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -471,7 +471,7 @@ StoragePolicyPtr MergeTreeData::getStoragePolicy() const ConditionSelectivityEstimator MergeTreeData::getConditionEstimatorByPredicate(const SelectQueryInfo & query_info, const StorageSnapshotPtr & storage_snapshot, ContextPtr local_context) const { - if (!local_context->getSettings().allow_statistic_optimize) + if (!local_context->getSettings().allow_statistics_optimize) return {}; const auto & parts = assert_cast(*storage_snapshot->data).parts; @@ -3242,8 +3242,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context const auto & new_column = new_metadata.getColumns().get(command.column_name); if (!old_column.type->equals(*new_column.type)) throw Exception(ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN, - "ALTER types of column {} with statistic is not not safe " - "because it can change the representation of statistic", + "ALTER types of column {} with statistics is not not safe " + "because it can change the representation of statistics", backQuoteIfNeed(command.column_name)); } } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 3309a5fcb92..ab2ed7725d8 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -92,7 +92,7 @@ void MergeTreeWhereOptimizer::optimize(SelectQueryInfo & select_query_info, cons where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere; where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere; where_optimizer_context.is_final = select.final(); - where_optimizer_context.use_statistic = context->getSettingsRef().allow_statistic_optimize; + where_optimizer_context.use_statistics = context->getSettingsRef().allow_statistics_optimize; RPNBuilderTreeContext tree_context(context, std::move(block_with_constants), {} /*prepared_sets*/); RPNBuilderTreeNode node(select.where().get(), tree_context); @@ -123,7 +123,7 @@ MergeTreeWhereOptimizer::FilterActionsOptimizeResult MergeTreeWhereOptimizer::op where_optimizer_context.move_all_conditions_to_prewhere = context->getSettingsRef().move_all_conditions_to_prewhere; where_optimizer_context.move_primary_key_columns_to_end_of_prewhere = context->getSettingsRef().move_primary_key_columns_to_end_of_prewhere; where_optimizer_context.is_final = is_final; - where_optimizer_context.use_statistic = context->getSettingsRef().allow_statistic_optimize; + where_optimizer_context.use_statistics = context->getSettingsRef().allow_statistics_optimize; RPNBuilderTreeContext tree_context(context); RPNBuilderTreeNode node(&filter_dag->findInOutputs(filter_column_name), tree_context); @@ -276,7 +276,7 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const RPNBuilderTree if (cond.viable) cond.good = isConditionGood(node, table_columns); - if (where_optimizer_context.use_statistic) + if (where_optimizer_context.use_statistics) { cond.good = cond.viable; diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h index 813f4a78ea4..92a692ab148 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.h +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.h @@ -104,7 +104,7 @@ private: bool move_all_conditions_to_prewhere = false; bool move_primary_key_columns_to_end_of_prewhere = false; bool is_final = false; - bool use_statistic = false; + bool use_statistics = false; }; struct OptimizeResult diff --git a/src/Storages/Statistics/Statistics.cpp b/src/Storages/Statistics/Statistics.cpp index 933de06fa97..0f63a286f75 100644 --- a/src/Storages/Statistics/Statistics.cpp +++ b/src/Storages/Statistics/Statistics.cpp @@ -127,16 +127,16 @@ UInt64 ColumnStatistics::count() const return rows; } -void MergeTreeStatisticsFactory::registerCreator(StatisticsType stat_type, Creator creator) +void MergeTreeStatisticsFactory::registerCreator(StatisticsType stats_type, Creator creator) { - if (!creators.emplace(stat_type, std::move(creator)).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticsFactory: the statistic creator type {} is not unique", stat_type); + if (!creators.emplace(stats_type, std::move(creator)).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticsFactory: the statistics creator type {} is not unique", stats_type); } -void MergeTreeStatisticsFactory::registerValidator(StatisticsType stat_type, Validator validator) +void MergeTreeStatisticsFactory::registerValidator(StatisticsType stats_type, Validator validator) { - if (!validators.emplace(stat_type, std::move(validator)).second) - throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticsFactory: the statistic validator type {} is not unique", stat_type); + if (!validators.emplace(stats_type, std::move(validator)).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeTreeStatisticsFactory: the statistics validator type {} is not unique", stats_type); } diff --git a/src/Storages/Statistics/Statistics.h b/src/Storages/Statistics/Statistics.h index 1c111ba3a93..1415f0a5d2f 100644 --- a/src/Storages/Statistics/Statistics.h +++ b/src/Storages/Statistics/Statistics.h @@ -14,8 +14,8 @@ /// this is for user-defined statistic. -constexpr auto STAT_FILE_PREFIX = "statistic_"; -constexpr auto STAT_FILE_SUFFIX = ".stat"; +constexpr auto STATS_FILE_PREFIX = "statistics_"; +constexpr auto STATS_FILE_SUFFIX = ".stats"; namespace DB { @@ -88,11 +88,11 @@ public: void validate(const ColumnStatisticsDescription & stats, DataTypePtr data_type) const; - using Creator = std::function; + using Creator = std::function; - using Validator = std::function; + using Validator = std::function; - ColumnStatisticsPtr get(const ColumnStatisticsDescription & stat) const; + ColumnStatisticsPtr get(const ColumnStatisticsDescription & stats) const; ColumnsStatistics getMany(const ColumnsDescription & columns) const; diff --git a/src/Storages/Statistics/TDigestStatistics.cpp b/src/Storages/Statistics/TDigestStatistics.cpp index 0cb0282f015..aa5662c979d 100644 --- a/src/Storages/Statistics/TDigestStatistics.cpp +++ b/src/Storages/Statistics/TDigestStatistics.cpp @@ -8,6 +8,11 @@ namespace ErrorCodes extern const int ILLEGAL_STATISTICS; } +TDigestStatistics::TDigestStatistics(const SingleStatisticsDescription & stat_): + IStatistics(stat_) +{ +} + Float64 TDigestStatistics::estimateLess(Float64 val) const { return data.getCountLessThan(val); @@ -49,7 +54,7 @@ void TDigestValidator(const SingleStatisticsDescription &, DataTypePtr data_type { data_type = removeNullable(data_type); if (!data_type->isValueRepresentedByNumber()) - throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "TDigest does not support type {}", data_type->getName()); + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'tdigest' does not support type {}", data_type->getName()); } } diff --git a/src/Storages/Statistics/TDigestStatistics.h b/src/Storages/Statistics/TDigestStatistics.h index bcf4b15fd60..7c361b8751f 100644 --- a/src/Storages/Statistics/TDigestStatistics.h +++ b/src/Storages/Statistics/TDigestStatistics.h @@ -11,9 +11,7 @@ namespace DB class TDigestStatistics : public IStatistics { public: - explicit TDigestStatistics(const SingleStatisticsDescription & stat_) : IStatistics(stat_) - { - } + explicit TDigestStatistics(const SingleStatisticsDescription & stat_); Float64 estimateLess(Float64 val) const; diff --git a/src/Storages/Statistics/UniqStatistics.cpp b/src/Storages/Statistics/UniqStatistics.cpp index 3d0645a9553..7f99a91cf86 100644 --- a/src/Storages/Statistics/UniqStatistics.cpp +++ b/src/Storages/Statistics/UniqStatistics.cpp @@ -1,4 +1,5 @@ #include +#include #include namespace DB @@ -13,46 +14,46 @@ UniqStatistics::UniqStatistics(const SingleStatisticsDescription & stat_, const : IStatistics(stat_) { arena = std::make_unique(); - AggregateFunctionProperties property; - property.returns_default_when_only_null = true; - uniq_collector = AggregateFunctionFactory::instance().get("uniq", NullsAction::IGNORE_NULLS, {data_type}, Array(), property); - data = arena->alignedAlloc(uniq_collector->sizeOfData(), uniq_collector->alignOfData()); - uniq_collector->create(data); + AggregateFunctionProperties properties; + properties.returns_default_when_only_null = true; + collector = AggregateFunctionFactory::instance().get("uniq", NullsAction::IGNORE_NULLS, {data_type}, Array(), properties); + data = arena->alignedAlloc(collector->sizeOfData(), collector->alignOfData()); + collector->create(data); } UniqStatistics::~UniqStatistics() { - uniq_collector->destroy(data); + collector->destroy(data); } UInt64 UniqStatistics::getCardinality() { auto column = DataTypeUInt64().createColumn(); - uniq_collector->insertResultInto(data, *column, nullptr); + collector->insertResultInto(data, *column, nullptr); return column->getUInt(0); } void UniqStatistics::serialize(WriteBuffer & buf) { - uniq_collector->serialize(data, buf); + collector->serialize(data, buf); } void UniqStatistics::deserialize(ReadBuffer & buf) { - uniq_collector->deserialize(data, buf); + collector->deserialize(data, buf); } void UniqStatistics::update(const ColumnPtr & column) { const IColumn * col_ptr = column.get(); - uniq_collector->addBatchSinglePlace(0, column->size(), data, &col_ptr, nullptr); + collector->addBatchSinglePlace(0, column->size(), data, &col_ptr, nullptr); } void UniqValidator(const SingleStatisticsDescription &, DataTypePtr data_type) { data_type = removeNullable(data_type); if (!data_type->isValueRepresentedByNumber()) - throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type Uniq does not support type {}", data_type->getName()); + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics of type 'uniq' does not support type {}", data_type->getName()); } StatisticsPtr UniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type) diff --git a/src/Storages/Statistics/UniqStatistics.h b/src/Storages/Statistics/UniqStatistics.h index 75a893c080c..0d86a6e458a 100644 --- a/src/Storages/Statistics/UniqStatistics.h +++ b/src/Storages/Statistics/UniqStatistics.h @@ -2,7 +2,6 @@ #include #include -#include #include namespace DB @@ -10,10 +9,6 @@ namespace DB class UniqStatistics : public IStatistics { - std::unique_ptr arena; - AggregateFunctionPtr uniq_collector; - AggregateDataPtr data; - public: UniqStatistics(const SingleStatisticsDescription & stat_, const DataTypePtr & data_type); @@ -26,6 +21,13 @@ public: void deserialize(ReadBuffer & buf) override; void update(const ColumnPtr & column) override; + +private: + + std::unique_ptr arena; + AggregateFunctionPtr collector; + AggregateDataPtr data; + }; StatisticsPtr UniqCreator(const SingleStatisticsDescription & stat, DataTypePtr data_type); diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index 29761fd1ded..3de7b8159b7 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -25,13 +25,13 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; }; -static StatisticsType stringToStatisticType(String type) +static StatisticsType stringToStatisticsType(String type) { if (type == "tdigest") return StatisticsType::TDigest; if (type == "uniq") return StatisticsType::Uniq; - throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type: {}. Supported statistic types are `tdigest` and `uniq`.", type); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistics type: {}. Supported statistics types are `tdigest` and `uniq`.", type); } String SingleStatisticsDescription::getTypeName() const @@ -43,7 +43,7 @@ String SingleStatisticsDescription::getTypeName() const case StatisticsType::Uniq: return "Uniq"; default: - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistic type: {}. Supported statistic types are `tdigest` and `uniq`.", type); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown statistics type: {}. Supported statistics types are `tdigest` and `uniq`.", type); } } @@ -61,12 +61,12 @@ bool ColumnStatisticsDescription::operator==(const ColumnStatisticsDescription & if (types_to_desc.size() != other.types_to_desc.size()) return false; - for (const auto & s : types_to_desc) + for (const auto & [type, desc] : types_to_desc) { - StatisticsType stats_type = s.first; + StatisticsType stats_type = type; if (!other.types_to_desc.contains(stats_type)) return false; - if (!(s.second == other.types_to_desc.at(stats_type))) + if (!(desc == other.types_to_desc.at(stats_type))) return false; } @@ -80,25 +80,27 @@ bool ColumnStatisticsDescription::empty() const bool ColumnStatisticsDescription::contains(const String & stat_type) const { - return types_to_desc.contains(stringToStatisticType(stat_type)); + return types_to_desc.contains(stringToStatisticsType(stat_type)); } -void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & other, const ColumnDescription & column, bool if_not_exists) +void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & other, const String & merging_column_name, DataTypePtr merging_column_type, bool if_not_exists) { + chassert(merging_column_type); + if (column_name.empty()) { - column_name = column.name; - data_type = column.type; + column_name = merging_column_name; + data_type = merging_column_type; } - for (const auto & iter: other.types_to_desc) + for (const auto & [stats_type, stats_desc]: other.types_to_desc) { - if (!if_not_exists && types_to_desc.contains(iter.first)) + if (!if_not_exists && types_to_desc.contains(stats_type)) { - throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistic type name {} has existed in column {}", iter.first, column_name); + throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics type name {} has existed in column {}", stats_type, column_name); } - else if (!types_to_desc.contains(iter.first)) - types_to_desc.emplace(iter.first, iter.second); + else if (!types_to_desc.contains(stats_type)) + types_to_desc.emplace(stats_type, stats_desc); } } @@ -119,40 +121,39 @@ std::vector ColumnStatisticsDescription::getStatist { const auto * stat_definition_ast = definition_ast->as(); if (!stat_definition_ast) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot AST to ASTStatisticDeclaration"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot cast AST to ASTStatisticDeclaration"); + + StatisticsTypeDescMap statistics_types; + for (const auto & stat_ast : stat_definition_ast->types->children) + { + String stat_type_name = stat_ast->as().name; + auto stat_type = stringToStatisticsType(Poco::toLower(stat_type_name)); + if (statistics_types.contains(stat_type)) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Statistics type {} was specified more than once", stat_type_name); + SingleStatisticsDescription stat(stat_type, stat_ast->clone()); + + statistics_types.emplace(stat.type, stat); + } std::vector result; result.reserve(stat_definition_ast->columns->children.size()); - StatisticsTypeDescMap statistic_types; - for (const auto & stat_ast : stat_definition_ast->types->children) - { - String stat_type_name = stat_ast->as().name; - auto stat_type = stringToStatisticType(Poco::toLower(stat_type_name)); - if (statistic_types.contains(stat_type)) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Statistic type {} was specified more than once", stat_type_name); - SingleStatisticsDescription stat(stat_type, stat_ast->clone()); - - statistic_types.emplace(stat.type, stat); - } - for (const auto & column_ast : stat_definition_ast->columns->children) { - - ColumnStatisticsDescription types_to_desc_desc; + ColumnStatisticsDescription stats; String physical_column_name = column_ast->as().name(); if (!columns.hasPhysical(physical_column_name)) throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column name {}", physical_column_name); const auto & column = columns.getPhysical(physical_column_name); - types_to_desc_desc.column_name = column.name; - types_to_desc_desc.types_to_desc = statistic_types; - result.push_back(types_to_desc_desc); + stats.column_name = column.name; + stats.types_to_desc = statistics_types; + result.push_back(stats); } if (result.empty()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Empty statistic column list is not allowed."); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Empty statistics column list is not allowed."); return result; } @@ -161,27 +162,22 @@ ColumnStatisticsDescription ColumnStatisticsDescription::getStatisticFromColumnD { const auto & stat_type_list_ast = column.stat_type->as().arguments; if (stat_type_list_ast->children.empty()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "We expect at least one statistic type for column {}", queryToString(column)); + throw Exception(ErrorCodes::INCORRECT_QUERY, "We expect at least one statistics type for column {}", queryToString(column)); ColumnStatisticsDescription stats; stats.column_name = column.name; for (const auto & ast : stat_type_list_ast->children) { const auto & stat_type = ast->as().name; - SingleStatisticsDescription stat(stringToStatisticType(Poco::toLower(stat_type)), ast->clone()); - stats.add(stat.type, stat); + SingleStatisticsDescription stat(stringToStatisticsType(Poco::toLower(stat_type)), ast->clone()); + if (stats.types_to_desc.contains(stat.type)) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Column {} already contains statistics type {}", stats.column_name, stat_type); + stats.types_to_desc.emplace(stat.type, std::move(stat)); } return stats; } -void ColumnStatisticsDescription::add(StatisticsType stat_type, const SingleStatisticsDescription & desc) -{ - if (types_to_desc.contains(stat_type)) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Column {} already contains statistic type {}", column_name, stat_type); - types_to_desc.emplace(stat_type, desc); -} - ASTPtr ColumnStatisticsDescription::getAST() const { auto function_node = std::make_shared(); diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index da362b9b47d..b064644c020 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -31,7 +31,6 @@ struct SingleStatisticsDescription bool operator==(const SingleStatisticsDescription & other) const; }; -struct ColumnDescription; class ColumnsDescription; struct ColumnStatisticsDescription @@ -42,14 +41,12 @@ struct ColumnStatisticsDescription bool contains(const String & stat_type) const; - void merge(const ColumnStatisticsDescription & other, const ColumnDescription & column, bool if_not_exists); + void merge(const ColumnStatisticsDescription & other, const String & column_name, DataTypePtr column_type, bool if_not_exists); void assign(const ColumnStatisticsDescription & other); void clear(); - void add(StatisticsType stat_type, const SingleStatisticsDescription & desc); - ASTPtr getAST() const; static std::vector getStatisticsDescriptionsFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns);