mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #30180 from CurtizJ/remove-sample-by
Allow to remove `SAMPLE BY` expression
This commit is contained in:
commit
b1a74af44d
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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 " : "")
|
||||
|
@ -41,6 +41,7 @@ public:
|
||||
RESET_SETTING,
|
||||
MODIFY_QUERY,
|
||||
REMOVE_TTL,
|
||||
REMOVE_SAMPLE_BY,
|
||||
|
||||
ADD_INDEX,
|
||||
DROP_INDEX,
|
||||
|
@ -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))
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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_);
|
||||
|
@ -1014,10 +1014,17 @@ void StorageReplicatedMergeTree::setTableStructure(
|
||||
}
|
||||
|
||||
if (metadata_diff.sampling_expression_changed)
|
||||
{
|
||||
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)
|
||||
new_metadata.secondary_indices = IndicesDescription::parse(metadata_diff.new_skip_indices, new_columns, getContext());
|
||||
|
@ -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
|
||||
|
@ -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
|
44
tests/queries/0_stateless/02097_remove_sample_by.sql
Normal file
44
tests/queries/0_stateless/02097_remove_sample_by.sql
Normal 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;
|
Loading…
Reference in New Issue
Block a user