mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Add default columns alter
This commit is contained in:
parent
93907f0985
commit
5b133dd1ce
@ -12,6 +12,7 @@
|
||||
#include <Interpreters/addTypeConversionToAST.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/RenameColumnVisitor.h>
|
||||
#include <Parsers/ASTAlterQuery.h>
|
||||
#include <Parsers/ASTColumnDeclaration.h>
|
||||
#include <Parsers/ASTConstraintDeclaration.h>
|
||||
@ -451,6 +452,19 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata) const
|
||||
else if (type == RENAME_COLUMN)
|
||||
{
|
||||
metadata.columns.rename(column_name, rename_to);
|
||||
RenameColumnData rename_data{column_name, rename_to};
|
||||
RenameColumnVisitor rename_visitor(rename_data);
|
||||
for (auto & column : metadata.columns)
|
||||
{
|
||||
metadata.columns.modify(column.name, [&](ColumnDescription & column_to_modify) {
|
||||
if (column_to_modify.default_desc.expression)
|
||||
rename_visitor.visit(column_to_modify.default_desc.expression);
|
||||
if (column_to_modify.ttl)
|
||||
rename_visitor.visit(column_to_modify.ttl);
|
||||
});
|
||||
}
|
||||
if (metadata.ttl_for_table_ast)
|
||||
rename_visitor.visit(metadata.ttl_for_table_ast);
|
||||
}
|
||||
else
|
||||
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
|
||||
|
@ -192,7 +192,7 @@ MergeTreeData::MergeTreeData(
|
||||
min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING;
|
||||
}
|
||||
|
||||
setTTLExpressions(metadata.columns.getColumnTTLs(), metadata.ttl_for_table_ast);
|
||||
setTTLExpressions(metadata.columns, metadata.ttl_for_table_ast);
|
||||
|
||||
/// format_file always contained on any data path
|
||||
PathWithDisk version_file;
|
||||
@ -610,14 +610,17 @@ void checkTTLExpression(const ExpressionActionsPtr & ttl_expression, const Strin
|
||||
}
|
||||
|
||||
|
||||
void MergeTreeData::setTTLExpressions(const ColumnsDescription::ColumnTTLs & new_column_ttls,
|
||||
void MergeTreeData::setTTLExpressions(const ColumnsDescription & columns,
|
||||
const ASTPtr & new_ttl_table_ast, bool only_check)
|
||||
{
|
||||
auto create_ttl_entry = [this](ASTPtr ttl_ast)
|
||||
|
||||
auto new_column_ttls = columns.getColumnTTLs();
|
||||
|
||||
auto create_ttl_entry = [this, &columns](ASTPtr ttl_ast)
|
||||
{
|
||||
TTLEntry result;
|
||||
|
||||
auto syntax_result = SyntaxAnalyzer(global_context).analyze(ttl_ast, getColumns().getAllPhysical());
|
||||
auto syntax_result = SyntaxAnalyzer(global_context).analyze(ttl_ast, columns.getAllPhysical());
|
||||
result.expression = ExpressionAnalyzer(ttl_ast, syntax_result, global_context).getActions(false);
|
||||
result.destination_type = PartDestinationType::DELETE;
|
||||
result.result_column = ttl_ast->getColumnName();
|
||||
@ -1500,7 +1503,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
|
||||
|
||||
setProperties(metadata, /* only_check = */ true);
|
||||
|
||||
setTTLExpressions(metadata.columns.getColumnTTLs(), metadata.ttl_for_table_ast, /* only_check = */ true);
|
||||
setTTLExpressions(metadata.columns, metadata.ttl_for_table_ast, /* only_check = */ true);
|
||||
|
||||
if (settings_ast)
|
||||
{
|
||||
|
@ -868,8 +868,9 @@ protected:
|
||||
|
||||
void initPartitionKey();
|
||||
|
||||
void setTTLExpressions(const ColumnsDescription::ColumnTTLs & new_column_ttls,
|
||||
void setTTLExpressions(const ColumnsDescription & columns,
|
||||
const ASTPtr & new_ttl_table_ast, bool only_check = false);
|
||||
|
||||
void checkStoragePolicy(const StoragePolicyPtr & new_storage_policy);
|
||||
|
||||
void setStoragePolicy(const String & new_storage_policy_name, bool only_check = false);
|
||||
|
@ -237,7 +237,7 @@ void StorageMergeTree::alter(
|
||||
/// Reinitialize primary key because primary key column types might have changed.
|
||||
setProperties(metadata);
|
||||
|
||||
setTTLExpressions(metadata.columns.getColumnTTLs(), metadata.ttl_for_table_ast);
|
||||
setTTLExpressions(metadata.columns, metadata.ttl_for_table_ast);
|
||||
|
||||
DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id.table_name, metadata);
|
||||
|
||||
|
@ -496,7 +496,7 @@ void StorageReplicatedMergeTree::setTableStructure(ColumnsDescription new_column
|
||||
/// Even if the primary/sorting keys didn't change we must reinitialize it
|
||||
/// because primary key column types might have changed.
|
||||
setProperties(metadata);
|
||||
setTTLExpressions(new_columns.getColumnTTLs(), metadata.ttl_for_table_ast);
|
||||
setTTLExpressions(new_columns, metadata.ttl_for_table_ast);
|
||||
}
|
||||
|
||||
|
||||
|
@ -0,0 +1,17 @@
|
||||
date key value1 value2
|
||||
2019-10-02 1 1 Hello 1
|
||||
CREATE TABLE default.table_rename_with_default (`date` Date, `key` UInt64, `value1` String, `value2` String DEFAULT concat(\'Hello \', value1), `value3` String ALIAS concat(\'Word \', value1)) ENGINE = MergeTree() PARTITION BY date ORDER BY key SETTINGS index_granularity = 8192
|
||||
date key renamed_value1 value2
|
||||
2019-10-02 1 1 Hello 1
|
||||
CREATE TABLE default.table_rename_with_default (`date` Date, `key` UInt64, `renamed_value1` String, `value2` String DEFAULT concat(\'Hello \', renamed_value1), `value3` String ALIAS concat(\'Word \', renamed_value1)) ENGINE = MergeTree() PARTITION BY date ORDER BY key SETTINGS index_granularity = 8192
|
||||
Hello 1
|
||||
Word 1
|
||||
date1 date2 value1 value2
|
||||
2019-10-02 2018-10-02 1 1
|
||||
CREATE TABLE default.table_rename_with_ttl (`date1` Date, `date2` Date, `value1` String, `value2` String TTL date1 + toIntervalMonth(10000)) ENGINE = ReplicatedMergeTree(\'/clickhouse/test/table_rename_with_ttl\', \'1\') ORDER BY tuple() TTL date2 + toIntervalMonth(10000) SETTINGS index_granularity = 8192
|
||||
renamed_date1 date2 value1 value2
|
||||
2019-10-02 2018-10-02 1 1
|
||||
CREATE TABLE default.table_rename_with_ttl (`renamed_date1` Date, `date2` Date, `value1` String, `value2` String TTL renamed_date1 + toIntervalMonth(10000)) ENGINE = ReplicatedMergeTree(\'/clickhouse/test/table_rename_with_ttl\', \'1\') ORDER BY tuple() TTL date2 + toIntervalMonth(10000) SETTINGS index_granularity = 8192
|
||||
renamed_date1 renamed_date2 value1 value2
|
||||
2019-10-02 2018-10-02 1 1
|
||||
CREATE TABLE default.table_rename_with_ttl (`renamed_date1` Date, `renamed_date2` Date, `value1` String, `value2` String TTL renamed_date1 + toIntervalMonth(10000)) ENGINE = ReplicatedMergeTree(\'/clickhouse/test/table_rename_with_ttl\', \'1\') ORDER BY tuple() TTL renamed_date2 + toIntervalMonth(10000) SETTINGS index_granularity = 8192
|
@ -0,0 +1,63 @@
|
||||
DROP TABLE IF EXISTS table_rename_with_default;
|
||||
|
||||
CREATE TABLE table_rename_with_default
|
||||
(
|
||||
date Date,
|
||||
key UInt64,
|
||||
value1 String,
|
||||
value2 String DEFAULT concat('Hello ', value1),
|
||||
value3 String ALIAS concat('Word ', value1)
|
||||
)
|
||||
ENGINE = MergeTree()
|
||||
PARTITION BY date
|
||||
ORDER BY key;
|
||||
|
||||
INSERT INTO table_rename_with_default (date, key, value1) SELECT toDate('2019-10-01') + number % 3, number, toString(number) from numbers(9);
|
||||
|
||||
SELECT * FROM table_rename_with_default WHERE key = 1 FORMAT TSVWithNames;
|
||||
|
||||
SHOW CREATE TABLE table_rename_with_default;
|
||||
|
||||
ALTER TABLE table_rename_with_default RENAME COLUMN value1 TO renamed_value1;
|
||||
|
||||
SELECT * FROM table_rename_with_default WHERE key = 1 FORMAT TSVWithNames;
|
||||
|
||||
SHOW CREATE TABLE table_rename_with_default;
|
||||
|
||||
SELECT value2 FROM table_rename_with_default WHERE key = 1;
|
||||
SELECT value3 FROM table_rename_with_default WHERE key = 1;
|
||||
|
||||
DROP TABLE IF EXISTS table_rename_with_default;
|
||||
|
||||
DROP TABLE IF EXISTS table_rename_with_ttl;
|
||||
|
||||
CREATE TABLE table_rename_with_ttl
|
||||
(
|
||||
date1 Date,
|
||||
date2 Date,
|
||||
value1 String,
|
||||
value2 String TTL date1 + INTERVAL 10000 MONTH
|
||||
)
|
||||
ENGINE = ReplicatedMergeTree('/clickhouse/test/table_rename_with_ttl', '1')
|
||||
ORDER BY tuple()
|
||||
TTL date2 + INTERVAL 10000 MONTH;
|
||||
|
||||
INSERT INTO table_rename_with_ttl SELECT toDate('2019-10-01') + number % 3, toDate('2018-10-01') + number % 3, toString(number), toString(number) from numbers(9);
|
||||
|
||||
SELECT * FROM table_rename_with_ttl WHERE value1 = '1' FORMAT TSVWithNames;
|
||||
|
||||
SHOW CREATE TABLE table_rename_with_ttl;
|
||||
|
||||
ALTER TABLE table_rename_with_ttl RENAME COLUMN date1 TO renamed_date1;
|
||||
|
||||
SELECT * FROM table_rename_with_ttl WHERE value1 = '1' FORMAT TSVWithNames;
|
||||
|
||||
SHOW CREATE TABLE table_rename_with_ttl;
|
||||
|
||||
ALTER TABLE table_rename_with_ttl RENAME COLUMN date2 TO renamed_date2;
|
||||
|
||||
SELECT * FROM table_rename_with_ttl WHERE value1 = '1' FORMAT TSVWithNames;
|
||||
|
||||
SHOW CREATE TABLE table_rename_with_ttl;
|
||||
|
||||
DROP TABLE IF EXISTS table_rename_with_ttl;
|
Loading…
Reference in New Issue
Block a user