Merge pull request #64934 from hanfei1991/hanfei/stats_rmt

support statistics on replicated merge tree
This commit is contained in:
Han Fei 2024-06-15 23:11:44 +00:00 committed by GitHub
commit cce1af65a3
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
8 changed files with 87 additions and 22 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,12 +676,11 @@ 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");
column.statistics = ColumnStatisticsDescription::fromColumnDeclaration(col_decl);
column.statistics.data_type = column.type;
column.statistics = ColumnStatisticsDescription::fromColumnDeclaration(col_decl, column.type);
}
if (col_decl.ttl)

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

@ -9,6 +9,7 @@
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadBuffer.h>
@ -24,7 +25,6 @@
#include <Interpreters/Context.h>
#include <Storages/IStorage.h>
#include <Common/typeid_cast.h>
#include "Parsers/ASTSetQuery.h"
#include <Core/Defines.h>
#include <Compression/CompressionFactory.h>
#include <Interpreters/ExpressionAnalyzer.h>
@ -207,6 +207,13 @@ 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, 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

@ -169,9 +169,9 @@ std::vector<ColumnStatisticsDescription> ColumnStatisticsDescription::fromAST(co
return result;
}
ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(const ASTColumnDeclaration & column)
ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type)
{
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;
@ -185,7 +185,7 @@ ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(c
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));
}
stats.data_type = data_type;
return stats;
}

View File

@ -55,7 +55,7 @@ struct ColumnStatisticsDescription
ASTPtr getAST() const;
static std::vector<ColumnStatisticsDescription> fromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns);
static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column);
static ColumnStatisticsDescription fromColumnDeclaration(const ASTColumnDeclaration & column, DataTypePtr data_type);
using StatisticsTypeDescMap = std::map<StatisticsType, SingleStatisticsDescription>;
StatisticsTypeDescMap types_to_desc;

View File

@ -6,7 +6,11 @@ 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
)
@ -122,3 +126,58 @@ 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"
)
node2.query("insert into test_stat values(1,2,3), (2,3,4)")
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "a", True)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "c", True)
node1.query(
"ALTER TABLE test_stat RENAME COLUMN c TO d", settings={"alter_sync": "2"}
)
assert node2.query("select sum(a), sum(d) from test_stat") == "3\t7\n"
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "a", True)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "c", False)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_1", "d", True)
node1.query(
"ALTER TABLE test_stat CLEAR STATISTICS d", settings={"alter_sync": "2"}
)
node1.query(
"ALTER TABLE test_stat ADD STATISTICS b type tdigest",
settings={"alter_sync": "2"},
)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "a", True)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "b", False)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_2", "d", False)
node1.query(
"ALTER TABLE test_stat MATERIALIZE STATISTICS b", settings={"alter_sync": "2"}
)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "a", True)
check_stat_file_on_disk(node2, "test_stat", "all_0_0_0_3", "b", True)