Merge pull request #30180 from CurtizJ/remove-sample-by

Allow to remove `SAMPLE BY` expression
This commit is contained in:
Anton Popov 2021-10-21 17:09:53 +03:00 committed by GitHub
commit b1a74af44d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 89 additions and 6 deletions

View File

@ -180,6 +180,8 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo
if (metadata.sampling_key.definition_ast)
storage_ast.set(storage_ast.sample_by, metadata.sampling_key.definition_ast);
else if (storage_ast.sample_by != nullptr) /// SAMPLE BY was removed
storage_ast.sample_by = nullptr;
if (metadata.table_ttl.definition_ast)
storage_ast.set(storage_ast.ttl_table, metadata.table_ttl.definition_ast);

View File

@ -270,6 +270,7 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS
required_access.emplace_back(AccessType::ALTER_ORDER_BY, database, table);
break;
}
case ASTAlterCommand::REMOVE_SAMPLE_BY:
case ASTAlterCommand::MODIFY_SAMPLE_BY:
{
required_access.emplace_back(AccessType::ALTER_SAMPLE_BY, database, table);

View File

@ -157,6 +157,10 @@ void ASTAlterCommand::formatImpl(const FormatSettings & settings, FormatState &
settings.ostr << (settings.hilite ? hilite_keyword : "") << "MODIFY SAMPLE BY " << (settings.hilite ? hilite_none : "");
sample_by->formatImpl(settings, state, frame);
}
else if (type == ASTAlterCommand::REMOVE_SAMPLE_BY)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "REMOVE SAMPLE BY" << (settings.hilite ? hilite_none : "");
}
else if (type == ASTAlterCommand::ADD_INDEX)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << "ADD INDEX " << (if_not_exists ? "IF NOT EXISTS " : "")

View File

@ -41,6 +41,7 @@ public:
RESET_SETTING,
MODIFY_QUERY,
REMOVE_TTL,
REMOVE_SAMPLE_BY,
ADD_INDEX,
DROP_INDEX,

View File

@ -104,6 +104,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ParserKeyword s_ttl("TTL");
ParserKeyword s_remove_ttl("REMOVE TTL");
ParserKeyword s_remove_sample_by("REMOVE SAMPLE BY");
ParserCompoundIdentifier parser_name;
ParserStringLiteral parser_string_literal;
@ -669,6 +670,10 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
command->type = ASTAlterCommand::MODIFY_SAMPLE_BY;
}
else if (s_remove_sample_by.ignore(pos, expected))
{
command->type = ASTAlterCommand::REMOVE_SAMPLE_BY;
}
else if (s_delete.ignore(pos, expected))
{
if (s_in_partition.ignore(pos, expected))

View File

@ -205,6 +205,13 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
command.sample_by = command_ast->sample_by;
return command;
}
else if (command_ast->type == ASTAlterCommand::REMOVE_SAMPLE_BY)
{
AlterCommand command;
command.ast = command_ast->clone();
command.type = AlterCommand::REMOVE_SAMPLE_BY;
return command;
}
else if (command_ast->type == ASTAlterCommand::ADD_INDEX)
{
AlterCommand command;
@ -463,6 +470,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context)
{
metadata.sampling_key.recalculateWithNewAST(sample_by, metadata.columns, context);
}
else if (type == REMOVE_SAMPLE_BY)
{
metadata.sampling_key = {};
}
else if (type == COMMENT_COLUMN)
{
metadata.columns.modify(column_name,
@ -745,7 +756,7 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada
return false;
/// We remove properties on metadata level
if (isRemovingProperty() || type == REMOVE_TTL)
if (isRemovingProperty() || type == REMOVE_TTL || type == REMOVE_SAMPLE_BY)
return false;
if (type == DROP_COLUMN || type == DROP_INDEX || type == DROP_PROJECTION || type == RENAME_COLUMN)
@ -1208,6 +1219,10 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, ContextPt
{
throw Exception{"Table doesn't have any table TTL expression, cannot remove", ErrorCodes::BAD_ARGUMENTS};
}
else if (command.type == AlterCommand::REMOVE_SAMPLE_BY && !metadata.hasSamplingKey())
{
throw Exception{"Table doesn't have SAMPLE BY, cannot remove", ErrorCodes::BAD_ARGUMENTS};
}
/// Collect default expressions for MODIFY and ADD comands
if (command.type == AlterCommand::MODIFY_COLUMN || command.type == AlterCommand::ADD_COLUMN)

View File

@ -45,7 +45,8 @@ struct AlterCommand
RENAME_COLUMN,
REMOVE_TTL,
MODIFY_DATABASE_SETTING,
COMMENT_TABLE
COMMENT_TABLE,
REMOVE_SAMPLE_BY,
};
/// Which property user wants to remove from column

View File

@ -250,7 +250,7 @@ MergeTreeData::MergeTreeData(
{
/// This is for backward compatibility.
checkSampleExpression(metadata_, attach || settings->compatibility_allow_sampling_expression_not_in_primary_key,
settings->check_sample_column_is_correct);
settings->check_sample_column_is_correct && !attach);
}
checkTTLExpressions(metadata_, metadata_);

View File

@ -1015,8 +1015,15 @@ void StorageReplicatedMergeTree::setTableStructure(
if (metadata_diff.sampling_expression_changed)
{
auto sample_by_ast = parse_key_expr(metadata_diff.new_sampling_expression);
new_metadata.sampling_key.recalculateWithNewAST(sample_by_ast, new_metadata.columns, getContext());
if (!metadata_diff.new_sampling_expression.empty())
{
auto sample_by_ast = parse_key_expr(metadata_diff.new_sampling_expression);
new_metadata.sampling_key.recalculateWithNewAST(sample_by_ast, new_metadata.columns, getContext());
}
else /// SAMPLE BY was removed
{
new_metadata.sampling_key = {};
}
}
if (metadata_diff.skip_indices_changed)

View File

@ -1,6 +1,6 @@
AlterQuery t1 (children 1)
ExpressionList (children 1)
AlterCommand 32 (children 1)
AlterCommand 33 (children 1)
Function equals (children 1)
ExpressionList (children 2)
Identifier date

View File

@ -0,0 +1,3 @@
CREATE TABLE default.t_remove_sample_by\n(\n `id` UInt64\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192
CREATE TABLE default.t_remove_sample_by\n(\n `id` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/default/t_remove_sample_by\', \'1\')\nORDER BY id\nSETTINGS index_granularity = 8192
CREATE TABLE default.t_remove_sample_by\n(\n `id` String\n)\nENGINE = MergeTree\nORDER BY id\nSETTINGS index_granularity = 8192

View File

@ -0,0 +1,44 @@
-- Tags: zookeeper
DROP TABLE IF EXISTS t_remove_sample_by;
CREATE TABLE t_remove_sample_by(id UInt64) ENGINE = MergeTree ORDER BY id SAMPLE BY id;
ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY;
SHOW CREATE TABLE t_remove_sample_by;
ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY; -- { serverError 36 }
SELECT * FROM t_remove_sample_by SAMPLE 1 / 10; -- { serverError 141 }
DROP TABLE t_remove_sample_by;
CREATE TABLE t_remove_sample_by(id UInt64)
ENGINE = ReplicatedMergeTree('/clickhouse/{database}/t_remove_sample_by', '1')
ORDER BY id SAMPLE BY id;
ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY;
SHOW CREATE TABLE t_remove_sample_by;
DROP TABLE t_remove_sample_by;
CREATE TABLE t_remove_sample_by(id UInt64) ENGINE = Memory;
ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY; -- { serverError 36 }
DROP TABLE t_remove_sample_by;
CREATE TABLE t_remove_sample_by(id String)
ENGINE = MergeTree ORDER BY id SAMPLE BY id
SETTINGS check_sample_column_is_correct = 0;
ALTER TABLE t_remove_sample_by RESET SETTING check_sample_column_is_correct;
DETACH TABLE t_remove_sample_by;
ATTACH TABLE t_remove_sample_by;
INSERT INTO t_remove_sample_by VALUES (1);
SELECT * FROM t_remove_sample_by SAMPLE 1 / 10; -- { serverError 59 }
ALTER TABLE t_remove_sample_by REMOVE SAMPLE BY;
SHOW CREATE TABLE t_remove_sample_by;
DROP TABLE t_remove_sample_by;