mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
add ALTER COLUMN MODIFY/RESET SETTING
Signed-off-by: Duc Canh Le <duccanh.le@ahrefs.com>
This commit is contained in:
parent
f755e77534
commit
22afcd46f4
@ -467,10 +467,10 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
|
||||
|
||||
if (!column.settings.empty())
|
||||
{
|
||||
auto per_column_settings = std::make_shared<ASTSetQuery>();
|
||||
per_column_settings->is_standalone = false;
|
||||
per_column_settings->changes = column.settings;
|
||||
column_declaration->per_column_settings = std::move(per_column_settings);
|
||||
auto settings = std::make_shared<ASTSetQuery>();
|
||||
settings->is_standalone = false;
|
||||
settings->changes = column.settings;
|
||||
column_declaration->settings = std::move(settings);
|
||||
}
|
||||
|
||||
columns_list->children.push_back(column_declaration_ptr);
|
||||
@ -680,9 +680,9 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
|
||||
if (col_decl.ttl)
|
||||
column.ttl = col_decl.ttl;
|
||||
|
||||
if (col_decl.per_column_settings)
|
||||
if (col_decl.settings)
|
||||
{
|
||||
column.settings = col_decl.per_column_settings->as<ASTSetQuery &>().changes;
|
||||
column.settings = col_decl.settings->as<ASTSetQuery &>().changes;
|
||||
MergeTreeColumnSettings::validate(column.settings);
|
||||
}
|
||||
|
||||
@ -890,33 +890,22 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat
|
||||
throw Exception(ErrorCodes::DUPLICATE_COLUMN, "Column {} already exists", backQuoteIfNeed(column.name));
|
||||
}
|
||||
|
||||
if (create.storage && create.storage->engine)
|
||||
/// Check if _row_exists for lightweight delete column in column_lists for merge tree family.
|
||||
if (create.storage && create.storage->engine && endsWith(create.storage->engine->name, "MergeTree"))
|
||||
{
|
||||
/// Check if _row_exists for lightweight delete column in column_lists for merge tree family.
|
||||
if (endsWith(create.storage->engine->name, "MergeTree"))
|
||||
{
|
||||
auto search = all_columns.find(LightweightDeleteDescription::FILTER_COLUMN.name);
|
||||
if (search != all_columns.end())
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Cannot create table with column '{}' for *MergeTree engines because it "
|
||||
"is reserved for lightweight delete feature",
|
||||
LightweightDeleteDescription::FILTER_COLUMN.name);
|
||||
auto search_block_number = all_columns.find(BlockNumberColumn::name);
|
||||
if (search_block_number != all_columns.end())
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Cannot create table with column '{}' for *MergeTree engines because it "
|
||||
"is reserved for storing block number",
|
||||
BlockNumberColumn::name);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Only merge tree family supports column with custom column setting
|
||||
if (std::any_of(
|
||||
properties.columns.begin(),
|
||||
properties.columns.end(),
|
||||
[](const ColumnDescription & column) { return !column.settings.empty(); }))
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create table with column level settings for non-MergeTree engines");
|
||||
}
|
||||
auto search = all_columns.find(LightweightDeleteDescription::FILTER_COLUMN.name);
|
||||
if (search != all_columns.end())
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Cannot create table with column '{}' for *MergeTree engines because it "
|
||||
"is reserved for lightweight delete feature",
|
||||
LightweightDeleteDescription::FILTER_COLUMN.name);
|
||||
|
||||
auto search_block_number = all_columns.find(BlockNumberColumn::name);
|
||||
if (search_block_number != all_columns.end())
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
|
||||
"Cannot create table with column '{}' for *MergeTree engines because it "
|
||||
"is reserved for storing block number",
|
||||
BlockNumberColumn::name);
|
||||
}
|
||||
|
||||
const auto & settings = getContext()->getSettingsRef();
|
||||
|
@ -57,83 +57,83 @@ ASTPtr ASTColumnDeclaration::clone() const
|
||||
res->children.push_back(res->collation);
|
||||
}
|
||||
|
||||
if (per_column_settings)
|
||||
if (settings)
|
||||
{
|
||||
res->per_column_settings = per_column_settings->clone();
|
||||
res->children.push_back(res->per_column_settings);
|
||||
res->settings = settings->clone();
|
||||
res->children.push_back(res->settings);
|
||||
}
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
|
||||
void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, FormatState & state, FormatStateStacked frame) const
|
||||
{
|
||||
frame.need_parens = false;
|
||||
|
||||
/// We have to always backquote column names to avoid ambiguouty with INDEX and other declarations in CREATE query.
|
||||
settings.ostr << backQuote(name);
|
||||
format_settings.ostr << backQuote(name);
|
||||
|
||||
if (type)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
format_settings.ostr << ' ';
|
||||
|
||||
FormatStateStacked type_frame = frame;
|
||||
type_frame.indent = 0;
|
||||
|
||||
type->formatImpl(settings, state, type_frame);
|
||||
type->formatImpl(format_settings, state, type_frame);
|
||||
}
|
||||
|
||||
if (null_modifier)
|
||||
{
|
||||
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "")
|
||||
<< (*null_modifier ? "" : "NOT ") << "NULL" << (settings.hilite ? hilite_none : "");
|
||||
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "")
|
||||
<< (*null_modifier ? "" : "NOT ") << "NULL" << (format_settings.hilite ? hilite_none : "");
|
||||
}
|
||||
|
||||
if (default_expression)
|
||||
{
|
||||
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << default_specifier << (settings.hilite ? hilite_none : "");
|
||||
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << default_specifier << (format_settings.hilite ? hilite_none : "");
|
||||
if (!ephemeral_default)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
default_expression->formatImpl(settings, state, frame);
|
||||
format_settings.ostr << ' ';
|
||||
default_expression->formatImpl(format_settings, state, frame);
|
||||
}
|
||||
}
|
||||
|
||||
if (comment)
|
||||
{
|
||||
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COMMENT" << (settings.hilite ? hilite_none : "") << ' ';
|
||||
comment->formatImpl(settings, state, frame);
|
||||
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "COMMENT" << (format_settings.hilite ? hilite_none : "") << ' ';
|
||||
comment->formatImpl(format_settings, state, frame);
|
||||
}
|
||||
|
||||
if (codec)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
codec->formatImpl(settings, state, frame);
|
||||
format_settings.ostr << ' ';
|
||||
codec->formatImpl(format_settings, state, frame);
|
||||
}
|
||||
|
||||
if (stat_type)
|
||||
{
|
||||
settings.ostr << ' ';
|
||||
stat_type->formatImpl(settings, state, frame);
|
||||
format_settings.ostr << ' ';
|
||||
stat_type->formatImpl(format_settings, state, frame);
|
||||
}
|
||||
|
||||
if (ttl)
|
||||
{
|
||||
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "TTL" << (settings.hilite ? hilite_none : "") << ' ';
|
||||
ttl->formatImpl(settings, state, frame);
|
||||
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "TTL" << (format_settings.hilite ? hilite_none : "") << ' ';
|
||||
ttl->formatImpl(format_settings, state, frame);
|
||||
}
|
||||
|
||||
if (collation)
|
||||
{
|
||||
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "COLLATE" << (settings.hilite ? hilite_none : "") << ' ';
|
||||
collation->formatImpl(settings, state, frame);
|
||||
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "COLLATE" << (format_settings.hilite ? hilite_none : "") << ' ';
|
||||
collation->formatImpl(format_settings, state, frame);
|
||||
}
|
||||
|
||||
if (per_column_settings)
|
||||
if (settings)
|
||||
{
|
||||
settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << "SETTINGS" << (settings.hilite ? hilite_none : "") << ' ' << '(';
|
||||
per_column_settings->formatImpl(settings, state, frame);
|
||||
settings.ostr << ')';
|
||||
format_settings.ostr << ' ' << (format_settings.hilite ? hilite_keyword : "") << "SETTINGS" << (format_settings.hilite ? hilite_none : "") << ' ' << '(';
|
||||
settings->formatImpl(format_settings, state, frame);
|
||||
format_settings.ostr << ')';
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -22,13 +22,13 @@ public:
|
||||
ASTPtr stat_type;
|
||||
ASTPtr ttl;
|
||||
ASTPtr collation;
|
||||
ASTPtr per_column_settings;
|
||||
ASTPtr settings;
|
||||
bool primary_key_specifier = false;
|
||||
|
||||
String getID(char delim) const override { return "ColumnDeclaration" + (delim + name); }
|
||||
|
||||
ASTPtr clone() const override;
|
||||
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
void formatImpl(const FormatSettings & format_settings, FormatState & state, FormatStateStacked frame) const override;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -110,7 +110,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
ParserKeyword s_comment("COMMENT");
|
||||
ParserKeyword s_codec("CODEC");
|
||||
ParserKeyword s_ttl("TTL");
|
||||
ParserKeyword s_setting("SETTING");
|
||||
ParserKeyword s_settings("SETTINGS");
|
||||
|
||||
ParserKeyword s_remove_ttl("REMOVE TTL");
|
||||
ParserKeyword s_remove_sample_by("REMOVE SAMPLE BY");
|
||||
@ -702,15 +702,21 @@ 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_setting.ignore(pos, expected))
|
||||
{
|
||||
command->remove_property = "SETTING";
|
||||
if (!parser_reset_setting.parse(pos, command->settings_resets, expected))
|
||||
return false;
|
||||
}
|
||||
else if (s_settings.ignore(pos, expected))
|
||||
command->remove_property = "SETTINGS";
|
||||
else
|
||||
return false;
|
||||
}
|
||||
else if (s_modify_setting.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_settings.parse(pos, command->settings_changes, expected))
|
||||
return false;
|
||||
}
|
||||
else if (s_reset_setting.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_reset_setting.parse(pos, command->settings_resets, expected))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (s_first.ignore(pos, expected))
|
||||
|
@ -135,6 +135,8 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
ParserKeyword s_stat{"STATISTIC"};
|
||||
ParserKeyword s_ttl{"TTL"};
|
||||
ParserKeyword s_remove{"REMOVE"};
|
||||
ParserKeyword s_modify_setting("MODIFY SETTING");
|
||||
ParserKeyword s_reset_setting("RESET SETTING");
|
||||
ParserKeyword s_type{"TYPE"};
|
||||
ParserKeyword s_collate{"COLLATE"};
|
||||
ParserKeyword s_primary_key{"PRIMARY KEY"};
|
||||
@ -159,10 +161,12 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
/// This keyword may occur only in MODIFY COLUMN query. We check it here
|
||||
/// because ParserDataType parses types as an arbitrary identifiers and
|
||||
/// doesn't check that parsed string is existing data type. In this way
|
||||
/// REMOVE keyword can be parsed as data type and further parsing will fail.
|
||||
/// So we just check this keyword and in case of success return column
|
||||
/// declaration with name only.
|
||||
if (!require_type && s_remove.checkWithoutMoving(pos, expected))
|
||||
/// REMOVE, MODIFY SETTING, or RESET SETTING can be parsed as data type
|
||||
/// and further parsing will fail. So we just check these keyword and in
|
||||
/// case of success return column declaration with name only.
|
||||
if (!require_type
|
||||
&& (s_remove.checkWithoutMoving(pos, expected) || s_modify_setting.checkWithoutMoving(pos, expected)
|
||||
|| s_reset_setting.checkWithoutMoving(pos, expected)))
|
||||
{
|
||||
if (!check_keywords_after_name)
|
||||
return false;
|
||||
@ -181,10 +185,10 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
ASTPtr default_expression;
|
||||
ASTPtr comment_expression;
|
||||
ASTPtr codec_expression;
|
||||
ASTPtr per_column_settings;
|
||||
ASTPtr stat_type_expression;
|
||||
ASTPtr ttl_expression;
|
||||
ASTPtr collation_expression;
|
||||
ASTPtr settings;
|
||||
bool primary_key_specifier = false;
|
||||
|
||||
auto null_check_without_moving = [&]() -> bool
|
||||
@ -325,24 +329,19 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
primary_key_specifier = true;
|
||||
}
|
||||
|
||||
auto old_pos = pos;
|
||||
if (s_settings.ignore(pos, expected))
|
||||
{
|
||||
ParserToken parser_opening_bracket(TokenType::OpeningRoundBracket);
|
||||
if (parser_opening_bracket.ignore(pos, expected))
|
||||
if (parser_opening_bracket.check(pos, expected))
|
||||
{
|
||||
if (!settings_parser.parse(pos, per_column_settings, expected))
|
||||
if (!settings_parser.parse(pos, settings, expected))
|
||||
return false;
|
||||
ParserToken parser_closing_bracket(TokenType::ClosingRoundBracket);
|
||||
if (!parser_closing_bracket.ignore(pos, expected))
|
||||
return false;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// This could be settings in alter query
|
||||
/// E.g: ALTER TABLE alter_enum_array MODIFY COLUMN x String SETTINGS mutations_sync=2;
|
||||
pos = old_pos;
|
||||
}
|
||||
/// This could be settings in alter query
|
||||
/// E.g: ALTER TABLE alter_enum_array MODIFY COLUMN x String SETTINGS mutations_sync=2;
|
||||
}
|
||||
|
||||
node = column_declaration;
|
||||
@ -375,10 +374,10 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
|
||||
column_declaration->children.push_back(std::move(codec_expression));
|
||||
}
|
||||
|
||||
if (per_column_settings)
|
||||
if (settings)
|
||||
{
|
||||
column_declaration->per_column_settings = per_column_settings;
|
||||
column_declaration->children.push_back(std::move(per_column_settings));
|
||||
column_declaration->settings = settings;
|
||||
column_declaration->children.push_back(std::move(settings));
|
||||
}
|
||||
|
||||
if (stat_type_expression)
|
||||
|
@ -75,8 +75,8 @@ AlterCommand::RemoveProperty removePropertyFromString(const String & property)
|
||||
return AlterCommand::RemoveProperty::CODEC;
|
||||
else if (property == "TTL")
|
||||
return AlterCommand::RemoveProperty::TTL;
|
||||
else if (property == "SETTING")
|
||||
return AlterCommand::RemoveProperty::SETTING;
|
||||
else if (property == "SETTINGS")
|
||||
return AlterCommand::RemoveProperty::SETTINGS;
|
||||
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot remove unknown property '{}'", property);
|
||||
}
|
||||
@ -152,16 +152,6 @@ 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)
|
||||
{
|
||||
@ -186,8 +176,24 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
|
||||
if (ast_col_decl.codec)
|
||||
command.codec = ast_col_decl.codec;
|
||||
|
||||
if (ast_col_decl.per_column_settings)
|
||||
command.settings_changes = ast_col_decl.per_column_settings->as<ASTSetQuery &>().changes;
|
||||
if (ast_col_decl.settings)
|
||||
command.settings_changes = ast_col_decl.settings->as<ASTSetQuery &>().changes;
|
||||
|
||||
/// At most only one of ast_col_decl.settings or command_ast->settings_changes is non-null
|
||||
if (command_ast->settings_changes)
|
||||
{
|
||||
command.settings_changes = command_ast->settings_changes->as<ASTSetQuery &>().changes;
|
||||
command.append_column_setting = true;
|
||||
}
|
||||
|
||||
if (command_ast->settings_resets)
|
||||
{
|
||||
for (const ASTPtr & identifier_ast : command_ast->settings_resets->children)
|
||||
{
|
||||
const auto & identifier = identifier_ast->as<ASTIdentifier &>();
|
||||
command.settings_resets.emplace(identifier.name());
|
||||
}
|
||||
}
|
||||
|
||||
if (command_ast->column)
|
||||
command.after_column = getIdentifierName(command_ast->column);
|
||||
@ -517,10 +523,9 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
{
|
||||
column.ttl.reset();
|
||||
}
|
||||
else if (to_remove == RemoveProperty::SETTING)
|
||||
else if (to_remove == RemoveProperty::SETTINGS)
|
||||
{
|
||||
for (const auto & setting : settings_resets)
|
||||
column.settings.removeSetting(setting);
|
||||
column.settings.clear();
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -539,7 +544,17 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
|
||||
if (!settings_changes.empty())
|
||||
{
|
||||
MergeTreeColumnSettings::validate(settings_changes);
|
||||
column.settings = settings_changes;
|
||||
if (append_column_setting)
|
||||
for (const auto & change : settings_changes)
|
||||
column.settings.setSetting(change.name, change.value);
|
||||
else
|
||||
column.settings = settings_changes;
|
||||
}
|
||||
|
||||
if (!settings_resets.empty())
|
||||
{
|
||||
for (const auto & setting : settings_resets)
|
||||
column.settings.removeSetting(setting);
|
||||
}
|
||||
|
||||
/// User specified default expression or changed
|
||||
@ -1384,15 +1399,6 @@ 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::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);
|
||||
|
@ -65,7 +65,7 @@ struct AlterCommand
|
||||
COMMENT,
|
||||
CODEC,
|
||||
TTL,
|
||||
SETTING
|
||||
SETTINGS
|
||||
};
|
||||
|
||||
Type type = UNKNOWN;
|
||||
@ -138,10 +138,10 @@ struct AlterCommand
|
||||
/// For ADD and MODIFY
|
||||
ASTPtr codec = nullptr;
|
||||
|
||||
/// For MODIFY SETTING or MODIFY COLUMN with SETTINGS (...)
|
||||
/// For MODIFY SETTING or MODIFY COLUMN MODIFY SETTING
|
||||
SettingsChanges settings_changes;
|
||||
|
||||
/// For RESET SETTING or MODIFY COLUMN REMOVE SETTING (...)
|
||||
/// For RESET SETTING or MODIFY COLUMN RESET SETTING
|
||||
std::set<String> settings_resets;
|
||||
|
||||
/// For MODIFY_QUERY
|
||||
@ -156,6 +156,9 @@ struct AlterCommand
|
||||
/// What to remove from column (or TTL)
|
||||
RemoveProperty to_remove = RemoveProperty::NO_PROPERTY;
|
||||
|
||||
/// Is this MODIFY COLUMN MODIFY SETTING or MODIFY COLUMN <column with settings declaration)
|
||||
bool append_column_setting = false;
|
||||
|
||||
static std::optional<AlterCommand> parse(const ASTAlterCommand * command);
|
||||
|
||||
void apply(StorageInMemoryMetadata & metadata, ContextPtr context) const;
|
||||
|
@ -159,8 +159,8 @@ void ColumnDescription::readText(ReadBuffer & buf)
|
||||
if (col_ast->ttl)
|
||||
ttl = col_ast->ttl;
|
||||
|
||||
if (col_ast->per_column_settings)
|
||||
settings = col_ast->per_column_settings->as<ASTSetQuery &>().changes;
|
||||
if (col_ast->settings)
|
||||
settings = col_ast->settings->as<ASTSetQuery &>().changes;
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse column description");
|
||||
|
@ -1,7 +1,9 @@
|
||||
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
|
||||
CREATE TABLE default.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/default/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
|
||||
CREATE TABLE default.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (min_compress_block_size = 8192, 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/default/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192
|
||||
CREATE TABLE default.t\n(\n `id` UInt64 CODEC(ZSTD(1)),\n `long_string` String CODEC(ZSTD(9, 24)) SETTINGS (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/default/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192
|
||||
CREATE TABLE default.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/default/t/2870\', \'r1\')\nORDER BY id\nSETTINGS min_bytes_for_wide_part = 1, index_granularity = 8192
|
||||
CREATE TABLE default.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/default/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,7 +1,5 @@
|
||||
-- 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)),
|
||||
@ -21,7 +19,15 @@ INSERT INTO TABLE t SELECT number, randomPrintableASCII(1000), randomPrintableAS
|
||||
|
||||
SELECT count() FROM t;
|
||||
|
||||
ALTER TABLE t MODIFY COLUMN long_string REMOVE SETTING min_compress_block_size, max_compress_block_size;
|
||||
ALTER TABLE t MODIFY COLUMN long_string MODIFY SETTING min_compress_block_size = 8192;
|
||||
|
||||
SHOW CREATE t;
|
||||
|
||||
ALTER TABLE t MODIFY COLUMN long_string RESET SETTING min_compress_block_size;
|
||||
|
||||
SHOW CREATE t;
|
||||
|
||||
ALTER TABLE t MODIFY COLUMN long_string REMOVE SETTINGS;
|
||||
|
||||
SHOW CREATE t;
|
||||
|
||||
@ -46,7 +52,9 @@ SETTINGS min_bytes_for_wide_part = 1;
|
||||
INSERT INTO TABLE t2 SELECT number, tuple(number, number), concat('{"key": ', toString(number), ' ,"value": ', toString(rand(number+1)), '}') FROM numbers(1000);
|
||||
SELECT tup, json.key AS key FROM t2 ORDER BY key LIMIT 10;
|
||||
|
||||
DROP TABLE t2;
|
||||
|
||||
-- Non-supported column setting
|
||||
CREATE TABLE t3
|
||||
(
|
||||
`id` UInt64 CODEC(ZSTD(1)),
|
||||
@ -56,6 +64,7 @@ ENGINE = MergeTree
|
||||
ORDER BY id
|
||||
SETTINGS min_bytes_for_wide_part = 1; -- {serverError 115}
|
||||
|
||||
-- Invalid setting values
|
||||
CREATE TABLE t4
|
||||
(
|
||||
`id` UInt64 CODEC(ZSTD(1)),
|
||||
@ -64,4 +73,3 @@ CREATE TABLE t4
|
||||
ENGINE = TinyLog
|
||||
ORDER BY id; -- {serverError 44}
|
||||
|
||||
DROP DATABASE db_02780;
|
Loading…
Reference in New Issue
Block a user