Fix ALTER COMMENT queries ON CLUSTER

This commit is contained in:
Nikolay Degterinsky 2023-11-09 06:19:00 +00:00
parent e33dc02c8c
commit 96f558f51a
6 changed files with 86 additions and 1 deletions

View File

@ -725,7 +725,8 @@ bool DDLWorker::taskShouldBeExecutedOnLeader(const ASTPtr & ast_ddl, const Stora
// Setting alters should be executed on all replicas
if (alter->isSettingsAlter() ||
alter->isFreezeAlter() ||
alter->isMovePartitionToDiskOrVolumeAlter())
alter->isMovePartitionToDiskOrVolumeAlter() ||
alter->isCommentAlter())
return false;
}

View File

@ -486,6 +486,11 @@ bool ASTAlterQuery::isDropPartitionAlter() const
return isOneCommandTypeOnly(ASTAlterCommand::DROP_PARTITION) || isOneCommandTypeOnly(ASTAlterCommand::DROP_DETACHED_PARTITION);
}
bool ASTAlterQuery::isCommentAlter() const
{
return isOneCommandTypeOnly(ASTAlterCommand::COMMENT_COLUMN) || isOneCommandTypeOnly(ASTAlterCommand::MODIFY_COMMENT);
}
bool ASTAlterQuery::isMovePartitionToDiskOrVolumeAlter() const
{
if (command_list)

View File

@ -239,6 +239,8 @@ public:
bool isMovePartitionToDiskOrVolumeAlter() const;
bool isCommentAlter() const;
String getID(char) const override;
ASTPtr clone() const override;

View File

@ -0,0 +1,16 @@
<clickhouse>
<remote_servers>
<cluster>
<shard>
<replica>
<host>node_1</host>
<port>9000</port>
</replica>
<replica>
<host>node_2</host>
<port>9000</port>
</replica>
</shard>
</cluster>
</remote_servers>
</clickhouse>

View File

@ -0,0 +1,61 @@
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__)
node_1 = cluster.add_instance(
"node_1",
main_configs=["configs/clusters.xml"],
with_zookeeper=True,
macros={"shard": 1, "replica": 1},
)
node_2 = cluster.add_instance(
"node_2",
main_configs=["configs/clusters.xml"],
with_zookeeper=True,
macros={"shard": 1, "replica": 2},
)
def assert_create_query(nodes, database_name, table_name, expected):
query = "SELECT create_table_query FROM system.tables WHERE database='{}' AND table='{}'".format(
database_name, table_name
)
for node in nodes:
assert_eq_with_retry(node, query, expected)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_comment(started_cluster):
node_1.query(
"CREATE TABLE test_table ON CLUSTER 'cluster' (id Int64) ENGINE=ReplicatedMergeTree() ORDER BY id"
)
node_1.query(
"ALTER TABLE test_table ON CLUSTER 'cluster' COMMENT COLUMN id 'column_comment_1'"
)
node_1.query(
"ALTER TABLE test_table ON CLUSTER 'cluster' MODIFY COMMENT 'table_comment_1';"
)
expected = "CREATE TABLE default.test_table (`id` Int64 COMMENT \\'column_comment_1\\') ENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\') ORDER BY id SETTINGS index_granularity = 8192 COMMENT \\'table_comment_1\\'"
assert_create_query([node_1, node_2], "default", "test_table", expected)
node_1.query(
"ALTER TABLE test_table ON CLUSTER 'cluster' COMMENT COLUMN id 'column_comment_2'"
)
node_1.query(
"ALTER TABLE test_table ON CLUSTER 'cluster' MODIFY COMMENT 'table_comment_2';"
)
expected = "CREATE TABLE default.test_table (`id` Int64 COMMENT \\'column_comment_2\\') ENGINE = ReplicatedMergeTree(\\'/clickhouse/tables/{uuid}/{shard}\\', \\'{replica}\\') ORDER BY id SETTINGS index_granularity = 8192 COMMENT \\'table_comment_2\\'"
assert_create_query([node_1, node_2], "default", "test_table", expected)