address comments

This commit is contained in:
Han Fei 2024-08-23 23:13:53 +02:00
parent 61fa4e7a47
commit 0f265ce33d
9 changed files with 26 additions and 42 deletions

View File

@ -700,7 +700,6 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
col_decl.codec, column.type, sanity_check_compression_codecs, allow_experimental_codecs, enable_deflate_qpl_codec, enable_zstd_qat_codec);
}
column.statistics.column_name = column.name; /// We assign column name here for better exception error message.
if (col_decl.statistics_desc)
{
if (!skip_checks && !context_->getSettingsRef().allow_experimental_statistics)

View File

@ -705,9 +705,9 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
}
auto stats_vec = ColumnStatisticsDescription::fromAST(statistics_decl, metadata.columns);
for (const auto & stats : stats_vec)
for (const auto & [stats_column_name, stats] : stats_vec)
{
metadata.columns.modify(stats.column_name,
metadata.columns.modify(stats_column_name,
[&](ColumnDescription & column) { column.statistics.merge(stats, column.name, column.type, if_not_exists); });
}
}
@ -739,9 +739,9 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
}
auto stats_vec = ColumnStatisticsDescription::fromAST(statistics_decl, metadata.columns);
for (const auto & stats : stats_vec)
for (const auto & [stats_column_name, stats] : stats_vec)
{
metadata.columns.modify(stats.column_name,
metadata.columns.modify(stats_column_name,
[&](ColumnDescription & column) { column.statistics.assign(stats); });
}
}
@ -866,8 +866,6 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
rename_visitor.visit(column_to_modify.default_desc.expression);
if (column_to_modify.ttl)
rename_visitor.visit(column_to_modify.ttl);
if (column_to_modify.name == column_name && !column_to_modify.statistics.empty())
column_to_modify.statistics.column_name = rename_to;
});
}
if (metadata.table_ttl.definition_ast)

View File

@ -209,11 +209,7 @@ void ColumnDescription::readText(ReadBuffer & buf)
settings = col_ast->settings->as<ASTSetQuery &>().changes;
if (col_ast->statistics_desc)
{
statistics = ColumnStatisticsDescription::fromColumnDeclaration(*col_ast, type);
/// every column has name `x` here, so we have to set the name manually.
statistics.column_name = name;
}
}
else
throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description");

View File

@ -62,7 +62,7 @@ static ColumnsStatistics getStatisticsForColumns(
const auto * desc = all_columns.tryGet(column.name);
if (desc && !desc->statistics.empty())
{
auto statistics = MergeTreeStatisticsFactory::instance().get(desc->statistics);
auto statistics = MergeTreeStatisticsFactory::instance().get(*desc);
all_statistics.push_back(std::move(statistics));
}
}

View File

@ -546,7 +546,7 @@ static std::set<ColumnStatisticsPtr> getStatisticsToRecalculate(const StorageMet
{
if (!col_desc.statistics.empty() && materialized_stats.contains(col_desc.name))
{
stats_to_recalc.insert(stats_factory.get(col_desc.statistics));
stats_to_recalc.insert(stats_factory.get(col_desc));
}
}
return stats_to_recalc;
@ -1530,7 +1530,7 @@ private:
if (ctx->materialized_statistics.contains(col.name))
{
stats_to_rewrite.push_back(MergeTreeStatisticsFactory::instance().get(col.statistics));
stats_to_rewrite.push_back(MergeTreeStatisticsFactory::instance().get(col));
}
else
{

View File

@ -58,8 +58,8 @@ IStatistics::IStatistics(const SingleStatisticsDescription & stat_)
{
}
ColumnStatistics::ColumnStatistics(const ColumnStatisticsDescription & stats_desc_)
: stats_desc(stats_desc_)
ColumnStatistics::ColumnStatistics(const ColumnStatisticsDescription & stats_desc_, const String & column_name_)
: stats_desc(stats_desc_), column_name(column_name_)
{
}
@ -176,7 +176,7 @@ String ColumnStatistics::getFileName() const
const String & ColumnStatistics::columnName() const
{
return stats_desc.column_name;
return column_name;
}
UInt64 ColumnStatistics::rowCount() const
@ -227,15 +227,15 @@ void MergeTreeStatisticsFactory::validate(const ColumnStatisticsDescription & st
}
}
ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnStatisticsDescription & stats) const
ColumnStatisticsPtr MergeTreeStatisticsFactory::get(const ColumnDescription & column_desc) const
{
ColumnStatisticsPtr column_stat = std::make_shared<ColumnStatistics>(stats);
for (const auto & [type, desc] : stats.types_to_desc)
ColumnStatisticsPtr column_stat = std::make_shared<ColumnStatistics>(column_desc.statistics, column_desc.name);
for (const auto & [type, desc] : column_desc.statistics.types_to_desc)
{
auto it = creators.find(type);
if (it == creators.end())
throw Exception(ErrorCodes::INCORRECT_QUERY, "Unknown statistic type '{}'. Available types: 'tdigest' 'uniq' and 'count_min'", type);
auto stat_ptr = (it->second)(desc, stats.data_type);
auto stat_ptr = (it->second)(desc, column_desc.type);
column_stat->stats[type] = stat_ptr;
}
return column_stat;
@ -246,7 +246,7 @@ ColumnsStatistics MergeTreeStatisticsFactory::getMany(const ColumnsDescription &
ColumnsStatistics result;
for (const auto & col : columns)
if (!col.statistics.empty())
result.push_back(get(col.statistics));
result.push_back(get(col));
return result;
}

View File

@ -54,7 +54,7 @@ using StatisticsPtr = std::shared_ptr<IStatistics>;
class ColumnStatistics
{
public:
explicit ColumnStatistics(const ColumnStatisticsDescription & stats_desc_);
explicit ColumnStatistics(const ColumnStatisticsDescription & stats_desc_, const String & column_name_);
void serialize(WriteBuffer & buf);
void deserialize(ReadBuffer & buf);
@ -73,10 +73,12 @@ public:
private:
friend class MergeTreeStatisticsFactory;
ColumnStatisticsDescription stats_desc;
String column_name;
std::map<StatisticsType, StatisticsPtr> stats;
UInt64 rows = 0; /// the number of rows in the column
};
struct ColumnDescription;
class ColumnsDescription;
using ColumnStatisticsPtr = std::shared_ptr<ColumnStatistics>;
using ColumnsStatistics = std::vector<ColumnStatisticsPtr>;
@ -91,7 +93,7 @@ public:
using Validator = std::function<void(const SingleStatisticsDescription & stats, const DataTypePtr & data_type)>;
using Creator = std::function<StatisticsPtr(const SingleStatisticsDescription & stats, const DataTypePtr & data_type)>;
ColumnStatisticsPtr get(const ColumnStatisticsDescription & stats) const;
ColumnStatisticsPtr get(const ColumnDescription & column_desc) const;
ColumnsStatistics getMany(const ColumnsDescription & columns) const;
void registerValidator(StatisticsType type, Validator validator);

View File

@ -96,16 +96,13 @@ void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & othe
{
chassert(merging_column_type);
if (column_name.empty())
column_name = merging_column_name;
data_type = merging_column_type;
for (const auto & [stats_type, stats_desc]: other.types_to_desc)
{
if (!if_not_exists && types_to_desc.contains(stats_type))
{
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics type name {} has existed in column {}", stats_type, column_name);
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Statistics type name {} has existed in column {}", stats_type, merging_column_name);
}
else if (!types_to_desc.contains(stats_type))
types_to_desc.emplace(stats_type, stats_desc);
@ -114,12 +111,6 @@ void ColumnStatisticsDescription::merge(const ColumnStatisticsDescription & othe
void ColumnStatisticsDescription::assign(const ColumnStatisticsDescription & other)
{
/// If the statistics is empty, it's possible that we have not assign a column_name.
if (empty() && column_name == "")
column_name = other.column_name;
if (other.column_name != column_name)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot assign statistics from column {} to {}", other.column_name, column_name);
types_to_desc = other.types_to_desc;
data_type = other.data_type;
}
@ -129,7 +120,7 @@ void ColumnStatisticsDescription::clear()
types_to_desc.clear();
}
std::vector<ColumnStatisticsDescription> ColumnStatisticsDescription::fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns)
std::vector<std::pair<String, ColumnStatisticsDescription>> ColumnStatisticsDescription::fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns)
{
const auto * stat_definition_ast = definition_ast->as<ASTStatisticsDeclaration>();
if (!stat_definition_ast)
@ -147,7 +138,7 @@ std::vector<ColumnStatisticsDescription> ColumnStatisticsDescription::fromAST(co
statistics_types.emplace(stat.type, stat);
}
std::vector<ColumnStatisticsDescription> result;
std::vector<std::pair<String, ColumnStatisticsDescription>> result;
result.reserve(stat_definition_ast->columns->children.size());
for (const auto & column_ast : stat_definition_ast->columns->children)
@ -159,10 +150,9 @@ std::vector<ColumnStatisticsDescription> ColumnStatisticsDescription::fromAST(co
throw Exception(ErrorCodes::INCORRECT_QUERY, "Incorrect column name {}", physical_column_name);
const auto & column = columns.getPhysical(physical_column_name);
stats.column_name = column.name;
stats.data_type = column.type;
stats.types_to_desc = statistics_types;
result.push_back(stats);
result.emplace_back(physical_column_name, stats);
}
if (result.empty())
@ -177,14 +167,13 @@ ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(c
if (stat_type_list_ast->children.empty())
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<const ASTFunction &>().name;
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);
throw Exception(ErrorCodes::INCORRECT_QUERY, "Column {} already contains statistics type {}", column.name, stat_type);
stats.types_to_desc.emplace(stat.type, std::move(stat));
}
stats.data_type = data_type;

View File

@ -55,12 +55,12 @@ struct ColumnStatisticsDescription
ASTPtr getAST() const;
static std::vector<ColumnStatisticsDescription> fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns);
/// get a vector of <column name, statistics desc> pair
static std::vector<std::pair<String, ColumnStatisticsDescription>> fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns);
static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type);
using StatisticsTypeDescMap = std::map<StatisticsType, SingleStatisticsDescription>;
StatisticsTypeDescMap types_to_desc;
String column_name;
DataTypePtr data_type;
};