Skip to content

Commit

Permalink
Merge pull request #51304 from ClickHouse/alter-table-add-comment
Browse files Browse the repository at this point in the history
Add support for ALTER TABLE MODIFY COMMENT
  • Loading branch information
alexey-milovidov committed Sep 7, 2023
2 parents de59c52 + ce8ab5c commit 1b30f91
Show file tree
Hide file tree
Showing 12 changed files with 160 additions and 20 deletions.
6 changes: 6 additions & 0 deletions docs/en/sql-reference/statements/alter/comment.md
Expand Up @@ -57,3 +57,9 @@ Output of a removed comment:
│ │
└─────────┘
```

**Caveats**

For Replicated tables, the comment can be different on different replicas. Modifying the comment applies to a single replica.

The feature is available since version 23.9. It does not work in previous ClickHouse versions.
4 changes: 2 additions & 2 deletions src/Interpreters/InterpreterAlterQuery.cpp
Expand Up @@ -156,10 +156,10 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter)
if (typeid_cast<DatabaseReplicated *>(database.get()))
{
int command_types_count = !mutation_commands.empty() + !partition_commands.empty() + !alter_commands.empty();
bool mixed_settings_amd_metadata_alter = alter_commands.hasSettingsAlterCommand() && !alter_commands.isSettingsAlter();
bool mixed_settings_amd_metadata_alter = alter_commands.hasNonReplicatedAlterCommand() && !alter_commands.areNonReplicatedAlterCommands();
if (1 < command_types_count || mixed_settings_amd_metadata_alter)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "For Replicated databases it's not allowed "
"to execute ALTERs of different types in single query");
"to execute ALTERs of different types (replicated and non replicated) in single query");
}

if (mutation_commands.hasNonEmptyMutationCommands())
Expand Down
1 change: 1 addition & 0 deletions src/Parsers/ParserCreateQuery.cpp
Expand Up @@ -526,6 +526,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;
Expand Down
9 changes: 7 additions & 2 deletions src/Storages/AlterCommands.cpp
Expand Up @@ -1350,9 +1350,14 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
validateColumnsDefaultsAndGetSampleBlock(default_expr_list, all_columns.getAll(), context);
}

bool AlterCommands::hasSettingsAlterCommand() const
bool AlterCommands::hasNonReplicatedAlterCommand() const
{
return std::any_of(begin(), end(), [](const AlterCommand & c) { return c.isSettingsAlter(); });
return std::any_of(begin(), end(), [](const AlterCommand & c) { return c.isSettingsAlter() || c.isCommentAlter(); });
}

bool AlterCommands::areNonReplicatedAlterCommands() const
{
return std::all_of(begin(), end(), [](const AlterCommand & c) { return c.isSettingsAlter() || c.isCommentAlter(); });
}

bool AlterCommands::isSettingsAlter() const
Expand Down
7 changes: 5 additions & 2 deletions src/Storages/AlterCommands.h
Expand Up @@ -196,8 +196,11 @@ class AlterCommands : public std::vector<AlterCommand>
/// Commands have to be prepared before apply.
void apply(StorageInMemoryMetadata & metadata, ContextPtr context) const;

/// At least one command modify settings.
bool hasSettingsAlterCommand() const;
/// At least one command modify settings or comments.
bool hasNonReplicatedAlterCommand() const;

/// All commands modify settings or comments.
bool areNonReplicatedAlterCommands() const;

/// All commands modify settings only.
bool isSettingsAlter() const;
Expand Down
1 change: 1 addition & 0 deletions src/Storages/MergeTree/registerStorageMergeTree.cpp
Expand Up @@ -80,6 +80,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.
Expand Down
5 changes: 5 additions & 0 deletions src/Storages/StorageMergeTree.cpp
Expand Up @@ -332,6 +332,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())
Expand Down
34 changes: 30 additions & 4 deletions src/Storages/StorageReplicatedMergeTree.cpp
Expand Up @@ -5694,6 +5694,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)
Expand Down Expand Up @@ -5763,12 +5774,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);
}

Expand Down
3 changes: 2 additions & 1 deletion tests/queries/0_stateless/00725_comment_columns_long.sql
@@ -1,4 +1,5 @@
-- Tags: long
-- Tags: long, no-replicated-database
-- Tag no-replicated-database: Unsupported type of ALTER query

DROP TABLE IF EXISTS check_query_comment_column;

Expand Down
Expand Up @@ -124,21 +124,21 @@ CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\n
comment= Test table with comment

change a comment
CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/02020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'Test table with comment\'
comment= Test table with comment
CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/02020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'new comment on a table\'
comment= new comment on a table

remove a comment
CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/02020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'Test table with comment\'
comment= Test table with comment
CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/02020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192
comment=

add a comment back
CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/02020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'Test table with comment\'
comment= Test table with comment
CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/02020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'another comment on a table\'
comment= another comment on a table

detach table
CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/02020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'Test table with comment\'
CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/02020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'another comment on a table\'

re-attach table
CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/02020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'Test table with comment\'
comment= Test table with comment
CREATE TABLE default.comment_test_table\n(\n `k` UInt64,\n `s` String\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/02020_alter_table_modify_comment_default\', \'1\')\nORDER BY k\nSETTINGS index_granularity = 8192\nCOMMENT \'another comment on a table\'
comment= another comment on a table

@@ -0,0 +1,20 @@
Hello
World
Hello
World
Hello
World
Hello
World
Hello
World

World

World

World

World

World
72 changes: 72 additions & 0 deletions tests/queries/0_stateless/02792_alter_table_modify_comment.sql
@@ -0,0 +1,72 @@
-- Tags: no-replicated-database
-- Tag no-replicated-database: Unsupported type of ALTER query

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;

0 comments on commit 1b30f91

Please sign in to comment.