mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Revert accident changes
This commit is contained in:
parent
c0dafb0283
commit
2c4047b280
@ -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)
|
||||
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 : "") << " 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
|
||||
{
|
||||
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)
|
||||
|
@ -1,17 +0,0 @@
|
||||
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
|
@ -1,58 +0,0 @@
|
||||
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;
|
Loading…
Reference in New Issue
Block a user