support statistics on replicated merge tree

This commit is contained in:
Han Fei 2024-06-06 20:18:42 +02:00
parent afba7bb4ed
commit 823a7d37f3
7 changed files with 45 additions and 16 deletions

View File

@ -450,8 +450,8 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
if (!column.statistics.empty())
{
column_declaration->stat_type = column.statistics.getAST();
column_declaration->children.push_back(column_declaration->stat_type);
column_declaration->statistics_desc = column.statistics.getAST();
column_declaration->children.push_back(column_declaration->statistics_desc);
}
if (column.ttl)
@ -676,7 +676,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
}
column.statistics.column_name = column.name; /// We assign column name here for better exception error message.
if (col_decl.stat_type)
if (col_decl.statistics_desc)
{
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");

View File

@ -39,10 +39,10 @@ ASTPtr ASTColumnDeclaration::clone() const
res->children.push_back(res->codec);
}
if (stat_type)
if (statistics_desc)
{
res->stat_type = stat_type->clone();
res->children.push_back(res->stat_type);
res->statistics_desc = statistics_desc->clone();
res->children.push_back(res->statistics_desc);
}
if (ttl)
@ -111,10 +111,10 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo
codec->formatImpl(format_settings, state, frame);
}
if (stat_type)
if (statistics_desc)
{
format_settings.ostr << ' ';
stat_type->formatImpl(format_settings, state, frame);
statistics_desc->formatImpl(format_settings, state, frame);
}
if (ttl)

View File

@ -19,7 +19,7 @@ public:
bool ephemeral_default = false;
ASTPtr comment;
ASTPtr codec;
ASTPtr stat_type;
ASTPtr statistics_desc;
ASTPtr ttl;
ASTPtr collation;
ASTPtr settings;

View File

@ -193,7 +193,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
ASTPtr default_expression;
ASTPtr comment_expression;
ASTPtr codec_expression;
ASTPtr stat_type_expression;
ASTPtr statistics_desc_expression;
ASTPtr ttl_expression;
ASTPtr collation_expression;
ASTPtr settings;
@ -325,7 +325,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
if (s_stat.ignore(pos, expected))
{
if (!stat_type_parser.parse(pos, stat_type_expression, expected))
if (!stat_type_parser.parse(pos, statistics_desc_expression, expected))
return false;
}
@ -398,10 +398,10 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
column_declaration->children.push_back(std::move(settings));
}
if (stat_type_expression)
if (statistics_desc_expression)
{
column_declaration->stat_type = stat_type_expression;
column_declaration->children.push_back(std::move(stat_type_expression));
column_declaration->statistics_desc = statistics_desc_expression;
column_declaration->children.push_back(std::move(statistics_desc_expression));
}
if (ttl_expression)

View File

@ -25,6 +25,7 @@
#include <Storages/IStorage.h>
#include <Common/typeid_cast.h>
#include "Parsers/ASTSetQuery.h"
#include "Storages/StatisticsDescription.h"
#include <Core/Defines.h>
#include <Compression/CompressionFactory.h>
#include <Interpreters/ExpressionAnalyzer.h>
@ -207,6 +208,9 @@ void ColumnDescription::readText(ReadBuffer & buf)
if (col_ast->settings)
settings = col_ast->settings->as<ASTSetQuery &>().changes;
if (col_ast->statistics_desc)
statistics = ColumnStatisticsDescription::fromColumnDeclaration(*col_ast);
}
else
throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description");

View File

@ -171,7 +171,7 @@ std::vector<ColumnStatisticsDescription> ColumnStatisticsDescription::fromAST(co
ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(const ASTColumnDeclaration & column)
{
const auto & stat_type_list_ast = column.stat_type->as<ASTFunction &>().arguments;
const auto & stat_type_list_ast = column.statistics_desc->as<ASTFunction &>().arguments;
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;

View File

@ -6,9 +6,12 @@ from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance(
"node1", user_configs=["config/config.xml"], with_zookeeper=False
"node1", user_configs=["config/config.xml"], with_zookeeper=True
)
node2 = cluster.add_instance(
"node2", user_configs=["config/config.xml"], with_zookeeper=True
)
@pytest.fixture(scope="module")
def started_cluster():
@ -122,3 +125,25 @@ def test_single_node_normal(started_cluster):
"""
)
run_test_single_node(started_cluster)
def test_replicated_table_ddl(started_cluster):
node1.query("DROP TABLE IF EXISTS test_stat")
node2.query("DROP TABLE IF EXISTS test_stat")
node1.query(
"""
CREATE TABLE test_stat(a Int64 STATISTICS(tdigest, uniq), b Int64 STATISTICS(tdigest, uniq), c Int64 STATISTICS(tdigest))
ENGINE = ReplicatedMergeTree('/clickhouse/test/statistics', '1') ORDER BY a;
"""
)
node2.query(
"""
CREATE TABLE test_stat(a Int64 STATISTICS(tdigest, uniq), b Int64 STATISTICS(tdigest, uniq), c Int64 STATISTICS(tdigest))
ENGINE = ReplicatedMergeTree('/clickhouse/test/statistics', '2') ORDER BY a;
"""
)
node1.query("ALTER TABLE test_stat MODIFY STATISTICS c TYPE tdigest, uniq", settings={"alter_sync":"2"});
node1.query("ALTER TABLE test_stat DROP STATISTICS b", settings={"alter_sync":"2"});
assert node2.query("SHOW CREATE TABLE test_stat") == "CREATE TABLE default.test_stat\\n(\\n `a` Int64 STATISTICS(tdigest, uniq),\\n `b` Int64,\\n `c` Int64 STATISTICS(tdigest, uniq)\\n)\\nENGINE = ReplicatedMergeTree(\\'/clickhouse/test/statistics\\', \\'2\\')\\nORDER BY a\\nSETTINGS index_granularity = 8192\n"