From 823a7d37f3bab8d4aa9cb8b2ff8b7b087e8a3037 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 6 Jun 2024 20:18:42 +0200 Subject: [PATCH 1/6] support statistics on replicated merge tree --- src/Interpreters/InterpreterCreateQuery.cpp | 6 ++--- src/Parsers/ASTColumnDeclaration.cpp | 10 +++---- src/Parsers/ASTColumnDeclaration.h | 2 +- src/Parsers/ParserCreateQuery.h | 10 +++---- src/Storages/ColumnsDescription.cpp | 4 +++ src/Storages/StatisticsDescription.cpp | 2 +- .../test_manipulate_statistics/test.py | 27 ++++++++++++++++++- 7 files changed, 45 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 66936dc25d7..a51d3e6dade 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -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"); diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 6c29e0bf9d5..4a8a3d2967d 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -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) diff --git a/src/Parsers/ASTColumnDeclaration.h b/src/Parsers/ASTColumnDeclaration.h index d775928d05c..914916d5074 100644 --- a/src/Parsers/ASTColumnDeclaration.h +++ b/src/Parsers/ASTColumnDeclaration.h @@ -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; diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 27bb524970d..5f6df33176f 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -193,7 +193,7 @@ bool IParserColumnDeclaration::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::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::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) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 69e39323219..d1babb817bf 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -25,6 +25,7 @@ #include #include #include "Parsers/ASTSetQuery.h" +#include "Storages/StatisticsDescription.h" #include #include #include @@ -207,6 +208,9 @@ void ColumnDescription::readText(ReadBuffer & buf) if (col_ast->settings) settings = col_ast->settings->as().changes; + + if (col_ast->statistics_desc) + statistics = ColumnStatisticsDescription::fromColumnDeclaration(*col_ast); } else throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description"); diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index dff1b7d3602..fc06c66b50e 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -171,7 +171,7 @@ std::vector ColumnStatisticsDescription::fromAST(co ColumnStatisticsDescription ColumnStatisticsDescription::fromColumnDeclaration(const ASTColumnDeclaration & column) { - const auto & stat_type_list_ast = column.stat_type->as().arguments; + const auto & stat_type_list_ast = column.statistics_desc->as().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; diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index 2b26af940d1..bffee89ffc6 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -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" From 0bd5164f83b1751999c667de4ec6327f9a59c5d6 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Thu, 6 Jun 2024 21:30:16 +0200 Subject: [PATCH 2/6] fix style --- src/Storages/ColumnsDescription.cpp | 3 +-- .../integration/test_manipulate_statistics/test.py | 14 +++++++++++--- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index d1babb817bf..556f8a6e42d 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -24,8 +25,6 @@ #include #include #include -#include "Parsers/ASTSetQuery.h" -#include "Storages/StatisticsDescription.h" #include #include #include diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index bffee89ffc6..53ab9682ad2 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -13,6 +13,7 @@ node2 = cluster.add_instance( "node2", user_configs=["config/config.xml"], with_zookeeper=True ) + @pytest.fixture(scope="module") def started_cluster(): try: @@ -126,6 +127,7 @@ 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") @@ -143,7 +145,13 @@ def test_replicated_table_ddl(started_cluster): """ ) - 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"}); + 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" + 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" + ) From e0668ab8fa1fd90fc88c6d1088230279b6fb28b7 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Tue, 11 Jun 2024 17:03:54 +0200 Subject: [PATCH 3/6] write more tests --- src/Interpreters/InterpreterCreateQuery.cpp | 3 +-- src/Storages/ColumnsDescription.cpp | 6 +++++- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 6 ++++++ src/Storages/StatisticsDescription.cpp | 4 ++-- src/Storages/StatisticsDescription.h | 2 +- .../test_manipulate_statistics/test.py | 18 ++++++++++++++++++ 6 files changed, 33 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a51d3e6dade..b2d65662eaa 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -680,8 +680,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( { 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) diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 556f8a6e42d..c07583cd39d 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -209,7 +209,11 @@ void ColumnDescription::readText(ReadBuffer & buf) settings = col_ast->settings->as().changes; if (col_ast->statistics_desc) - statistics = ColumnStatisticsDescription::fromColumnDeclaration(*col_ast); + { + 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"); diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 8e304936747..0dbddf05697 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -471,7 +471,13 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( ColumnsStatistics statistics; if (context->getSettingsRef().materialize_statistics_on_insert) + { + for (auto col : metadata_snapshot->getColumns()) + LOG_INFO(log, "column col {} stats {}", col.name, col.statistics.column_name); statistics = MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns()); + for (auto stats : statistics) + LOG_INFO(log, "writing stats {}", stats->columnName()); + } /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) diff --git a/src/Storages/StatisticsDescription.cpp b/src/Storages/StatisticsDescription.cpp index fc06c66b50e..f10fb78f933 100644 --- a/src/Storages/StatisticsDescription.cpp +++ b/src/Storages/StatisticsDescription.cpp @@ -169,7 +169,7 @@ std::vector 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.statistics_desc->as().arguments; if (stat_type_list_ast->children.empty()) @@ -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; } diff --git a/src/Storages/StatisticsDescription.h b/src/Storages/StatisticsDescription.h index 59ad8944850..4862fb79d45 100644 --- a/src/Storages/StatisticsDescription.h +++ b/src/Storages/StatisticsDescription.h @@ -55,7 +55,7 @@ struct ColumnStatisticsDescription ASTPtr getAST() const; static std::vector 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; StatisticsTypeDescMap types_to_desc; diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index 53ab9682ad2..0ce90731e8d 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -155,3 +155,21 @@ def test_replicated_table_ddl(started_cluster): 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", "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) From af4541755a43fcacf1a1a58edcf6b07bd5fcedec Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 12 Jun 2024 14:27:13 +0200 Subject: [PATCH 4/6] fix black --- .../test_manipulate_statistics/test.py | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index 0ce90731e8d..9485b611c01 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -156,20 +156,29 @@ def test_replicated_table_ddl(started_cluster): == "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)"); + 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", "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") + 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"}) + 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"}) + 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) From d798fffb1d772266095f599173212874a3897ab1 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Wed, 12 Jun 2024 16:46:14 +0200 Subject: [PATCH 5/6] fix --- src/Storages/MergeTree/MergeTreeDataWriter.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 0dbddf05697..8e304936747 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -471,13 +471,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( ColumnsStatistics statistics; if (context->getSettingsRef().materialize_statistics_on_insert) - { - for (auto col : metadata_snapshot->getColumns()) - LOG_INFO(log, "column col {} stats {}", col.name, col.statistics.column_name); statistics = MergeTreeStatisticsFactory::instance().getMany(metadata_snapshot->getColumns()); - for (auto stats : statistics) - LOG_INFO(log, "writing stats {}", stats->columnName()); - } /// If we need to calculate some columns to sort. if (metadata_snapshot->hasSortingKey() || metadata_snapshot->hasSecondaryIndices()) From aacb85495b826d26c84a17b28c3a479124aaf319 Mon Sep 17 00:00:00 2001 From: Han Fei Date: Fri, 14 Jun 2024 13:58:04 +0200 Subject: [PATCH 6/6] Update tests/integration/test_manipulate_statistics/test.py --- tests/integration/test_manipulate_statistics/test.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/test_manipulate_statistics/test.py b/tests/integration/test_manipulate_statistics/test.py index 9485b611c01..a602cce63df 100644 --- a/tests/integration/test_manipulate_statistics/test.py +++ b/tests/integration/test_manipulate_statistics/test.py @@ -157,7 +157,6 @@ def test_replicated_table_ddl(started_cluster): ) 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", "a", True) check_stat_file_on_disk(node2, "test_stat", "all_0_0_0", "c", True) node1.query(