提交 2c4047b2 编写于 作者: A alesapin

Revert accident changes

上级 c0dafb02
......@@ -99,42 +99,12 @@ 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 (to_remove != RemoveProperty::NO_PROPERTY)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " REMOVE ";
switch (to_remove)
{
case RemoveProperty::DEFAULT:
settings.ostr << "DEFAULT";
break;
case RemoveProperty::MATERIALIZED:
settings.ostr << "MATERIALIZED";
break;
case RemoveProperty::ALIAS:
settings.ostr << "ALIAS";
break;
case RemoveProperty::COMMENT:
settings.ostr << "COMMENT";
break;
case RemoveProperty::CODEC:
settings.ostr << "CODEC";
break;
case RemoveProperty::TTL:
settings.ostr << "TTL";
break;
default:
__builtin_unreachable();
}
}
else
if (first)
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " FIRST " << (settings.hilite ? hilite_none : "");
else if (column) /// AFTER
{
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);
}
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << " AFTER " << (settings.hilite ? hilite_none : "");
column->formatImpl(settings, state, frame);
}
}
else if (type == ASTAlterCommand::COMMENT_COLUMN)
......@@ -308,14 +278,7 @@ void ASTAlterCommand::formatImpl(
else if (type == ASTAlterCommand::MODIFY_TTL)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY TTL " << (settings.hilite ? hilite_none : "");
if (ttl)
{
ttl->formatImpl(settings, state, frame);
}
else if (to_remove == RemoveProperty::TTL)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str<< " REMOVE " << (settings.hilite ? hilite_none : "");
}
ttl->formatImpl(settings, state, frame);
}
else if (type == ASTAlterCommand::MATERIALIZE_TTL)
{
......
......@@ -9,22 +9,6 @@
namespace DB
{
/// Which property user wants to remove from column
enum class RemoveProperty
{
NO_PROPERTY,
/// Default specifiers
DEFAULT,
MATERIALIZED,
ALIAS,
/// Other properties
COMMENT,
CODEC,
TTL
};
/** ALTER query:
* ALTER TABLE [db.]name_type
* ADD COLUMN col_name type [AFTER col_after],
......@@ -183,8 +167,6 @@ public:
/// Target column name
ASTPtr rename_to;
RemoveProperty to_remove = RemoveProperty::NO_PROPERTY;
String getID(char delim) const override { return "AlterCommand" + (delim + std::to_string(static_cast<int>(type))); }
ASTPtr clone() const override;
......
......@@ -82,14 +82,6 @@ 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");
ParserCompoundIdentifier parser_name;
ParserStringLiteral parser_string_literal;
ParserCompoundColumnDeclaration parser_col_decl;
......@@ -438,46 +430,17 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
if (s_if_exists.ignore(pos, expected))
command->if_exists = true;
ASTPtr column_name;
Pos stop_pos = pos;
if (!parser_name.parse(pos, column_name, expected))
if (!parser_modify_col_decl.parse(pos, command->col_decl, expected))
return false;
if (s_remove.ignore(pos, expected))
if (s_first.ignore(pos, expected))
command->first = true;
else if (s_after.ignore(pos, expected))
{
if (s_default.ignore(pos, expected))
command->to_remove = RemoveProperty::DEFAULT;
else if (s_materialized.ignore(pos, expected))
command->to_remove = RemoveProperty::MATERIALIZED;
else if (s_alias.ignore(pos, expected))
command->to_remove = RemoveProperty::ALIAS;
else if (s_comment.ignore(pos, expected))
command->to_remove = RemoveProperty::COMMENT;
else if (s_codec.ignore(pos, expected))
command->to_remove = RemoveProperty::CODEC;
else if (s_ttl.ignore(pos, expected))
command->to_remove = RemoveProperty::TTL;
else
if (!parser_name.parse(pos, command->column, expected))
return false;
auto column_declaration = std::make_shared<ASTColumnDeclaration>();
tryGetIdentifierNameInto(column_name, column_declaration->name);
command->col_decl = column_declaration;
}
else
{
pos = stop_pos;
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 (!parser_name.parse(pos, command->column, expected))
return false;
}
}
command->type = ASTAlterCommand::MODIFY_COLUMN;
}
else if (s_modify_order_by.ignore(pos, expected))
......@@ -529,9 +492,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
}
else if (s_modify_ttl.ignore(pos, expected))
{
if (s_remove.ignore(pos, expected))
command->to_remove = RemoveProperty::TTL;
else if (!parser_ttl_list.parse(pos, command->ttl, expected))
if (!parser_ttl_list.parse(pos, command->ttl, expected))
return false;
command->type = ASTAlterCommand::MODIFY_TTL;
}
......
......@@ -111,9 +111,8 @@ 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 = command_ast->to_remove;
command.column_name = ast_col_decl.name;
if (ast_col_decl.type)
{
command.data_type = data_type_factory.get(ast_col_decl.type);
......@@ -302,45 +301,24 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con
{
metadata.columns.modify(column_name, after_column, first, [&](ColumnDescription & column)
{
if (to_remove == RemoveProperty::DEFAULT
|| to_remove == RemoveProperty::MATERIALIZED
|| to_remove == RemoveProperty::ALIAS)
{
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 (codec)
column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(codec, data_type ? data_type : column.type, false);
if (comment)
column.comment = *comment;
if (comment)
column.comment = *comment;
if (ttl)
column.ttl = ttl;
if (ttl)
column.ttl = ttl;
if (data_type)
column.type = data_type;
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;
}
/// 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;
}
});
......@@ -470,10 +448,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con
}
else if (type == MODIFY_TTL)
{
if (to_remove == RemoveProperty::TTL)
metadata.table_ttl = TTLTableDescription{};
else
metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST(ttl, metadata.columns, context, metadata.primary_key);
metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST(ttl, metadata.columns, context, metadata.primary_key);
}
else if (type == MODIFY_QUERY)
{
......@@ -615,10 +590,6 @@ bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metada
if (type != MODIFY_COLUMN || data_type == nullptr)
return false;
/// We remove properties on metadata level
if (type == MODIFY_COLUMN && to_remove != RemoveProperty::NO_PROPERTY)
return false;
for (const auto & column : metadata.columns.getAllPhysical())
{
if (column.name == column_name && !isMetadataOnlyConversion(column.type.get(), data_type.get()))
......@@ -812,30 +783,14 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata)
if (!has_column && command.if_exists)
command.ignore = true;
if (has_column)
if (has_column && command.data_type)
{
auto column_from_table = columns.get(command.column_name);
if (command.to_remove != RemoveProperty::NO_PROPERTY)
{
auto column_default = columns.getDefault(command.column_name);
if (!column_default
&& (command.to_remove == RemoveProperty::ALIAS || command.to_remove == RemoveProperty::DEFAULT
|| command.to_remove == RemoveProperty::MATERIALIZED))
command.ignore = true;
if (command.to_remove == RemoveProperty::TTL && column_from_table.ttl == nullptr)
command.ignore = true;
if (command.to_remove == RemoveProperty::COMMENT && column_from_table.comment.empty())
command.ignore = true;
if (command.to_remove == RemoveProperty::CODEC && column_from_table.codec == nullptr)
command.ignore = true;
}
else if (command.data_type && !command.default_expression && column_from_table.default_desc.expression)
if (!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)
......@@ -850,11 +805,6 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata)
if (!has_column && command.if_exists)
command.ignore = true;
}
else if (command.type == AlterCommand::MODIFY_TTL)
{
if (!metadata.hasAnyTTL())
command.ignore = true;
}
}
prepared = true;
}
......@@ -907,34 +857,6 @@ 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 == 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 == 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 == 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)
};
}
}
modified_columns.emplace(column_name);
}
......@@ -1126,7 +1048,7 @@ MutationCommands AlterCommands::getMutationCommands(StorageInMemoryMetadata meta
{
for (const auto & alter_cmd : *this)
{
if (alter_cmd.isTTLAlter(metadata) && alter_cmd.to_remove != RemoveProperty::TTL)
if (alter_cmd.isTTLAlter(metadata))
{
result.push_back(createMaterializeTTLCommand());
break;
......
......@@ -107,13 +107,16 @@ struct AlterCommand
/// Target column name
String rename_to;
/// What to remove from column (or TTL)
RemoveProperty to_remove;
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
......@@ -161,6 +164,9 @@ 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;
......
......@@ -298,7 +298,7 @@ def test_mixed_granularity_single_node(start_dynamic_cluster, node):
#still works
assert node.query("SELECT count() from table_with_default_granularity") == '6\n'
@pytest.mark.skip(reason="flaky")
def test_version_update_two_nodes(start_dynamic_cluster):
node11.query("INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)")
node12.query("SYSTEM SYNC REPLICA table_with_default_granularity", timeout=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
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;
DROP TABLE IF EXISTS prop_table;
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册