ClickHouse/dbms/src/Storages/AlterCommands.cpp

616 lines
23 KiB
C++
Raw Normal View History

#include <Storages/AlterCommands.h>
#include <Storages/IStorage.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/NestedUtils.h>
#include <Interpreters/Context.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Parsers/ASTIdentifier.h>
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
#include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTColumnDeclaration.h>
2019-07-24 12:56:39 +00:00
#include <Parsers/ASTSetQuery.h>
#include <Common/typeid_cast.h>
2018-12-21 14:40:20 +00:00
#include <Compression/CompressionFactory.h>
#include <Parsers/queryToString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
2019-07-24 12:56:39 +00:00
extern const int UNKNOWN_SETTING;
}
std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_ast)
{
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
2018-12-21 14:40:20 +00:00
const CompressionCodecFactory & compression_codec_factory = CompressionCodecFactory::instance();
if (command_ast->type == ASTAlterCommand::ADD_COLUMN)
{
AlterCommand command;
command.type = AlterCommand::ADD_COLUMN;
const auto & ast_col_decl = command_ast->col_decl->as<ASTColumnDeclaration &>();
command.column_name = ast_col_decl.name;
if (ast_col_decl.type)
{
command.data_type = data_type_factory.get(ast_col_decl.type);
}
if (ast_col_decl.default_expression)
{
command.default_kind = columnDefaultKindFromString(ast_col_decl.default_specifier);
command.default_expression = ast_col_decl.default_expression;
}
2019-03-15 14:59:03 +00:00
if (ast_col_decl.comment)
{
const auto & ast_comment = typeid_cast<ASTLiteral &>(*ast_col_decl.comment);
command.comment = ast_comment.value.get<String>();
}
2018-12-21 14:40:20 +00:00
if (ast_col_decl.codec)
command.codec = compression_codec_factory.get(ast_col_decl.codec, command.data_type);
2018-12-21 14:40:20 +00:00
if (command_ast->column)
2019-08-08 20:02:30 +00:00
command.after_column = getIdentifierName(command_ast->column);
if (ast_col_decl.ttl)
command.ttl = ast_col_decl.ttl;
command.if_not_exists = command_ast->if_not_exists;
return command;
}
else if (command_ast->type == ASTAlterCommand::DROP_COLUMN && !command_ast->partition)
{
if (command_ast->clear_column)
throw Exception("\"ALTER TABLE table CLEAR COLUMN column\" queries are not supported yet. Use \"CLEAR COLUMN column IN PARTITION\".", ErrorCodes::NOT_IMPLEMENTED);
AlterCommand command;
command.type = AlterCommand::DROP_COLUMN;
2019-08-08 20:02:30 +00:00
command.column_name = getIdentifierName(command_ast->column);
command.if_exists = command_ast->if_exists;
return command;
}
else if (command_ast->type == ASTAlterCommand::MODIFY_COLUMN)
{
AlterCommand command;
command.type = AlterCommand::MODIFY_COLUMN;
const auto & ast_col_decl = command_ast->col_decl->as<ASTColumnDeclaration &>();
command.column_name = ast_col_decl.name;
if (ast_col_decl.type)
{
command.data_type = data_type_factory.get(ast_col_decl.type);
}
if (ast_col_decl.default_expression)
{
command.default_kind = columnDefaultKindFromString(ast_col_decl.default_specifier);
command.default_expression = ast_col_decl.default_expression;
}
2018-11-14 22:46:39 +00:00
if (ast_col_decl.comment)
{
const auto & ast_comment = ast_col_decl.comment->as<ASTLiteral &>();
2018-11-14 22:46:39 +00:00
command.comment = ast_comment.value.get<String>();
}
2019-03-15 14:59:03 +00:00
if (ast_col_decl.ttl)
command.ttl = ast_col_decl.ttl;
2019-03-15 14:59:03 +00:00
if (ast_col_decl.codec)
command.codec = compression_codec_factory.get(ast_col_decl.codec, command.data_type);
2019-03-15 14:59:03 +00:00
command.if_exists = command_ast->if_exists;
2018-11-14 22:46:39 +00:00
return command;
}
else if (command_ast->type == ASTAlterCommand::COMMENT_COLUMN)
{
AlterCommand command;
command.type = COMMENT_COLUMN;
2019-08-08 20:02:30 +00:00
command.column_name = getIdentifierName(command_ast->column);
const auto & ast_comment = command_ast->comment->as<ASTLiteral &>();
command.comment = ast_comment.value.get<String>();
command.if_exists = command_ast->if_exists;
return command;
}
else if (command_ast->type == ASTAlterCommand::MODIFY_ORDER_BY)
{
AlterCommand command;
command.type = AlterCommand::MODIFY_ORDER_BY;
command.order_by = command_ast->order_by;
return command;
}
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
else if (command_ast->type == ASTAlterCommand::ADD_INDEX)
{
AlterCommand command;
command.index_decl = command_ast->index_decl;
command.type = AlterCommand::ADD_INDEX;
const auto & ast_index_decl = command_ast->index_decl->as<ASTIndexDeclaration &>();
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
command.index_name = ast_index_decl.name;
if (command_ast->index)
command.after_index_name = command_ast->index->as<ASTIdentifier &>().name;
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
command.if_not_exists = command_ast->if_not_exists;
return command;
}
2019-05-09 17:04:05 +00:00
else if (command_ast->type == ASTAlterCommand::DROP_INDEX && !command_ast->partition)
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
{
if (command_ast->clear_column)
2019-05-09 17:04:05 +00:00
throw Exception("\"ALTER TABLE table CLEAR INDEX index\" queries are not supported yet. Use \"CLEAR INDEX index IN PARTITION\".", ErrorCodes::NOT_IMPLEMENTED);
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
AlterCommand command;
command.type = AlterCommand::DROP_INDEX;
command.index_name = command_ast->index->as<ASTIdentifier &>().name;
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
command.if_exists = command_ast->if_exists;
return command;
}
else if (command_ast->type == ASTAlterCommand::MODIFY_TTL)
{
AlterCommand command;
command.type = AlterCommand::MODIFY_TTL;
command.ttl = command_ast->ttl;
return command;
}
2019-07-24 12:56:39 +00:00
else if (command_ast->type == ASTAlterCommand::MODIFY_SETTING)
{
AlterCommand command;
command.type = AlterCommand::MODIFY_SETTING;
command.settings_changes = command_ast->settings_changes->as<ASTSetQuery &>().changes;
return command;
}
else
return {};
}
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
void AlterCommand::apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
2019-07-24 12:56:39 +00:00
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast, SettingsChanges & changes) const
{
if (type == ADD_COLUMN)
{
2019-05-21 11:24:32 +00:00
ColumnDescription column(column_name, data_type, false);
if (default_expression)
{
column.default_desc.kind = default_kind;
column.default_desc.expression = default_expression;
}
column.comment = comment;
column.codec = codec;
column.ttl = ttl;
columns_description.add(column, after_column);
2018-12-21 14:40:20 +00:00
/// Slow, because each time a list is copied
columns_description.flattenNested();
}
else if (type == DROP_COLUMN)
{
columns_description.remove(column_name);
}
else if (type == MODIFY_COLUMN)
{
columns_description.modify(column_name, [&](ColumnDescription & column)
{
if (codec)
{
/// User doesn't specify data type, it means that datatype doesn't change
/// let's use info about old type
if (data_type == nullptr)
codec->useInfoAboutType(column.type);
column.codec = codec;
}
2019-05-02 23:56:42 +00:00
if (!isMutable())
{
column.comment = comment;
return;
}
if (ttl)
column.ttl = ttl;
column.type = data_type;
column.default_desc.kind = default_kind;
column.default_desc.expression = default_expression;
});
}
else if (type == MODIFY_ORDER_BY)
{
if (!primary_key_ast && order_by_ast)
{
/// Primary and sorting key become independent after this ALTER so we have to
/// save the old ORDER BY expression as the new primary key.
primary_key_ast = order_by_ast->clone();
}
order_by_ast = order_by;
}
else if (type == COMMENT_COLUMN)
{
columns_description.modify(column_name, [&](ColumnDescription & column) { column.comment = comment; });
}
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
else if (type == ADD_INDEX)
{
if (std::any_of(
indices_description.indices.cbegin(),
indices_description.indices.cend(),
[this](const ASTPtr & index_ast)
{
return index_ast->as<ASTIndexDeclaration &>().name == index_name;
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
}))
{
if (if_not_exists)
return;
else
throw Exception{"Cannot add index " + index_name + ": index with this name already exists",
ErrorCodes::ILLEGAL_COLUMN};
}
auto insert_it = indices_description.indices.end();
if (!after_index_name.empty())
{
insert_it = std::find_if(
indices_description.indices.begin(),
indices_description.indices.end(),
[this](const ASTPtr & index_ast)
{
return index_ast->as<ASTIndexDeclaration &>().name == after_index_name;
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
});
if (insert_it == indices_description.indices.end())
throw Exception("Wrong index name. Cannot find index " + backQuote(after_index_name) + " to insert after.",
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
ErrorCodes::LOGICAL_ERROR);
++insert_it;
}
indices_description.indices.emplace(insert_it, std::dynamic_pointer_cast<ASTIndexDeclaration>(index_decl));
}
else if (type == DROP_INDEX)
{
auto erase_it = std::find_if(
indices_description.indices.begin(),
indices_description.indices.end(),
[this](const ASTPtr & index_ast)
{
return index_ast->as<ASTIndexDeclaration &>().name == index_name;
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
});
if (erase_it == indices_description.indices.end())
{
if (if_exists)
return;
throw Exception("Wrong index name. Cannot find index " + backQuote(index_name) + " to drop.",
ErrorCodes::LOGICAL_ERROR);
}
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
indices_description.indices.erase(erase_it);
}
else if (type == MODIFY_TTL)
{
ttl_table_ast = ttl;
}
2019-07-24 12:56:39 +00:00
else if (type == MODIFY_SETTING)
{
2019-07-24 15:22:16 +00:00
changes.insert(changes.end(), settings_changes.begin(), settings_changes.end());
2019-07-24 12:56:39 +00:00
}
else
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
}
2019-05-02 23:56:42 +00:00
bool AlterCommand::isMutable() const
{
2019-07-24 12:56:39 +00:00
if (type == COMMENT_COLUMN || type == MODIFY_SETTING)
return false;
if (type == MODIFY_COLUMN)
return data_type.get() || default_expression;
return true;
}
2019-08-06 12:52:08 +00:00
bool AlterCommand::isSettingsAlter() const
{
return type == MODIFY_SETTING;
}
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
void AlterCommands::apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
2019-07-24 12:56:39 +00:00
ASTPtr & order_by_ast, ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast, SettingsChanges & changes) const
{
auto new_columns_description = columns_description;
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
auto new_indices_description = indices_description;
auto new_order_by_ast = order_by_ast;
auto new_primary_key_ast = primary_key_ast;
auto new_ttl_table_ast = ttl_table_ast;
2019-07-24 12:56:39 +00:00
auto new_changes = changes;
for (const AlterCommand & command : *this)
if (!command.ignore)
2019-07-24 15:22:16 +00:00
command.apply(new_columns_description, new_indices_description, new_order_by_ast, new_primary_key_ast, new_ttl_table_ast, new_changes);
2019-05-02 23:56:42 +00:00
columns_description = std::move(new_columns_description);
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
indices_description = std::move(new_indices_description);
order_by_ast = std::move(new_order_by_ast);
primary_key_ast = std::move(new_primary_key_ast);
ttl_table_ast = std::move(new_ttl_table_ast);
2019-07-24 12:56:39 +00:00
changes = std::move(new_changes);
}
void AlterCommands::validate(const IStorage & table, const Context & context)
{
/// A temporary object that is used to keep track of the current state of columns after applying a subset of commands.
auto columns = table.getColumns();
/// Default expressions will be added to this list for type deduction.
auto default_expr_list = std::make_shared<ASTExpressionList>();
/// We will save ALTER ADD/MODIFY command indices (only the last for each column) for possible modification
/// (we might need to add deduced types or modify default expressions).
/// Saving indices because we can add new commands later and thus cause vector resize.
std::unordered_map<String, size_t> column_to_command_idx;
for (size_t i = 0; i < size(); ++i)
{
auto & command = (*this)[i];
if (command.type == AlterCommand::ADD_COLUMN || command.type == AlterCommand::MODIFY_COLUMN)
{
const auto & column_name = command.column_name;
if (command.type == AlterCommand::ADD_COLUMN)
{
if (columns.has(column_name) || columns.hasNested(column_name))
{
if (command.if_not_exists)
command.ignore = true;
else
throw Exception{"Cannot add column " + column_name + ": column with this name already exists", ErrorCodes::ILLEGAL_COLUMN};
}
}
else if (command.type == AlterCommand::MODIFY_COLUMN)
{
if (!columns.has(column_name))
{
if (command.if_exists)
command.ignore = true;
else
throw Exception{"Wrong column name. Cannot find column " + column_name + " to modify", ErrorCodes::ILLEGAL_COLUMN};
}
if (!command.ignore)
columns.remove(column_name);
}
if (!command.ignore)
{
column_to_command_idx[column_name] = i;
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions
2019-05-21 11:24:32 +00:00
columns.add(
ColumnDescription(column_name, command.data_type ? command.data_type : std::make_shared<DataTypeUInt8>(), false));
if (command.default_expression)
{
if (command.data_type)
{
const auto & final_column_name = column_name;
const auto tmp_column_name = final_column_name + "_tmp";
default_expr_list->children.emplace_back(setAlias(
makeASTFunction("CAST", std::make_shared<ASTIdentifier>(tmp_column_name),
std::make_shared<ASTLiteral>(command.data_type->getName())),
final_column_name));
default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), tmp_column_name));
}
else
{
/// no type explicitly specified, will deduce later
default_expr_list->children.emplace_back(
setAlias(command.default_expression->clone(), column_name));
}
}
}
}
else if (command.type == AlterCommand::DROP_COLUMN)
{
if (columns.has(command.column_name) || columns.hasNested(command.column_name))
2018-03-02 19:32:30 +00:00
{
for (const ColumnDescription & column : columns)
{
const auto & default_expression = column.default_desc.expression;
if (!default_expression)
continue;
ASTPtr query = default_expression->clone();
auto syntax_result = SyntaxAnalyzer(context).analyze(query, columns.getAll());
const auto actions = ExpressionAnalyzer(query, syntax_result, context).getActions(true);
const auto required_columns = actions->getRequiredColumns();
if (required_columns.end() != std::find(required_columns.begin(), required_columns.end(), command.column_name))
throw Exception(
"Cannot drop column " + command.column_name + ", because column " + column.name +
" depends on it", ErrorCodes::ILLEGAL_COLUMN);
}
columns.remove(command.column_name);
}
else if (command.if_exists)
command.ignore = true;
else
throw Exception("Wrong column name. Cannot find column " + command.column_name + " to drop",
ErrorCodes::ILLEGAL_COLUMN);
}
else if (command.type == AlterCommand::COMMENT_COLUMN)
{
if (!columns.has(command.column_name))
{
if (command.if_exists)
command.ignore = true;
else
throw Exception{"Wrong column name. Cannot find column " + command.column_name + " to comment", ErrorCodes::ILLEGAL_COLUMN};
}
}
2019-07-24 12:56:39 +00:00
else if (command.type == AlterCommand::MODIFY_SETTING)
{
for (const auto & change : command.settings_changes)
{
if (!table.hasSetting(change.name))
{
throw Exception{"Storage '" + table.getName() + "' doesn't have setting '" + change.name + "'", ErrorCodes::UNKNOWN_SETTING};
}
}
}
}
/** Existing defaulted columns may require default expression extensions with a type conversion,
* therefore we add them to default_expr_list to recalculate their types */
for (const auto & column : columns)
{
if (column.default_desc.expression)
{
const auto tmp_column_name = column.name + "_tmp";
default_expr_list->children.emplace_back(setAlias(
makeASTFunction("CAST", std::make_shared<ASTIdentifier>(tmp_column_name),
std::make_shared<ASTLiteral>(column.type->getName())),
column.name));
default_expr_list->children.emplace_back(setAlias(column.default_desc.expression->clone(), tmp_column_name));
}
}
ASTPtr query = default_expr_list;
auto syntax_result = SyntaxAnalyzer(context).analyze(query, columns.getAll());
const auto actions = ExpressionAnalyzer(query, syntax_result, context).getActions(true);
const auto block = actions->getSampleBlock();
/// set deduced types, modify default expression if necessary
for (const auto & column : columns)
{
AlterCommand * command = nullptr;
auto command_it = column_to_command_idx.find(column.name);
if (command_it != column_to_command_idx.end())
command = &(*this)[command_it->second];
if (!(command && command->default_expression) && !column.default_desc.expression)
continue;
const DataTypePtr & explicit_type = command ? command->data_type : column.type;
if (explicit_type)
{
const auto & tmp_column = block.getByName(column.name + "_tmp");
const auto & deduced_type = tmp_column.type;
if (!explicit_type->equals(*deduced_type))
{
if (!command)
{
/// column has no associated alter command, let's create it
/// add a new alter command to modify existing column
this->emplace_back(AlterCommand{AlterCommand::MODIFY_COLUMN,
2019-05-02 23:56:42 +00:00
column.name, explicit_type, column.default_desc.kind, column.default_desc.expression, {}, {}, {}, {}});
command = &back();
}
command->default_expression = makeASTFunction("CAST",
command->default_expression->clone(),
std::make_shared<ASTLiteral>(explicit_type->getName()));
}
}
else
{
/// just set deduced type
command->data_type = block.getByName(column.name).type;
}
}
}
2019-07-24 12:56:39 +00:00
void AlterCommands::applyForColumnsOnly(ColumnsDescription & columns_description) const
{
auto out_columns_description = columns_description;
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
IndicesDescription indices_description;
ASTPtr out_order_by;
ASTPtr out_primary_key;
ASTPtr out_ttl_table;
2019-07-24 12:56:39 +00:00
SettingsChanges out_changes;
apply(out_columns_description, indices_description, out_order_by, out_primary_key, out_ttl_table, out_changes);
if (out_order_by)
throw Exception("Storage doesn't support modifying ORDER BY expression", ErrorCodes::NOT_IMPLEMENTED);
if (out_primary_key)
throw Exception("Storage doesn't support modifying PRIMARY KEY expression", ErrorCodes::NOT_IMPLEMENTED);
Data Skipping Indices (#4143) * made index parser * added index parsing * some fixes * added index interface and factory * fixed compilation * ptrs * added indexParts * indextypes * index condition * IndexCondition * added indexes in selectexecutor * fix * changed comment * fix * added granularity * comments * fix * fix * added writing indexes * removed indexpart class * fix * added setSkipIndexes * add rw for MergeTreeIndexes * fixes * upd error * fix * fix * reading * test index * fixed nullptr error * fixed * fix * unique names * asts -> exprlist * minmax index * fix * fixed select * fixed merging * fixed mutation * working minmax * removed test index * fixed style * added indexes to checkDataPart * added tests for minmax index * fixed constructor * fix style * fixed includes * fixed setSkipIndexes * added indexes meta to zookeeper * added parsing * removed throw * alter cmds parse * fix * added alter * fix * alters fix * fix alters * fix "after" * fixed alter * alter fix + test * fixes * upd setSkipIndexes * fixed alter bug with drop all indices * fix metadata editing * new test and repl fix * rm test files * fixed repl alter * fix * fix * indices * MTReadStream * upd test for bug * fix * added useful parsers and ast classes * fix * fix comments * replaced columns * fix * fixed parsing * fixed printing * fix err * basic IndicesDescription * go to IndicesDescr * moved indices * go to indicesDescr * fix test minmax_index* * fixed MT alter * fixed bug with replMT indices storing in zk * rename * refactoring * docs ru * docs ru * docs en * refactor * rename tests * fix docs * refactoring * fix * fix * fix * fixed style * unique idx * unique * fix * better minmax calculation * upd * added getBlock * unique_condition * added termForAST * unique * fixed not * uniqueCondition::mayBeTrueOnGranule * fix * fixed bug with double column * is always true * fix * key set * spaces * test * tests * fix * unique * fix * fix * fixed bug with duplicate column * removed unused data * fix * fixes * __bitSwapLastTwo * fix
2019-02-05 14:50:25 +00:00
if (!indices_description.indices.empty())
throw Exception("Storage doesn't support modifying indices", ErrorCodes::NOT_IMPLEMENTED);
if (out_ttl_table)
throw Exception("Storage doesn't support modifying TTL expression", ErrorCodes::NOT_IMPLEMENTED);
2019-07-24 12:56:39 +00:00
if (!out_changes.empty())
throw Exception("Storage doesn't support modifying settings", ErrorCodes::NOT_IMPLEMENTED);
columns_description = std::move(out_columns_description);
}
2019-08-06 12:52:08 +00:00
void AlterCommands::applyForSettingsOnly(SettingsChanges & changes) const
{
ColumnsDescription out_columns_description;
IndicesDescription indices_description;
ASTPtr out_order_by;
ASTPtr out_primary_key;
ASTPtr out_ttl_table;
SettingsChanges out_changes;
apply(out_columns_description, indices_description, out_order_by, out_primary_key, out_ttl_table, out_changes);
if (out_columns_description.begin() != out_columns_description.end())
throw Exception("Alter modifying columns, but only settings change applied.", ErrorCodes::LOGICAL_ERROR);
if (out_order_by)
throw Exception("Alter modifying ORDER BY expression, but only settings change applied.", ErrorCodes::LOGICAL_ERROR);
if (out_primary_key)
throw Exception("Alter modifying PRIMARY KEY expression, but only settings change applied.", ErrorCodes::LOGICAL_ERROR);
if (!indices_description.indices.empty())
throw Exception("Alter modifying indices, but only settings change applied.", ErrorCodes::NOT_IMPLEMENTED);
if (out_ttl_table)
throw Exception("Alter modifying TTL, but only settings change applied.", ErrorCodes::NOT_IMPLEMENTED);
changes = std::move(out_changes);
}
2019-05-02 23:56:42 +00:00
bool AlterCommands::isMutable() const
2018-11-15 13:12:27 +00:00
{
for (const auto & param : *this)
{
2019-05-02 23:56:42 +00:00
if (param.isMutable())
2018-11-15 13:12:27 +00:00
return true;
}
return false;
}
2019-08-06 12:52:08 +00:00
bool AlterCommands::isSettingsAlter() const
{
return std::all_of(begin(), end(), [](const AlterCommand & c) { return c.isSettingsAlter(); });
}
}