fix logical err of modify statistics

This commit is contained in:
Han Fei 2024-08-23 16:38:48 +02:00
parent 008d02880b
commit 61fa4e7a47
3 changed files with 17 additions and 5 deletions

View File

@ -734,7 +734,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
{
if (!metadata.columns.has(statistics_column_name))
{
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Cannot add statistics for column {}: this column is not found", statistics_column_name);
throw Exception(ErrorCodes::ILLEGAL_STATISTICS, "Cannot modify statistics for column {}: this column is not found", statistics_column_name);
}
}

View File

@ -6,7 +6,6 @@
#include <Parsers/ASTStatisticsDeclaration.h>
#include <Parsers/queryToString.h>
#include <Parsers/ParserCreateQuery.h>
#include <Poco/Logger.h>
#include <Storages/ColumnsDescription.h>
@ -115,8 +114,11 @@ 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 {}", column_name, other.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;

View File

@ -6,11 +6,13 @@ from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance(
"node1", user_configs=["config/config.xml"], with_zookeeper=True
"node1", user_configs=["config/config.xml"], with_zookeeper=True,
macros={"replica": "a", "shard": "shard1"}
)
node2 = cluster.add_instance(
"node2", user_configs=["config/config.xml"], with_zookeeper=True
"node2", user_configs=["config/config.xml"], with_zookeeper=True,
macros={"replica": "b", "shard": "shard1"}
)
@ -183,3 +185,11 @@ def test_replicated_table_ddl(started_cluster):
)
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)
def test_replicated_db(started_cluster):
node1.query("CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')")
node2.query("CREATE DATABASE test ENGINE = Replicated('/test/shared_stats', '{shard}', '{replica}')")
node1.query("CREATE TABLE test.test_stats (a Int64, b Int64) ENGINE = ReplicatedMergeTree() ORDER BY()")
node2.query("ALTER TABLE test.test_stats MODIFY COLUMN b Float64")
node2.query("ALTER TABLE test.test_stats MODIFY STATISTICS b TYPE tdigest")