Alter remove column properties and TTLs (#14742)

This commit is contained in:
alesapin 2020-09-20 16:27:33 +03:00 committed by GitHub
parent 1474b80ecc
commit 5ffee8808a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 570 additions and 49 deletions

View File

@ -291,6 +291,8 @@ void DatabaseOrdinary::alterTable(const Context & context, const StorageID & tab
if (metadata.table_ttl.definition_ast)
storage_ast.set(storage_ast.ttl_table, metadata.table_ttl.definition_ast);
else if (storage_ast.ttl_table != nullptr) /// TTL was removed
storage_ast.ttl_table = nullptr;
if (metadata.settings_changes)
storage_ast.set(storage_ast.settings, metadata.settings_changes);

View File

@ -222,6 +222,11 @@ AccessRightsElements InterpreterAlterQuery::getRequiredAccessForCommand(const AS
required_access.emplace_back(AccessType::ALTER_TTL, database, table);
break;
}
case ASTAlterCommand::REMOVE_TTL:
{
required_access.emplace_back(AccessType::ALTER_TTL, database, table);
break;
}
case ASTAlterCommand::MATERIALIZE_TTL:
{
required_access.emplace_back(AccessType::ALTER_MATERIALIZE_TTL, database, table);

View File

@ -99,12 +99,19 @@ void ASTAlterCommand::formatImpl(
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY COLUMN " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
col_decl->formatImpl(settings, state, frame);
if (first)
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " FIRST " << (settings.hilite ? hilite_none : "");
else if (column) /// AFTER
if (!remove_property.empty())
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " AFTER " << (settings.hilite ? hilite_none : "");
column->formatImpl(settings, state, frame);
settings.ostr << (settings.hilite ? hilite_keyword : "") << " REMOVE " << remove_property;
}
else
{
if (first)
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " FIRST " << (settings.hilite ? hilite_none : "");
else if (column) /// AFTER
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " AFTER " << (settings.hilite ? hilite_none : "");
column->formatImpl(settings, state, frame);
}
}
}
else if (type == ASTAlterCommand::COMMENT_COLUMN)
@ -280,6 +287,10 @@ void ASTAlterCommand::formatImpl(
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY TTL " << (settings.hilite ? hilite_none : "");
ttl->formatImpl(settings, state, frame);
}
else if (type == ASTAlterCommand::REMOVE_TTL)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REMOVE TTL" << (settings.hilite ? hilite_none : "");
}
else if (type == ASTAlterCommand::MATERIALIZE_TTL)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MATERIALIZE TTL"

View File

@ -36,6 +36,7 @@ public:
MATERIALIZE_TTL,
MODIFY_SETTING,
MODIFY_QUERY,
REMOVE_TTL,
ADD_INDEX,
DROP_INDEX,
@ -167,6 +168,9 @@ public:
/// Target column name
ASTPtr rename_to;
/// Which property user want to remove
String remove_property;
String getID(char delim) const override { return "AlterCommand" + (delim + std::to_string(static_cast<int>(type))); }
ASTPtr clone() const override;

View File

@ -82,12 +82,23 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ParserKeyword s_where("WHERE");
ParserKeyword s_to("TO");
ParserKeyword s_remove("REMOVE");
ParserKeyword s_default("DEFAULT");
ParserKeyword s_materialized("MATERIALIZED");
ParserKeyword s_alias("ALIAS");
ParserKeyword s_comment("COMMENT");
ParserKeyword s_codec("CODEC");
ParserKeyword s_ttl("TTL");
ParserKeyword s_remove_ttl("REMOVE TTL");
ParserCompoundIdentifier parser_name;
ParserStringLiteral parser_string_literal;
ParserIdentifier parser_remove_property;
ParserCompoundColumnDeclaration parser_col_decl;
ParserIndexDeclaration parser_idx_decl;
ParserConstraintDeclaration parser_constraint_decl;
ParserCompoundColumnDeclaration parser_modify_col_decl(false);
ParserCompoundColumnDeclaration parser_modify_col_decl(false, false, true);
ParserPartition parser_partition;
ParserExpression parser_exp_elem;
ParserList parser_assignment_list(
@ -433,14 +444,33 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
if (!parser_modify_col_decl.parse(pos, command->col_decl, expected))
return false;
if (s_first.ignore(pos, expected))
command->first = true;
else if (s_after.ignore(pos, expected))
if (s_remove.ignore(pos, expected))
{
if (!parser_name.parse(pos, command->column, expected))
if (s_default.ignore(pos, expected))
command->remove_property = "DEFAULT";
else if (s_materialized.ignore(pos, expected))
command->remove_property = "MATERIALIZED";
else if (s_alias.ignore(pos, expected))
command->remove_property = "ALIAS";
else if (s_comment.ignore(pos, expected))
command->remove_property = "COMMENT";
else if (s_codec.ignore(pos, expected))
command->remove_property = "CODEC";
else if (s_ttl.ignore(pos, expected))
command->remove_property = "TTL";
else
return false;
}
else
{
if (s_first.ignore(pos, expected))
command->first = true;
else if (s_after.ignore(pos, expected))
{
if (!parser_name.parse(pos, command->column, expected))
return false;
}
}
command->type = ASTAlterCommand::MODIFY_COLUMN;
}
else if (s_modify_order_by.ignore(pos, expected))
@ -496,6 +526,10 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
return false;
command->type = ASTAlterCommand::MODIFY_TTL;
}
else if (s_remove_ttl.ignore(pos, expected))
{
command->type = ASTAlterCommand::REMOVE_TTL;
}
else if (s_materialize_ttl.ignore(pos, expected))
{
command->type = ASTAlterCommand::MATERIALIZE_TTL;

View File

@ -90,8 +90,10 @@ template <typename NameParser>
class IParserColumnDeclaration : public IParserBase
{
public:
explicit IParserColumnDeclaration(bool require_type_ = true, bool allow_null_modifiers_ = false)
: require_type(require_type_), allow_null_modifiers(allow_null_modifiers_)
explicit IParserColumnDeclaration(bool require_type_ = true, bool allow_null_modifiers_ = false, bool check_keywords_after_name_ = false)
: require_type(require_type_)
, allow_null_modifiers(allow_null_modifiers_)
, check_keywords_after_name(check_keywords_after_name_)
{
}
@ -104,6 +106,7 @@ protected:
bool require_type = true;
bool allow_null_modifiers = false;
bool check_keywords_after_name = false;
};
using ParserColumnDeclaration = IParserColumnDeclaration<ParserIdentifier>;
@ -122,6 +125,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
ParserKeyword s_comment{"COMMENT"};
ParserKeyword s_codec{"CODEC"};
ParserKeyword s_ttl{"TTL"};
ParserKeyword s_remove{"REMOVE"};
ParserTernaryOperatorExpression expr_parser;
ParserStringLiteral string_literal_parser;
ParserCodec codec_parser;
@ -132,6 +136,24 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
if (!name_parser.parse(pos, name, expected))
return false;
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
tryGetIdentifierNameInto(name, column_declaration->name);
/// 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
/// column declaration with name only.
if (s_remove.checkWithoutMoving(pos, expected))
{
if (!check_keywords_after_name)
return false;
node = column_declaration;
return true;
}
/** column name should be followed by type name if it
* is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS, COMMENT}
*/
@ -197,9 +219,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
return false;
}
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
node = column_declaration;
tryGetIdentifierNameInto(name, column_declaration->name);
if (type)
{

View File

@ -43,6 +43,30 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
}
namespace
{
AlterCommand::RemoveProperty removePropertyFromString(const String & property)
{
if (property.empty())
return AlterCommand::RemoveProperty::NO_PROPERTY;
else if (property == "DEFAULT")
return AlterCommand::RemoveProperty::DEFAULT;
else if (property == "MATERIALIZED")
return AlterCommand::RemoveProperty::MATERIALIZED;
else if (property == "ALIAS")
return AlterCommand::RemoveProperty::ALIAS;
else if (property == "COMMENT")
return AlterCommand::RemoveProperty::COMMENT;
else if (property == "CODEC")
return AlterCommand::RemoveProperty::CODEC;
else if (property == "TTL")
return AlterCommand::RemoveProperty::TTL;
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot remove unknown property '{}'", property);
}
}
std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_ast)
{
@ -111,8 +135,9 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
command.type = AlterCommand::MODIFY_COLUMN;
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 (ast_col_decl.type)
{
command.data_type = data_type_factory.get(ast_col_decl.type);
@ -237,6 +262,13 @@ std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_
command.ttl = command_ast->ttl;
return command;
}
else if (command_ast->type == ASTAlterCommand::REMOVE_TTL)
{
AlterCommand command;
command.ast = command_ast->clone();
command.type = AlterCommand::REMOVE_TTL;
return command;
}
else if (command_ast->type == ASTAlterCommand::MODIFY_SETTING)
{
AlterCommand command;
@ -301,24 +333,45 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con
{
metadata.columns.modify(column_name, after_column, first, [&](ColumnDescription & column)
{
if (codec)
column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type ? data_type : column.type, false);
if (comment)
column.comment = *comment;
if (ttl)
column.ttl = ttl;
if (data_type)
column.type = data_type;
/// User specified default expression or changed
/// datatype. We have to replace default.
if (default_expression || data_type)
if (to_remove == RemoveProperty::DEFAULT
|| to_remove == RemoveProperty::MATERIALIZED
|| to_remove == RemoveProperty::ALIAS)
{
column.default_desc.kind = default_kind;
column.default_desc.expression = default_expression;
column.default_desc = ColumnDefault{};
}
else if (to_remove == RemoveProperty::CODEC)
{
column.codec.reset();
}
else if (to_remove == RemoveProperty::COMMENT)
{
column.comment = String{};
}
else if (to_remove == RemoveProperty::TTL)
{
column.ttl.reset();
}
else
{
if (codec)
column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type ? data_type : column.type, false);
if (comment)
column.comment = *comment;
if (ttl)
column.ttl = ttl;
if (data_type)
column.type = data_type;
/// User specified default expression or changed
/// datatype. We have to replace default.
if (default_expression || data_type)
{
column.default_desc.kind = default_kind;
column.default_desc.expression = default_expression;
}
}
});
@ -450,6 +503,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con
{
metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST(ttl, metadata.columns, context, metadata.primary_key);
}
else if (type == REMOVE_TTL)
{
metadata.table_ttl = TTLTableDescription{};
}
else if (type == MODIFY_QUERY)
{
metadata.select = SelectQueryDescription::getSelectQueryFromASTForMatView(select, context);
@ -584,6 +641,10 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada
if (ignore)
return false;
/// We remove properties on metadata level
if (isRemovingProperty() || type == REMOVE_TTL)
return false;
if (type == DROP_COLUMN || type == DROP_INDEX || type == RENAME_COLUMN)
return true;
@ -636,6 +697,11 @@ bool AlterCommand::isTTLAlter(const StorageInMemoryMetadata & metadata) const
return ttl_changed;
}
bool AlterCommand::isRemovingProperty() const
{
return to_remove != RemoveProperty::NO_PROPERTY;
}
std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(StorageInMemoryMetadata & metadata, const Context & context) const
{
if (!isRequireMutationStage(metadata))
@ -716,6 +782,8 @@ String alterTypeToString(const AlterCommand::Type type)
return "MODIFY QUERY";
case AlterCommand::Type::RENAME_COLUMN:
return "RENAME COLUMN";
case AlterCommand::Type::REMOVE_TTL:
return "REMOVE TTL";
}
__builtin_unreachable();
}
@ -783,14 +851,15 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata)
if (!has_column && command.if_exists)
command.ignore = true;
if (has_column && command.data_type)
if (has_column)
{
auto column_from_table = columns.get(command.column_name);
if (!command.default_expression && column_from_table.default_desc.expression)
if (command.data_type && !command.default_expression && column_from_table.default_desc.expression)
{
command.default_kind = column_from_table.default_desc.kind;
command.default_expression = column_from_table.default_desc.expression;
}
}
}
else if (command.type == AlterCommand::ADD_COLUMN)
@ -857,6 +926,70 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con
if (command.codec)
CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(command.codec, command.data_type, !context.getSettingsRef().allow_suspicious_codecs);
auto column_default = all_columns.getDefault(column_name);
if (column_default)
{
if (command.to_remove == AlterCommand::RemoveProperty::DEFAULT && column_default->kind != ColumnDefaultKind::Default)
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Cannot remove DEFAULT from column {}, because column default type is {}. Use REMOVE {} to delete it",
backQuote(column_name), toString(column_default->kind), toString(column_default->kind));
}
if (command.to_remove == AlterCommand::RemoveProperty::MATERIALIZED && column_default->kind != ColumnDefaultKind::Materialized)
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Cannot remove MATERIALIZED from column {}, because column default type is {}. Use REMOVE {} to delete it",
backQuote(column_name), toString(column_default->kind), toString(column_default->kind));
}
if (command.to_remove == AlterCommand::RemoveProperty::ALIAS && column_default->kind != ColumnDefaultKind::Alias)
{
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Cannot remove ALIAS from column {}, because column default type is {}. Use REMOVE {} to delete it",
backQuote(column_name), toString(column_default->kind), toString(column_default->kind));
}
}
if (command.isRemovingProperty())
{
if (!column_default && command.to_remove == AlterCommand::RemoveProperty::DEFAULT)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Column {} doesn't have DEFAULT, cannot remove it",
backQuote(column_name));
if (!column_default && command.to_remove == AlterCommand::RemoveProperty::ALIAS)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Column {} doesn't have ALIAS, cannot remove it",
backQuote(column_name));
if (!column_default && command.to_remove == AlterCommand::RemoveProperty::MATERIALIZED)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Column {} doesn't have MATERIALIZED, cannot remove it",
backQuote(column_name));
auto column_from_table = all_columns.get(column_name);
if (command.to_remove == AlterCommand::RemoveProperty::TTL && column_from_table.ttl == nullptr)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Column {} doesn't have TTL, cannot remove it",
backQuote(column_name));
if (command.to_remove == AlterCommand::RemoveProperty::CODEC && column_from_table.codec == nullptr)
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Column {} doesn't have TTL, cannot remove it",
backQuote(column_name));
if (command.to_remove == AlterCommand::RemoveProperty::COMMENT && column_from_table.comment.empty())
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"Column {} doesn't have COMMENT, cannot remove it",
backQuote(column_name));
}
modified_columns.emplace(column_name);
}
@ -966,6 +1099,10 @@ void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Con
throw Exception{"Cannot rename column from nested struct to normal column and vice versa", ErrorCodes::BAD_ARGUMENTS};
}
}
else if (command.type == AlterCommand::REMOVE_TTL && !metadata.hasAnyTableTTL())
{
throw Exception{"Table doesn't have any table TTL expression, 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

@ -37,6 +37,22 @@ struct AlterCommand
MODIFY_SETTING,
MODIFY_QUERY,
RENAME_COLUMN,
REMOVE_TTL,
};
/// Which property user wants to remove from column
enum class RemoveProperty
{
NO_PROPERTY,
/// Default specifiers
DEFAULT,
MATERIALIZED,
ALIAS,
/// Other properties
COMMENT,
CODEC,
TTL
};
Type type;
@ -107,16 +123,13 @@ struct AlterCommand
/// Target column name
String rename_to;
/// What to remove from column (or TTL)
RemoveProperty to_remove = RemoveProperty::NO_PROPERTY;
static std::optional<AlterCommand> parse(const ASTAlterCommand * command);
void apply(StorageInMemoryMetadata & metadata, const Context & context) const;
/// Checks that alter query changes data. For MergeTree:
/// * column files (data and marks)
/// * each part meta (columns.txt)
/// in each part on disk (it's not lightweight alter).
bool isModifyingData(const StorageInMemoryMetadata & metadata) const;
/// Check that alter command require data modification (mutation) to be
/// executed. For example, cast from Date to UInt16 type can be executed
/// without any data modifications. But column drop or modify from UInt16 to
@ -132,6 +145,9 @@ struct AlterCommand
/// Checks that any TTL changed by alter
bool isTTLAlter(const StorageInMemoryMetadata & metadata) const;
/// Command removing some property from column or table
bool isRemovingProperty() const;
/// If possible, convert alter command to mutation command. In other case
/// return empty optional. Some storages may execute mutations after
/// metadata changes.
@ -164,9 +180,6 @@ public:
/// Commands have to be prepared before apply.
void apply(StorageInMemoryMetadata & metadata, const Context & context) const;
/// At least one command modify data on disk.
bool isModifyingData(const StorageInMemoryMetadata & metadata) const;
/// At least one command modify settings.
bool isSettingsAlter() const;

View File

@ -900,10 +900,17 @@ ColumnsDescription new_columns, const ReplicatedMergeTreeTableMetadata::Diff & m
if (metadata_diff.ttl_table_changed)
{
ParserTTLExpressionList parser;
auto ttl_for_table_ast = parseQuery(parser, metadata_diff.new_ttl_table, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST(
ttl_for_table_ast, new_metadata.columns, global_context, new_metadata.primary_key);
if (!metadata_diff.new_ttl_table.empty())
{
ParserTTLExpressionList parser;
auto ttl_for_table_ast = parseQuery(parser, metadata_diff.new_ttl_table, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST(
ttl_for_table_ast, new_metadata.columns, global_context, new_metadata.primary_key);
}
else /// TTL was removed
{
new_metadata.table_ttl = TTLTableDescription{};
}
}
}
@ -3818,7 +3825,12 @@ void StorageReplicatedMergeTree::alter(
future_metadata_in_zk.partition_key = serializeAST(*future_metadata.partition_key.expression_list_ast);
if (ast_to_str(future_metadata.table_ttl.definition_ast) != ast_to_str(current_metadata->table_ttl.definition_ast))
future_metadata_in_zk.ttl_table = serializeAST(*future_metadata.table_ttl.definition_ast);
{
if (future_metadata.table_ttl.definition_ast)
future_metadata_in_zk.ttl_table = serializeAST(*future_metadata.table_ttl.definition_ast);
else /// TTL was removed
future_metadata_in_zk.ttl_table = "";
}
String new_indices_str = future_metadata.secondary_indices.toString();
if (new_indices_str != current_metadata->secondary_indices.toString())

View File

@ -0,0 +1,4 @@
CREATE TABLE default.no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.r_no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01493_r_no_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.r_no_prop_table\n(\n `some_column` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01493_r_no_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192

View File

@ -0,0 +1,51 @@
DROP TABLE IF EXISTS no_prop_table;
CREATE TABLE no_prop_table
(
some_column UInt64
)
ENGINE MergeTree()
ORDER BY tuple();
SHOW CREATE TABLE no_prop_table;
-- just nothing happened
ALTER TABLE no_prop_table MODIFY COLUMN some_column REMOVE DEFAULT; --{serverError 36}
ALTER TABLE no_prop_table MODIFY COLUMN some_column REMOVE MATERIALIZED; --{serverError 36}
ALTER TABLE no_prop_table MODIFY COLUMN some_column REMOVE ALIAS; --{serverError 36}
ALTER TABLE no_prop_table MODIFY COLUMN some_column REMOVE CODEC; --{serverError 36}
ALTER TABLE no_prop_table MODIFY COLUMN some_column REMOVE COMMENT; --{serverError 36}
ALTER TABLE no_prop_table MODIFY COLUMN some_column REMOVE TTL; --{serverError 36}
ALTER TABLE no_prop_table REMOVE TTL; --{serverError 36}
SHOW CREATE TABLE no_prop_table;
DROP TABLE IF EXISTS no_prop_table;
DROP TABLE IF EXISTS r_no_prop_table;
CREATE TABLE r_no_prop_table
(
some_column UInt64
)
ENGINE ReplicatedMergeTree('/clickhouse/test/01493_r_no_prop_table', '1')
ORDER BY tuple();
SHOW CREATE TABLE r_no_prop_table;
ALTER TABLE r_no_prop_table MODIFY COLUMN some_column REMOVE DEFAULT; --{serverError 36}
ALTER TABLE r_no_prop_table MODIFY COLUMN some_column REMOVE MATERIALIZED; --{serverError 36}
ALTER TABLE r_no_prop_table MODIFY COLUMN some_column REMOVE ALIAS; --{serverError 36}
ALTER TABLE r_no_prop_table MODIFY COLUMN some_column REMOVE CODEC; --{serverError 36}
ALTER TABLE r_no_prop_table MODIFY COLUMN some_column REMOVE COMMENT; --{serverError 36}
ALTER TABLE r_no_prop_table MODIFY COLUMN some_column REMOVE TTL; --{serverError 36}
ALTER TABLE r_no_prop_table REMOVE TTL; --{serverError 36}
SHOW CREATE TABLE r_no_prop_table;
ALTER TABLE r_no_prop_table MODIFY COLUMN some_column REMOVE ttl; --{serverError 36}
ALTER TABLE r_no_prop_table remove TTL; --{serverError 36}
DROP TABLE IF EXISTS r_no_prop_table;

View File

@ -0,0 +1,20 @@
CREATE TABLE default.prop_table\n(\n `column_default` UInt64 DEFAULT 42,\n `column_materialized` UInt64 MATERIALIZED column_default * 42,\n `column_alias` UInt64 ALIAS column_default + 1,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date COMMENT \'Some comment\',\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
42 1764 43 str 2019-10-01 1
CREATE TABLE default.prop_table\n(\n `column_default` UInt64 DEFAULT 42,\n `column_materialized` UInt64 MATERIALIZED column_default * 42,\n `column_alias` UInt64 ALIAS column_default + 1,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
CREATE TABLE default.prop_table\n(\n `column_default` UInt64 DEFAULT 42,\n `column_materialized` UInt64 MATERIALIZED column_default * 42,\n `column_alias` UInt64 ALIAS column_default + 1,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
42 1764 0 str 2019-10-01 1
CREATE TABLE default.prop_table\n(\n `column_default` UInt64 DEFAULT 42,\n `column_materialized` UInt64 MATERIALIZED column_default * 42,\n `column_alias` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
42 1764 0 str 2019-10-01 1
42 1764 33 trs 2020-01-01 2
CREATE TABLE default.prop_table\n(\n `column_default` UInt64 DEFAULT 42,\n `column_materialized` UInt64,\n `column_alias` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
42 1764 0 str 2019-10-01 1
42 1764 33 trs 2020-01-01 2
42 11 44 rts 2020-02-01 3
CREATE TABLE default.prop_table\n(\n `column_default` UInt64,\n `column_materialized` UInt64,\n `column_alias` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
42 1764 0 str 2019-10-01 1
42 1764 33 trs 2020-01-01 2
42 11 44 rts 2020-02-01 3
0 22 55 tsr 2020-03-01 4
CREATE TABLE default.prop_table\n(\n `column_default` UInt64,\n `column_materialized` UInt64,\n `column_alias` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.prop_table\n(\n `column_default` UInt64,\n `column_materialized` UInt64,\n `column_alias` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192
4

View File

@ -0,0 +1,72 @@
DROP TABLE IF EXISTS prop_table;
CREATE TABLE prop_table
(
column_default UInt64 DEFAULT 42,
column_materialized UInt64 MATERIALIZED column_default * 42,
column_alias UInt64 ALIAS column_default + 1,
column_codec String CODEC(ZSTD(10)),
column_comment Date COMMENT 'Some comment',
column_ttl UInt64 TTL column_comment + INTERVAL 1 MONTH
)
ENGINE MergeTree()
ORDER BY tuple()
TTL column_comment + INTERVAL 2 MONTH;
SHOW CREATE TABLE prop_table;
SYSTEM STOP TTL MERGES prop_table;
INSERT INTO prop_table (column_codec, column_comment, column_ttl) VALUES ('str', toDate('2019-10-01'), 1);
SELECT column_default, column_materialized, column_alias, column_codec, column_comment, column_ttl FROM prop_table;
ALTER TABLE prop_table MODIFY COLUMN column_comment REMOVE COMMENT;
SHOW CREATE TABLE prop_table;
ALTER TABLE prop_table MODIFY COLUMN column_codec REMOVE CODEC;
SHOW CREATE TABLE prop_table;
ALTER TABLE prop_table MODIFY COLUMN column_alias REMOVE ALIAS;
SELECT column_default, column_materialized, column_alias, column_codec, column_comment, column_ttl FROM prop_table;
SHOW CREATE TABLE prop_table;
INSERT INTO prop_table (column_alias, column_codec, column_comment, column_ttl) VALUES (33, 'trs', toDate('2020-01-01'), 2);
SELECT column_default, column_materialized, column_alias, column_codec, column_comment, column_ttl FROM prop_table ORDER BY column_ttl;
ALTER TABLE prop_table MODIFY COLUMN column_materialized REMOVE MATERIALIZED;
SHOW CREATE TABLE prop_table;
INSERT INTO prop_table (column_materialized, column_alias, column_codec, column_comment, column_ttl) VALUES (11, 44, 'rts', toDate('2020-02-01'), 3);
SELECT column_default, column_materialized, column_alias, column_codec, column_comment, column_ttl FROM prop_table ORDER BY column_ttl;
ALTER TABLE prop_table MODIFY COLUMN column_default REMOVE DEFAULT;
SHOW CREATE TABLE prop_table;
INSERT INTO prop_table (column_materialized, column_alias, column_codec, column_comment, column_ttl) VALUES (22, 55, 'tsr', toDate('2020-03-01'), 4);
SELECT column_default, column_materialized, column_alias, column_codec, column_comment, column_ttl FROM prop_table ORDER BY column_ttl;
ALTER TABLE prop_table REMOVE TTL;
SHOW CREATE TABLE prop_table;
ALTER TABLE prop_table MODIFY COLUMN column_ttl REMOVE TTL;
SHOW CREATE TABLE prop_table;
SYSTEM START TTL MERGES prop_table;
OPTIMIZE TABLE prop_table FINAL;
SELECT COUNT() FROM prop_table;
DROP TABLE IF EXISTS prop_table;

View File

@ -0,0 +1,21 @@
CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date COMMENT \'Some comment\',\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date COMMENT \'Some comment\',\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
====== remove column comment ======
CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String CODEC(ZSTD(10)),\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
====== remove column codec ======
CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64 DEFAULT 42,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
====== remove column default ======
42 str 1
0 tsr 2
CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64 TTL column_comment + toIntervalMonth(1)\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
====== remove column TTL ======
CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nTTL column_comment + toIntervalMonth(2)\nSETTINGS index_granularity = 8192
====== remove table TTL ======
CREATE TABLE default.r_prop_table1\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'1\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default.r_prop_table2\n(\n `column_default` UInt64,\n `column_codec` String,\n `column_comment` Date,\n `column_ttl` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test_01493/r_prop_table\', \'2\')\nORDER BY tuple()\nSETTINGS index_granularity = 8192
3
3

View File

@ -0,0 +1,92 @@
DROP TABLE IF EXISTS r_prop_table1;
DROP TABLE IF EXISTS r_prop_table2;
SET replication_alter_partitions_sync = 2;
CREATE TABLE r_prop_table1
(
column_default UInt64 DEFAULT 42,
column_codec String CODEC(ZSTD(10)),
column_comment Date COMMENT 'Some comment',
column_ttl UInt64 TTL column_comment + INTERVAL 1 MONTH
)
ENGINE ReplicatedMergeTree('/clickhouse/test_01493/r_prop_table', '1')
ORDER BY tuple()
TTL column_comment + INTERVAL 2 MONTH;
CREATE TABLE r_prop_table2
(
column_default UInt64 DEFAULT 42,
column_codec String CODEC(ZSTD(10)),
column_comment Date COMMENT 'Some comment',
column_ttl UInt64 TTL column_comment + INTERVAL 1 MONTH
)
ENGINE ReplicatedMergeTree('/clickhouse/test_01493/r_prop_table', '2')
ORDER BY tuple()
TTL column_comment + INTERVAL 2 MONTH;
SHOW CREATE TABLE r_prop_table1;
SHOW CREATE TABLE r_prop_table2;
INSERT INTO r_prop_table1 (column_codec, column_comment, column_ttl) VALUES ('str', toDate('2020-10-01'), 1);
SYSTEM SYNC REPLICA r_prop_table2;
SELECT '====== remove column comment ======';
ALTER TABLE r_prop_table1 MODIFY COLUMN column_comment REMOVE COMMENT;
SHOW CREATE TABLE r_prop_table1;
SHOW CREATE TABLE r_prop_table2;
DETACH TABLE r_prop_table1;
ATTACH TABLE r_prop_table1;
SELECT '====== remove column codec ======';
ALTER TABLE r_prop_table2 MODIFY COLUMN column_codec REMOVE CODEC;
SHOW CREATE TABLE r_prop_table1;
SHOW CREATE TABLE r_prop_table2;
SELECT '====== remove column default ======';
ALTER TABLE r_prop_table2 MODIFY COLUMN column_default REMOVE DEFAULT;
INSERT INTO r_prop_table1 (column_codec, column_comment, column_ttl) VALUES ('tsr', now(), 2);
SYSTEM SYNC REPLICA r_prop_table2;
SELECT column_default, column_codec, column_ttl FROM r_prop_table1 ORDER BY column_ttl;
DETACH TABLE r_prop_table2;
ATTACH TABLE r_prop_table2;
SHOW CREATE TABLE r_prop_table1;
SHOW CREATE TABLE r_prop_table2;
SELECT '====== remove column TTL ======';
ALTER TABLE r_prop_table2 MODIFY COLUMN column_ttl REMOVE TTL;
SHOW CREATE TABLE r_prop_table1;
SHOW CREATE TABLE r_prop_table2;
SELECT '====== remove table TTL ======';
ALTER TABLE r_prop_table1 REMOVE TTL;
INSERT INTO r_prop_table1 (column_codec, column_comment, column_ttl) VALUES ('rts', now() - INTERVAL 1 YEAR, 3);
SYSTEM SYNC REPLICA r_prop_table2;
DETACH TABLE r_prop_table2;
ATTACH TABLE r_prop_table2;
SHOW CREATE TABLE r_prop_table1;
SHOW CREATE TABLE r_prop_table2;
OPTIMIZE TABLE r_prop_table2 FINAL;
SYSTEM SYNC REPLICA r_prop_table1;
SELECT COUNT() FROM r_prop_table1;
SELECT COUNT() FROM r_prop_table2;
DROP TABLE IF EXISTS r_prop_table1;
DROP TABLE IF EXISTS r_prop_table2;

View File

@ -0,0 +1 @@
CREATE TABLE default.default_table\n(\n `key` UInt64 DEFAULT 42,\n `value1` UInt64 MATERIALIZED key * key,\n `value2` UInt64 ALIAS value1 * key\n)\nENGINE = MergeTree()\nORDER BY tuple()\nSETTINGS index_granularity = 8192

View File

@ -0,0 +1,22 @@
DROP TABLE IF EXISTS default_table;
CREATE TABLE default_table (
key UInt64 DEFAULT 42,
value1 UInt64 MATERIALIZED key * key,
value2 ALIAS value1 * key
)
ENGINE = MergeTree()
ORDER BY tuple();
ALTER TABLE default_table MODIFY COLUMN key REMOVE MATERIALIZED; --{serverError 36}
ALTER TABLE default_table MODIFY COLUMN key REMOVE ALIAS; --{serverError 36}
ALTER TABLE default_table MODIFY COLUMN value1 REMOVE DEFAULT; --{serverError 36}
ALTER TABLE default_table MODIFY COLUMN value1 REMOVE ALIAS; --{serverError 36}
ALTER TABLE default_table MODIFY COLUMN value2 REMOVE DEFAULT; --{serverError 36}
ALTER TABLE default_table MODIFY COLUMN value2 REMOVE MATERIALIZED; --{serverError 36}
SHOW CREATE TABLE default_table;
DROP TABLE IF EXISTS default_table;