diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index adf3513ba40..11fc3ca4e29 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -449,6 +449,7 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) break; } + // If any part of storage definition is found create storage node if (!storage_like) return false; diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 027cd1af7c9..d4bf6374c02 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -79,6 +79,7 @@ ORDER BY expr [SAMPLE BY expr] [TTL expr [DELETE|TO DISK 'xxx'|TO VOLUME 'xxx'], ...] [SETTINGS name=value, ...] +[COMMENT 'comment'] See details in documentation: https://clickhouse.com/docs/en/engines/table-engines/mergetree-family/mergetree/. Other engines of the family support different syntax, see details in the corresponding documentation topics. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c02c96f62be..8e029214b06 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -326,6 +326,11 @@ void StorageMergeTree::alter( changeSettings(new_metadata.settings_changes, table_lock_holder); DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata); } + else if (commands.isCommentAlter()) + { + setInMemoryMetadata(new_metadata); + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(local_context, table_id, new_metadata); + } else { if (!maybe_mutation_commands.empty() && maybe_mutation_commands.containBarrierCommand()) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bb99e21e4ab..d4b85c72ccc 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5375,6 +5375,17 @@ void StorageReplicatedMergeTree::alter( return; } + if (commands.isCommentAlter()) + { + StorageInMemoryMetadata future_metadata = getInMemoryMetadata(); + commands.apply(future_metadata, query_context); + + setInMemoryMetadata(future_metadata); + + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id, future_metadata); + return; + } + auto ast_to_str = [](ASTPtr query) -> String { if (!query) @@ -5444,12 +5455,27 @@ void StorageReplicatedMergeTree::alter( String new_columns_str = future_metadata.columns.toString(); ops.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "columns", new_columns_str, -1)); - if (ast_to_str(current_metadata->settings_changes) != ast_to_str(future_metadata.settings_changes)) + bool settings_are_changed = (ast_to_str(current_metadata->settings_changes) != ast_to_str(future_metadata.settings_changes)); + bool comment_is_changed = (current_metadata->comment != future_metadata.comment); + + if (settings_are_changed || comment_is_changed) { - /// Just change settings StorageInMemoryMetadata metadata_copy = *current_metadata; - metadata_copy.settings_changes = future_metadata.settings_changes; - changeSettings(metadata_copy.settings_changes, table_lock_holder); + + if (settings_are_changed) + { + /// Just change settings + metadata_copy.settings_changes = future_metadata.settings_changes; + changeSettings(metadata_copy.settings_changes, table_lock_holder); + } + + /// The comment is not replicated as of today, but we can implement it later. + if (comment_is_changed) + { + metadata_copy.setComment(future_metadata.comment); + setInMemoryMetadata(metadata_copy); + } + DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(query_context, table_id, metadata_copy); } diff --git a/tests/queries/0_stateless/02792_alter_table_modify_comment.reference b/tests/queries/0_stateless/02792_alter_table_modify_comment.reference new file mode 100644 index 00000000000..b148993285c --- /dev/null +++ b/tests/queries/0_stateless/02792_alter_table_modify_comment.reference @@ -0,0 +1,20 @@ +Hello +World +Hello +World +Hello +World +Hello +World +Hello +World + +World + +World + +World + +World + +World diff --git a/tests/queries/0_stateless/02792_alter_table_modify_comment.sql b/tests/queries/0_stateless/02792_alter_table_modify_comment.sql new file mode 100644 index 00000000000..70c3d266bfe --- /dev/null +++ b/tests/queries/0_stateless/02792_alter_table_modify_comment.sql @@ -0,0 +1,69 @@ +DROP TABLE IF EXISTS t; + +# Memory, MergeTree, and ReplicatedMergeTree + +CREATE TABLE t (x UInt8) ENGINE = Memory COMMENT 'Hello'; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +ALTER TABLE t MODIFY COMMENT 'World'; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +DROP TABLE t; + +CREATE TABLE t (x UInt8) ENGINE = MergeTree ORDER BY () COMMENT 'Hello'; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +ALTER TABLE t MODIFY COMMENT 'World'; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +DROP TABLE t; + +# The case when there are many operations in one ALTER + +CREATE TABLE t (x UInt8) ENGINE = MergeTree ORDER BY () COMMENT 'Hello'; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +ALTER TABLE t MODIFY COMMENT 'World', MODIFY COLUMN x UInt16; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +DROP TABLE t; + +# Note that the table comment is not replicated. We can implement it later. + +CREATE TABLE t (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_comment_table1/t', '1') ORDER BY () COMMENT 'Hello'; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +ALTER TABLE t MODIFY COMMENT 'World'; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +DROP TABLE t SYNC; + +CREATE TABLE t (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_comment_table2/t', '1') ORDER BY () COMMENT 'Hello'; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +ALTER TABLE t MODIFY COMMENT 'World', MODIFY COLUMN x UInt16; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +DROP TABLE t SYNC; + +# The cases when there is no comment on creation + +CREATE TABLE t (x UInt8) ENGINE = Memory; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +ALTER TABLE t MODIFY COMMENT 'World'; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +DROP TABLE t; + +CREATE TABLE t (x UInt8) ENGINE = MergeTree ORDER BY (); +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +ALTER TABLE t MODIFY COMMENT 'World'; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +DROP TABLE t; + +CREATE TABLE t (x UInt8) ENGINE = MergeTree ORDER BY (); +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +ALTER TABLE t MODIFY COMMENT 'World', MODIFY COLUMN x UInt16; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +DROP TABLE t; + +CREATE TABLE t (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_comment_table3/t', '1') ORDER BY (); +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +ALTER TABLE t MODIFY COMMENT 'World'; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +DROP TABLE t SYNC; + +CREATE TABLE t (x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_comment_table4/t', '1') ORDER BY (); +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +ALTER TABLE t MODIFY COMMENT 'World', MODIFY COLUMN x UInt16; +SELECT comment FROM system.tables WHERE database = currentDatabase() AND table = 't'; +DROP TABLE t SYNC;