mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-02 04:22:03 +00:00
support ALTER table MODIFY column REMOVE SETTING ..
Signed-off-by: Duc Canh Le <duccanh.le@ahrefs.com>
This commit is contained in:
parent
50ad6457e4
commit
8e374882cc
@ -103,7 +103,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKeyword s_comment("COMMENT");
|
||||
ParserKeyword s_codec("CODEC");
|
||||
ParserKeyword s_ttl("TTL");
|
||||
ParserKeyword s_column_settings("COLUMN SETTINGS");
|
||||
ParserKeyword s_setting("SETTING");
|
||||
|
||||
ParserKeyword s_remove_ttl("REMOVE TTL");
|
||||
ParserKeyword s_remove_sample_by("REMOVE SAMPLE BY");
|
||||
@ -637,8 +637,12 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
command->remove_property = "CODEC";
|
||||
else if (s_ttl.ignore(pos, expected))
|
||||
command->remove_property = "TTL";
|
||||
else if (s_column_settings.ignore(pos, expected))
|
||||
command->remove_property = "COLUMN SETTINGS";
|
||||
else if (s_setting.ignore(pos, expected))
|
||||
{
|
||||
command->remove_property = "SETTING";
|
||||
if (!parser_reset_setting.parse(pos, command->settings_resets, expected))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
@ -66,8 +66,8 @@ AlterCommand::RemoveProperty removePropertyFromString(const String & property)
|
||||
return AlterCommand::RemoveProperty::CODEC;
|
||||
else if (property == "TTL")
|
||||
return AlterCommand::RemoveProperty::TTL;
|
||||
else if (property == "COLUMN SETTINGS")
|
||||
return AlterCommand::RemoveProperty::SETTINGS;
|
||||
else if (property == "SETTING")
|
||||
return AlterCommand::RemoveProperty::SETTING;
|
||||
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot remove unknown property '{}'", property);
|
||||
}
|
||||
@ -143,6 +143,16 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
const auto & ast_col_decl = command_ast->col_decl->as<ASTColumnDeclaration &>();
|
||||
command.column_name = ast_col_decl.name;
|
||||
command.to_remove = removePropertyFromString(command_ast->remove_property);
|
||||
if (command.to_remove == RemoveProperty::SETTING)
|
||||
{
|
||||
for (const ASTPtr & identifier_ast : command_ast->settings_resets->children)
|
||||
{
|
||||
const auto & identifier = identifier_ast->as<ASTIdentifier &>();
|
||||
auto insertion = command.settings_resets.emplace(identifier.name());
|
||||
if (!insertion.second)
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Duplicate setting name {}", backQuote(identifier.name()));
|
||||
}
|
||||
}
|
||||
|
||||
if (ast_col_decl.type)
|
||||
{
|
||||
@ -434,9 +444,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
{
|
||||
column.ttl.reset();
|
||||
}
|
||||
else if (to_remove == RemoveProperty::SETTINGS)
|
||||
else if (to_remove == RemoveProperty::SETTING)
|
||||
{
|
||||
column.settings.clear();
|
||||
for (const auto & setting : settings_resets)
|
||||
column.settings.removeSetting(setting);
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -1205,11 +1216,15 @@ void AlterCommands::validate(const StoragePtr & table, ContextPtr context) const
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Column {} doesn't have COMMENT, cannot remove it",
|
||||
backQuote(column_name));
|
||||
if (command.to_remove == AlterCommand::RemoveProperty::SETTINGS && column_from_table.settings.empty())
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"Column {} doesn't have SETTINGS, cannot remove it",
|
||||
backQuote(column_name));
|
||||
if (command.to_remove == AlterCommand::RemoveProperty::SETTING)
|
||||
{
|
||||
for (const auto & setting : command.settings_resets)
|
||||
{
|
||||
if (!column_from_table.settings.tryGet(setting))
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS, "Column {} doesn't have SETTINGS, cannot remove it", backQuote(column_name));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
modified_columns.emplace(column_name);
|
||||
|
@ -62,7 +62,7 @@ struct AlterCommand
|
||||
COMMENT,
|
||||
CODEC,
|
||||
TTL,
|
||||
SETTINGS
|
||||
SETTING
|
||||
};
|
||||
|
||||
Type type = UNKNOWN;
|
||||
@ -131,10 +131,10 @@ struct AlterCommand
|
||||
/// For ADD and MODIFY
|
||||
ASTPtr codec = nullptr;
|
||||
|
||||
/// For MODIFY SETTING or MODIFY COLUMN SETTINGS
|
||||
/// For MODIFY SETTING or MODIFY COLUMN with SETTINGS (...)
|
||||
SettingsChanges settings_changes;
|
||||
|
||||
/// For RESET SETTING
|
||||
/// For RESET SETTING or MODIFY COLUMN REMOVE SETTING (...)
|
||||
std::set<String> settings_resets;
|
||||
|
||||
/// For MODIFY_QUERY
|
||||
|
@ -1,4 +1,7 @@
|
||||
CREATE TABLE db_02780.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/db_02780/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192
|
||||
1000
|
||||
CREATE TABLE db_02780.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/db_02780/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192
|
||||
CREATE TABLE db_02780.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),\n `v1` String CODEC(ZSTD(1)),\n `v2` UInt64 CODEC(ZSTD(1)),\n `v3` Float32 CODEC(ZSTD(1)),\n `v4` Float64 CODEC(ZSTD(1))\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/db_02780/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192
|
||||
(0,0) 0
|
||||
(1,1) 1
|
||||
(2,2) 2
|
||||
|
@ -1,8 +1,11 @@
|
||||
-- Tags: no-random-merge-tree-settings
|
||||
DROP DATABASE IF EXISTS db_02780;
|
||||
CREATE DATABASE db_02780;
|
||||
USE db_02780;
|
||||
CREATE TABLE t
|
||||
(
|
||||
`id` UInt64 CODEC(ZSTD(1)),
|
||||
`long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840),
|
||||
`long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840),
|
||||
`v1` String CODEC(ZSTD(1)),
|
||||
`v2` UInt64 CODEC(ZSTD(1)),
|
||||
`v3` Float32 CODEC(ZSTD(1)),
|
||||
@ -12,10 +15,22 @@ ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/t/2870', 'r1')
|
||||
ORDER BY id
|
||||
SETTINGS min_bytes_for_wide_part = 1;
|
||||
|
||||
SHOW CREATE t;
|
||||
|
||||
INSERT INTO TABLE t SELECT number, randomPrintableASCII(1000), randomPrintableASCII(10), rand(number), rand(number+1), rand(number+2) FROM numbers(1000);
|
||||
|
||||
SELECT count() FROM t;
|
||||
|
||||
ALTER TABLE t MODIFY COLUMN long_string REMOVE SETTING min_compress_block_size, max_compress_block_size;
|
||||
|
||||
SHOW CREATE t;
|
||||
|
||||
ALTER TABLE t MODIFY COLUMN long_string String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 163840, max_compress_block_size = 163840);
|
||||
|
||||
SHOW CREATE t;
|
||||
|
||||
DROP TABLE t;
|
||||
|
||||
SET allow_experimental_object_type = 1;
|
||||
|
||||
CREATE TABLE t2
|
||||
@ -47,4 +62,6 @@ CREATE TABLE t4
|
||||
`long_string` String CODEC(ZSTD(1)) SETTINGS (min_compress_block_size = 81920, max_compress_block_size = 163840),
|
||||
)
|
||||
ENGINE = TinyLog
|
||||
ORDER BY id; -- {serverError 44}
|
||||
ORDER BY id; -- {serverError 44}
|
||||
|
||||
DROP DATABASE db_02780;
|
Loading…
Reference in New Issue
Block a user