ClickHouse/src/Storages/AlterCommands.cpp

1001 lines
35 KiB
C++
Raw Normal View History

2020-01-13 16:39:20 +00:00
#include <Compression/CompressionFactory.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeFactory.h>
2020-01-13 16:39:20 +00:00
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/NestedUtils.h>
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/addTypeConversionToAST.h>
2020-01-13 16:39:20 +00:00
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/SyntaxAnalyzer.h>
2020-04-02 16:11:10 +00:00
#include <Interpreters/RenameColumnVisitor.h>
2020-02-19 12:52:27 +00:00
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/ASTConstraintDeclaration.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.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/ASTLiteral.h>
2019-07-24 12:56:39 +00:00
#include <Parsers/ASTSetQuery.h>
2020-01-13 16:39:20 +00:00
#include <Storages/AlterCommands.h>
#include <Storages/IStorage.h>
#include <Common/typeid_cast.h>
2020-01-13 16:39:20 +00:00
#include <Parsers/queryToString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int BAD_ARGUMENTS;
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
extern const int LOGICAL_ERROR;
extern const int DUPLICATE_COLUMN;
2020-03-31 16:18:18 +00:00
extern const int NOT_IMPLEMENTED;
}
2020-05-04 00:11:49 +00:00
std::optional<AlterCommand> AlterCommand::parse(const ASTAlterCommand * command_ast, bool sanity_check_compression_codecs)
{
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;
2020-01-15 13:00:08 +00:00
command.ast = command_ast->clone();
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)
2020-05-04 00:11:49 +00:00
command.codec = compression_codec_factory.get(ast_col_decl.codec, command.data_type, sanity_check_compression_codecs);
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;
}
2020-03-17 13:49:50 +00:00
else if (command_ast->type == ASTAlterCommand::DROP_COLUMN)
{
AlterCommand command;
2020-01-15 13:00:08 +00:00
command.ast = command_ast->clone();
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;
2020-03-17 13:49:50 +00:00
if (command_ast->clear_column)
command.clear = true;
if (command_ast->partition)
command.partition = command_ast->partition;
return command;
}
else if (command_ast->type == ASTAlterCommand::MODIFY_COLUMN)
{
AlterCommand command;
2020-01-15 13:00:08 +00:00
command.ast = command_ast->clone();
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 &>();
command.comment.emplace(ast_comment.value.get<String>());
2018-11-14 22:46:39 +00:00
}
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)
2020-05-04 00:11:49 +00:00
command.codec = compression_codec_factory.get(ast_col_decl.codec, command.data_type, sanity_check_compression_codecs);
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;
2020-01-15 13:00:08 +00:00
command.ast = command_ast->clone();
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;
2020-01-15 13:00:08 +00:00
command.ast = command_ast->clone();
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;
2020-01-15 13:00:08 +00:00
command.ast = command_ast->clone();
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_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;
}
else if (command_ast->type == ASTAlterCommand::ADD_CONSTRAINT)
{
AlterCommand command;
2020-01-15 13:00:08 +00:00
command.ast = command_ast->clone();
command.constraint_decl = command_ast->constraint_decl;
command.type = AlterCommand::ADD_CONSTRAINT;
const auto & ast_constraint_decl = command_ast->constraint_decl->as<ASTConstraintDeclaration &>();
command.constraint_name = ast_constraint_decl.name;
command.if_not_exists = command_ast->if_not_exists;
return command;
}
2020-03-22 00:56:10 +00:00
else if (command_ast->type == ASTAlterCommand::DROP_CONSTRAINT)
{
AlterCommand command;
2020-01-15 13:00:08 +00:00
command.ast = command_ast->clone();
command.if_exists = command_ast->if_exists;
command.type = AlterCommand::DROP_CONSTRAINT;
command.constraint_name = command_ast->constraint->as<ASTIdentifier &>().name;
return command;
}
2020-03-17 13:49:50 +00:00
else if (command_ast->type == ASTAlterCommand::DROP_INDEX)
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;
2020-01-15 13:00:08 +00:00
command.ast = command_ast->clone();
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.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;
2020-03-17 13:49:50 +00:00
if (command_ast->clear_index)
command.clear = true;
if (command_ast->partition)
command.partition = 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
return command;
}
else if (command_ast->type == ASTAlterCommand::MODIFY_TTL)
{
AlterCommand command;
2020-01-15 13:00:08 +00:00
command.ast = command_ast->clone();
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;
2020-01-15 13:00:08 +00:00
command.ast = command_ast->clone();
2019-07-24 12:56:39 +00:00
command.type = AlterCommand::MODIFY_SETTING;
command.settings_changes = command_ast->settings_changes->as<ASTSetQuery &>().changes;
return command;
}
else if (command_ast->type == ASTAlterCommand::MODIFY_QUERY)
{
AlterCommand command;
2020-03-24 17:05:38 +00:00
command.ast = command_ast->clone();
command.type = AlterCommand::MODIFY_QUERY;
command.select = command_ast->select;
return command;
}
2020-03-24 17:05:38 +00:00
else if (command_ast->type == ASTAlterCommand::RENAME_COLUMN)
{
AlterCommand command;
command.ast = command_ast->clone();
command.type = AlterCommand::RENAME_COLUMN;
command.column_name = command_ast->column->as<ASTIdentifier &>().name;
command.rename_to = command_ast->rename_to->as<ASTIdentifier &>().name;
2020-04-03 16:45:49 +00:00
command.if_exists = command_ast->if_exists;
2020-03-24 17:05:38 +00:00
return command;
}
else
return {};
}
2019-12-26 18:17:05 +00:00
void AlterCommand::apply(StorageInMemoryMetadata & metadata) const
{
if (type == ADD_COLUMN)
{
2020-04-27 13:55:30 +00:00
ColumnDescription column(column_name, data_type);
if (default_expression)
{
column.default_desc.kind = default_kind;
column.default_desc.expression = default_expression;
}
2019-12-24 20:03:33 +00:00
if (comment)
column.comment = *comment;
column.codec = codec;
column.ttl = ttl;
2019-12-26 18:17:05 +00:00
metadata.columns.add(column, after_column);
2018-12-21 14:40:20 +00:00
/// Slow, because each time a list is copied
2019-12-26 18:17:05 +00:00
metadata.columns.flattenNested();
}
else if (type == DROP_COLUMN)
{
2020-03-17 13:49:50 +00:00
/// Otherwise just clear data on disk
if (!clear && !partition)
metadata.columns.remove(column_name);
}
else if (type == MODIFY_COLUMN)
{
2019-12-26 18:17:05 +00:00
metadata.columns.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);
2020-01-10 15:26:40 +00:00
else /// use info about new DataType
codec->useInfoAboutType(data_type);
column.codec = codec;
}
if (comment)
column.comment = *comment;
if (ttl)
column.ttl = ttl;
2019-12-24 18:07:51 +00:00
if (data_type)
column.type = data_type;
2019-12-24 18:07:51 +00:00
/// 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;
}
});
}
else if (type == MODIFY_ORDER_BY)
{
2019-12-27 14:46:11 +00:00
if (!metadata.primary_key_ast && metadata.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.
2019-12-27 14:46:11 +00:00
metadata.primary_key_ast = metadata.order_by_ast->clone();
}
2019-12-27 14:46:11 +00:00
metadata.order_by_ast = order_by;
}
else if (type == COMMENT_COLUMN)
{
2019-12-26 18:17:05 +00:00
metadata.columns.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(
2019-12-26 18:17:05 +00:00
metadata.indices.indices.cbegin(),
metadata.indices.indices.cend(),
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
[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};
}
2019-12-26 18:17:05 +00:00
auto insert_it = metadata.indices.indices.end();
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 (!after_index_name.empty())
{
insert_it = std::find_if(
2019-12-26 18:17:05 +00:00
metadata.indices.indices.begin(),
metadata.indices.indices.end(),
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
[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
});
2019-12-26 18:17:05 +00:00
if (insert_it == metadata.indices.indices.end())
throw Exception("Wrong index name. Cannot find index " + backQuote(after_index_name) + " to insert after.",
ErrorCodes::BAD_ARGUMENTS);
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
++insert_it;
}
2019-12-26 18:17:05 +00:00
metadata.indices.indices.emplace(insert_it, std::dynamic_pointer_cast<ASTIndexDeclaration>(index_decl));
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 == DROP_INDEX)
{
2020-03-17 13:49:50 +00:00
if (!partition && !clear)
{
2020-03-17 13:49:50 +00:00
auto erase_it = std::find_if(
metadata.indices.indices.begin(),
metadata.indices.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
2020-03-17 13:49:50 +00:00
if (erase_it == metadata.indices.indices.end())
{
if (if_exists)
return;
throw Exception("Wrong index name. Cannot find index " + backQuote(index_name) + " to drop.", ErrorCodes::BAD_ARGUMENTS);
}
metadata.indices.indices.erase(erase_it);
}
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_CONSTRAINT)
{
if (std::any_of(
2019-12-26 18:17:05 +00:00
metadata.constraints.constraints.cbegin(),
metadata.constraints.constraints.cend(),
[this](const ASTPtr & constraint_ast)
{
return constraint_ast->as<ASTConstraintDeclaration &>().name == constraint_name;
}))
{
if (if_not_exists)
return;
throw Exception("Cannot add constraint " + constraint_name + ": constraint with this name already exists",
ErrorCodes::ILLEGAL_COLUMN);
}
2019-12-26 18:17:05 +00:00
auto insert_it = metadata.constraints.constraints.end();
2019-12-26 18:17:05 +00:00
metadata.constraints.constraints.emplace(insert_it, std::dynamic_pointer_cast<ASTConstraintDeclaration>(constraint_decl));
}
else if (type == DROP_CONSTRAINT)
{
auto erase_it = std::find_if(
2019-12-26 18:17:05 +00:00
metadata.constraints.constraints.begin(),
metadata.constraints.constraints.end(),
[this](const ASTPtr & constraint_ast)
{
return constraint_ast->as<ASTConstraintDeclaration &>().name == constraint_name;
});
2019-12-26 18:17:05 +00:00
if (erase_it == metadata.constraints.constraints.end())
2019-06-02 15:08:28 +00:00
{
if (if_exists)
return;
throw Exception("Wrong constraint name. Cannot find constraint `" + constraint_name + "` to drop.",
ErrorCodes::BAD_ARGUMENTS);
}
2019-12-26 18:17:05 +00:00
metadata.constraints.constraints.erase(erase_it);
}
else if (type == MODIFY_TTL)
{
2019-12-27 14:46:11 +00:00
metadata.ttl_for_table_ast = ttl;
}
else if (type == MODIFY_QUERY)
{
metadata.select = select;
}
2019-07-24 12:56:39 +00:00
else if (type == MODIFY_SETTING)
{
2019-12-27 14:36:59 +00:00
auto & settings_from_storage = metadata.settings_ast->as<ASTSetQuery &>().changes;
2019-12-26 18:17:05 +00:00
for (const auto & change : settings_changes)
{
auto finder = [&change](const SettingChange & c) { return c.name == change.name; };
2019-12-29 11:25:26 +00:00
auto it = std::find_if(settings_from_storage.begin(), settings_from_storage.end(), finder);
if (it != settings_from_storage.end())
2019-12-26 18:17:05 +00:00
it->value = change.value;
else
2019-12-27 14:36:59 +00:00
settings_from_storage.push_back(change);
2019-12-26 18:17:05 +00:00
}
2019-07-24 12:56:39 +00:00
}
2020-03-24 17:05:38 +00:00
else if (type == RENAME_COLUMN)
{
metadata.columns.rename(column_name, rename_to);
2020-04-02 16:11:10 +00:00
RenameColumnData rename_data{column_name, rename_to};
RenameColumnVisitor rename_visitor(rename_data);
2020-04-22 06:22:14 +00:00
for (const auto & column : metadata.columns)
2020-04-02 16:11:10 +00:00
{
2020-04-02 18:33:17 +00:00
metadata.columns.modify(column.name, [&](ColumnDescription & column_to_modify)
{
2020-04-02 16:11:10 +00:00
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);
2020-05-12 11:26:44 +00:00
for (auto & constraint : metadata.constraints.constraints)
rename_visitor.visit(constraint);
2020-03-24 17:05:38 +00:00
}
else
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
}
bool AlterCommand::isModifyingData(const StorageInMemoryMetadata & metadata) const
{
2019-12-24 18:07:51 +00:00
/// Possible change data representation on disk
if (type == MODIFY_COLUMN)
{
if (data_type == nullptr)
return false;
2020-04-20 19:54:08 +00:00
/// It is allowed to ALTER data type to the same type as before.
for (const auto & column : metadata.columns.getAllPhysical())
if (column.name == column_name)
2020-04-20 19:53:19 +00:00
return !column.type->equals(*data_type);
return true;
}
2019-12-24 18:07:51 +00:00
return type == ADD_COLUMN /// We need to change columns.txt in each part for MergeTree
|| type == DROP_COLUMN /// We need to change columns.txt in each part for MergeTree
|| type == DROP_INDEX; /// We need to remove file from filesystem for MergeTree
}
2019-08-06 12:52:08 +00:00
bool AlterCommand::isSettingsAlter() const
{
return type == MODIFY_SETTING;
}
2020-01-13 16:39:20 +00:00
namespace
{
/// If true, then in order to ALTER the type of the column from the type from to the type to
/// we don't need to rewrite the data, we only need to update metadata and columns.txt in part directories.
/// The function works for Arrays and Nullables of the same structure.
bool isMetadataOnlyConversion(const IDataType * from, const IDataType * to)
{
2020-04-20 19:53:19 +00:00
if (from->equals(*to))
2020-01-13 16:39:20 +00:00
return true;
static const std::unordered_multimap<std::type_index, const std::type_info &> ALLOWED_CONVERSIONS =
{
{ typeid(DataTypeEnum8), typeid(DataTypeEnum8) },
{ typeid(DataTypeEnum8), typeid(DataTypeInt8) },
{ typeid(DataTypeEnum16), typeid(DataTypeEnum16) },
{ typeid(DataTypeEnum16), typeid(DataTypeInt16) },
{ typeid(DataTypeDateTime), typeid(DataTypeUInt32) },
{ typeid(DataTypeUInt32), typeid(DataTypeDateTime) },
{ typeid(DataTypeDate), typeid(DataTypeUInt16) },
{ typeid(DataTypeUInt16), typeid(DataTypeDate) },
};
while (true)
{
auto it_range = ALLOWED_CONVERSIONS.equal_range(typeid(*from));
for (auto it = it_range.first; it != it_range.second; ++it)
{
if (it->second == typeid(*to))
return true;
}
const auto * arr_from = typeid_cast<const DataTypeArray *>(from);
const auto * arr_to = typeid_cast<const DataTypeArray *>(to);
if (arr_from && arr_to)
{
from = arr_from->getNestedType().get();
to = arr_to->getNestedType().get();
continue;
}
const auto * nullable_from = typeid_cast<const DataTypeNullable *>(from);
const auto * nullable_to = typeid_cast<const DataTypeNullable *>(to);
if (nullable_from && nullable_to)
{
from = nullable_from->getNestedType().get();
to = nullable_to->getNestedType().get();
continue;
}
return false;
}
}
}
bool AlterCommand::isRequireMutationStage(const StorageInMemoryMetadata & metadata) const
{
2020-01-15 13:00:08 +00:00
if (ignore)
return false;
2020-03-30 12:51:05 +00:00
if (type == DROP_COLUMN || type == DROP_INDEX || type == RENAME_COLUMN)
2020-01-15 13:00:08 +00:00
return true;
2020-01-13 16:39:20 +00:00
if (type != MODIFY_COLUMN || data_type == nullptr)
return false;
for (const auto & column : metadata.columns.getAllPhysical())
{
2020-01-15 13:00:08 +00:00
if (column.name == column_name && !isMetadataOnlyConversion(column.type.get(), data_type.get()))
2020-01-13 16:39:20 +00:00
return true;
}
return false;
}
2019-12-27 14:36:59 +00:00
bool AlterCommand::isCommentAlter() const
{
if (type == COMMENT_COLUMN)
{
return true;
}
else if (type == MODIFY_COLUMN)
{
return comment.has_value()
&& codec == nullptr
&& data_type == nullptr
&& default_expression == nullptr
&& ttl == nullptr;
}
return false;
}
2020-05-19 01:53:01 +00:00
bool AlterCommand::isTTLAlter(const StorageInMemoryMetadata & metadata) const
{
if (type == MODIFY_TTL)
return true;
if (!ttl || type != MODIFY_COLUMN)
return false;
bool ttl_changed = true;
for (const auto & [name, ttl_ast] : metadata.columns.getColumnTTLs())
{
if (name == column_name && queryToString(*ttl) == queryToString(*ttl_ast))
{
ttl_changed = false;
break;
}
}
return ttl_changed;
}
2020-05-19 09:54:56 +00:00
std::optional<MutationCommand> AlterCommand::tryConvertToMutationCommand(StorageInMemoryMetadata & metadata) const
2020-01-13 16:39:20 +00:00
{
if (!isRequireMutationStage(metadata))
return {};
MutationCommand result;
2020-01-17 13:54:22 +00:00
if (type == MODIFY_COLUMN)
{
result.type = MutationCommand::Type::READ_COLUMN;
result.column_name = column_name;
result.data_type = data_type;
result.predicate = nullptr;
}
else if (type == DROP_COLUMN)
{
result.type = MutationCommand::Type::DROP_COLUMN;
result.column_name = column_name;
2020-03-17 13:49:50 +00:00
if (clear)
result.clear = true;
if (partition)
result.partition = partition;
2020-01-17 13:54:22 +00:00
result.predicate = nullptr;
}
else if (type == DROP_INDEX)
{
result.type = MutationCommand::Type::DROP_INDEX;
2020-03-17 13:49:50 +00:00
result.column_name = index_name;
if (clear)
result.clear = true;
if (partition)
result.partition = partition;
2020-01-17 13:54:22 +00:00
result.predicate = nullptr;
}
2020-03-30 12:51:05 +00:00
else if (type == RENAME_COLUMN)
{
result.type = MutationCommand::Type::RENAME_COLUMN;
result.column_name = column_name;
result.rename_to = rename_to;
}
2020-01-17 13:54:22 +00:00
2020-01-15 13:00:08 +00:00
result.ast = ast->clone();
2020-05-19 09:54:56 +00:00
apply(metadata);
2020-01-13 16:39:20 +00:00
return result;
}
2019-12-26 18:17:05 +00:00
String alterTypeToString(const AlterCommand::Type type)
{
switch (type)
{
case AlterCommand::Type::ADD_COLUMN:
return "ADD COLUMN";
case AlterCommand::Type::ADD_CONSTRAINT:
return "ADD CONSTRAINT";
case AlterCommand::Type::ADD_INDEX:
return "ADD INDEX";
case AlterCommand::Type::COMMENT_COLUMN:
return "COMMENT COLUMN";
case AlterCommand::Type::DROP_COLUMN:
return "DROP COLUMN";
case AlterCommand::Type::DROP_CONSTRAINT:
return "DROP CONSTRAINT";
case AlterCommand::Type::DROP_INDEX:
return "DROP INDEX";
case AlterCommand::Type::MODIFY_COLUMN:
return "MODIFY COLUMN";
case AlterCommand::Type::MODIFY_ORDER_BY:
return "MODIFY ORDER BY";
case AlterCommand::Type::MODIFY_TTL:
return "MODIFY TTL";
case AlterCommand::Type::MODIFY_SETTING:
return "MODIFY SETTING";
case AlterCommand::Type::MODIFY_QUERY:
return "MODIFY QUERY";
2020-03-24 17:05:38 +00:00
case AlterCommand::Type::RENAME_COLUMN:
return "RENAME COLUMN";
2019-12-26 18:17:05 +00:00
}
__builtin_unreachable();
}
void AlterCommands::apply(StorageInMemoryMetadata & metadata) const
{
2019-12-26 18:17:05 +00:00
if (!prepared)
throw DB::Exception("Alter commands is not prepared. Cannot apply. It's a bug", ErrorCodes::LOGICAL_ERROR);
2019-12-26 18:17:05 +00:00
auto metadata_copy = metadata;
for (const AlterCommand & command : *this)
if (!command.ignore)
2019-12-26 18:17:05 +00:00
command.apply(metadata_copy);
metadata = std::move(metadata_copy);
}
2019-12-26 18:17:05 +00:00
void AlterCommands::prepare(const StorageInMemoryMetadata & metadata)
{
2019-12-26 18:17:05 +00:00
auto columns = metadata.columns;
for (size_t i = 0; i < size(); ++i)
{
auto & command = (*this)[i];
2020-02-19 14:39:01 +00:00
bool has_column = columns.has(command.column_name) || columns.hasNested(command.column_name);
2020-02-19 12:52:27 +00:00
if (command.type == AlterCommand::MODIFY_COLUMN)
{
2020-02-19 12:52:27 +00:00
if (!has_column && command.if_exists)
command.ignore = true;
2020-02-20 09:22:17 +00:00
if (has_column && command.data_type)
{
2020-02-19 12:52:27 +00:00
auto column_from_table = columns.get(command.column_name);
if (!command.default_expression && column_from_table.default_desc.expression)
{
2020-02-19 12:52:27 +00:00
command.default_kind = column_from_table.default_desc.kind;
command.default_expression = column_from_table.default_desc.expression;
}
}
}
2020-02-19 14:39:01 +00:00
else if (command.type == AlterCommand::ADD_COLUMN)
{
if (has_column && command.if_not_exists)
command.ignore = true;
}
else if (command.type == AlterCommand::DROP_COLUMN
2020-04-03 16:45:49 +00:00
|| command.type == AlterCommand::COMMENT_COLUMN
|| command.type == AlterCommand::RENAME_COLUMN)
{
2020-02-19 12:52:27 +00:00
if (!has_column && command.if_exists)
command.ignore = true;
}
}
2019-12-26 18:17:05 +00:00
prepared = true;
}
2019-12-26 18:17:05 +00:00
void AlterCommands::validate(const StorageInMemoryMetadata & metadata, const Context & context) const
{
2020-02-19 14:39:01 +00:00
auto all_columns = metadata.columns;
2020-03-03 10:04:05 +00:00
/// Default expression for all added/modified columns
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
2020-05-19 09:54:56 +00:00
NameSet modified_columns, renamed_columns;
2019-12-26 18:17:05 +00:00
for (size_t i = 0; i < size(); ++i)
{
2020-04-22 06:22:14 +00:00
const auto & command = (*this)[i];
2020-02-19 12:52:27 +00:00
const auto & column_name = command.column_name;
if (command.type == AlterCommand::ADD_COLUMN)
2019-12-26 18:17:05 +00:00
{
2020-05-19 09:54:56 +00:00
if (all_columns.has(column_name) || all_columns.hasNested(column_name))
2020-02-19 14:39:01 +00:00
{
2020-02-19 12:52:27 +00:00
if (!command.if_not_exists)
throw Exception{"Cannot add column " + backQuote(column_name) + ": column with this name already exists",
ErrorCodes::DUPLICATE_COLUMN};
2020-02-19 14:39:01 +00:00
else
continue;
}
2019-08-06 12:52:08 +00:00
2020-02-19 12:52:27 +00:00
if (!command.data_type)
throw Exception{"Data type have to be specified for column " + backQuote(column_name) + " to add",
ErrorCodes::BAD_ARGUMENTS};
2020-02-19 12:52:27 +00:00
2020-04-27 13:55:30 +00:00
all_columns.add(ColumnDescription(column_name, command.data_type));
2020-02-19 12:52:27 +00:00
}
else if (command.type == AlterCommand::MODIFY_COLUMN)
{
2020-05-19 09:54:56 +00:00
if (!all_columns.has(column_name))
2020-02-19 14:39:01 +00:00
{
2020-02-19 12:52:27 +00:00
if (!command.if_exists)
throw Exception{"Wrong column name. Cannot find column " + backQuote(column_name) + " to modify",
ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK};
2020-02-19 14:39:01 +00:00
else
continue;
}
2020-05-19 09:54:56 +00:00
if (renamed_columns.count(column_name))
throw Exception{"Cannot rename and modify the same column " + backQuote(column_name) + " in a single ALTER query",
ErrorCodes::NOT_IMPLEMENTED};
modified_columns.emplace(column_name);
2019-12-26 18:17:05 +00:00
}
else if (command.type == AlterCommand::DROP_COLUMN)
{
2020-05-19 09:54:56 +00:00
if (all_columns.has(command.column_name) || all_columns.hasNested(command.column_name))
2019-12-26 18:17:05 +00:00
{
2020-05-19 09:54:56 +00:00
for (const ColumnDescription & column : all_columns)
2019-12-26 18:17:05 +00:00
{
const auto & default_expression = column.default_desc.expression;
2020-02-19 12:52:27 +00:00
if (default_expression)
{
ASTPtr query = default_expression->clone();
2020-05-19 09:54:56 +00:00
auto syntax_result = SyntaxAnalyzer(context).analyze(query, all_columns.getAll());
2020-02-19 12:52:27 +00:00
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 " + backQuote(command.column_name) + ", because column " + backQuote(column.name) + " depends on it",
2020-02-19 12:52:27 +00:00
ErrorCodes::ILLEGAL_COLUMN);
}
2019-12-26 18:17:05 +00:00
}
2020-05-19 09:54:56 +00:00
all_columns.remove(command.column_name);
2019-12-26 18:17:05 +00:00
}
else if (!command.if_exists)
throw Exception(
"Wrong column name. Cannot find column " + backQuote(command.column_name) + " to drop",
ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK);
2019-12-26 18:17:05 +00:00
}
else if (command.type == AlterCommand::COMMENT_COLUMN)
{
2020-05-19 09:54:56 +00:00
if (!all_columns.has(command.column_name))
2019-12-26 18:17:05 +00:00
{
if (!command.if_exists)
throw Exception{"Wrong column name. Cannot find column " + backQuote(command.column_name) + " to comment",
ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK};
2019-12-26 18:17:05 +00:00
}
}
2020-02-28 12:44:50 +00:00
else if (command.type == AlterCommand::MODIFY_SETTING)
{
if (metadata.settings_ast == nullptr)
throw Exception{"Cannot alter settings, because table engine doesn't support settings changes", ErrorCodes::BAD_ARGUMENTS};
}
2020-03-30 13:34:19 +00:00
else if (command.type == AlterCommand::RENAME_COLUMN)
{
2020-05-13 17:43:30 +00:00
for (size_t j = i + 1; j < size(); ++j)
{
auto next_command = (*this)[j];
if (next_command.type == AlterCommand::RENAME_COLUMN)
{
if (next_command.column_name == command.rename_to)
2020-05-15 15:11:13 +00:00
throw Exception{"Transitive renames in a single ALTER query are not allowed (don't make sense)",
2020-05-13 17:43:30 +00:00
ErrorCodes::NOT_IMPLEMENTED};
else if (next_command.column_name == command.column_name)
throw Exception{"Cannot rename column '" + backQuote(command.column_name)
+ "' to two different names in a single ALTER query",
ErrorCodes::BAD_ARGUMENTS};
}
}
2020-03-31 16:18:18 +00:00
/// TODO Implement nested rename
2020-05-13 17:43:30 +00:00
if (all_columns.hasNested(command.column_name))
2020-03-31 16:18:18 +00:00
{
throw Exception{"Cannot rename whole Nested struct", ErrorCodes::NOT_IMPLEMENTED};
}
2020-05-13 17:43:30 +00:00
if (!all_columns.has(command.column_name))
2020-03-30 13:34:19 +00:00
{
if (!command.if_exists)
throw Exception{"Wrong column name. Cannot find column " + backQuote(command.column_name) + " to rename",
ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK};
2020-05-13 17:43:30 +00:00
else
continue;
2020-03-30 13:34:19 +00:00
}
2020-05-13 17:43:30 +00:00
if (all_columns.has(command.rename_to))
2020-03-30 13:34:19 +00:00
throw Exception{"Cannot rename to " + backQuote(command.rename_to) + ": column with this name already exists",
ErrorCodes::DUPLICATE_COLUMN};
2020-05-19 09:54:56 +00:00
if (modified_columns.count(column_name))
throw Exception{"Cannot rename and modify the same column " + backQuote(column_name) + " in a single ALTER query",
ErrorCodes::NOT_IMPLEMENTED};
2020-03-30 13:34:19 +00:00
2020-03-31 16:18:18 +00:00
String from_nested_table_name = Nested::extractTableName(command.column_name);
String to_nested_table_name = Nested::extractTableName(command.rename_to);
bool from_nested = from_nested_table_name != command.column_name;
bool to_nested = to_nested_table_name != command.rename_to;
if (from_nested && to_nested)
{
if (from_nested_table_name != to_nested_table_name)
throw Exception{"Cannot rename column from one nested name to another", ErrorCodes::BAD_ARGUMENTS};
}
else if (!from_nested && !to_nested)
{
2020-05-13 17:43:30 +00:00
all_columns.rename(command.column_name, command.rename_to);
2020-05-19 09:54:56 +00:00
renamed_columns.emplace(command.column_name);
renamed_columns.emplace(command.rename_to);
2020-03-31 16:18:18 +00:00
}
else
{
throw Exception{"Cannot rename column from nested struct to normal column and vice versa", ErrorCodes::BAD_ARGUMENTS};
}
2020-03-30 13:34:19 +00:00
}
2020-02-19 12:52:27 +00:00
/// Collect default expressions for MODIFY and ADD comands
if (command.type == AlterCommand::MODIFY_COLUMN || command.type == AlterCommand::ADD_COLUMN)
{
if (command.default_expression)
{
/// If we modify default, but not type
if (!command.data_type)
{
default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), column_name));
}
else
{
const auto & final_column_name = column_name;
const auto tmp_column_name = final_column_name + "_tmp";
const auto data_type_ptr = command.data_type;
2020-02-19 12:52:27 +00:00
default_expr_list->children.emplace_back(setAlias(
addTypeConversionToAST(std::make_shared<ASTIdentifier>(tmp_column_name), data_type_ptr->getName()),
final_column_name));
default_expr_list->children.emplace_back(setAlias(command.default_expression->clone(), tmp_column_name));
}
} /// if we change data type for column with default
2020-05-19 09:54:56 +00:00
else if (all_columns.has(column_name) && command.data_type)
{
2020-05-19 09:54:56 +00:00
auto column_in_table = all_columns.get(column_name);
/// Column doesn't have a default, nothing to check
if (!column_in_table.default_desc.expression)
continue;
const auto & final_column_name = column_name;
const auto tmp_column_name = final_column_name + "_tmp";
const auto data_type_ptr = command.data_type;
default_expr_list->children.emplace_back(setAlias(
addTypeConversionToAST(std::make_shared<ASTIdentifier>(tmp_column_name), data_type_ptr->getName()), final_column_name));
default_expr_list->children.emplace_back(setAlias(column_in_table.default_desc.expression->clone(), tmp_column_name));
}
}
2020-02-19 12:52:27 +00:00
}
validateColumnsDefaultsAndGetSampleBlock(default_expr_list, all_columns.getAll(), context);
2020-02-19 12:52:27 +00:00
}
2019-08-06 12:52:08 +00:00
bool AlterCommands::isModifyingData(const StorageInMemoryMetadata & metadata) const
2018-11-15 13:12:27 +00:00
{
for (const auto & param : *this)
{
if (param.isModifyingData(metadata))
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(); });
}
2019-12-27 14:36:59 +00:00
bool AlterCommands::isCommentAlter() const
{
return std::all_of(begin(), end(), [](const AlterCommand & c) { return c.isCommentAlter(); });
}
2020-01-13 16:39:20 +00:00
2020-05-19 01:53:01 +00:00
static MutationCommand createMaterializeTTLCommand()
{
MutationCommand command;
auto ast = std::make_shared<ASTAlterCommand>();
ast->type = ASTAlterCommand::MATERIALIZE_TTL;
command.type = MutationCommand::MATERIALIZE_TTL;
command.ast = std::move(ast);
return command;
}
2020-01-13 16:39:20 +00:00
MutationCommands AlterCommands::getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl) const
2020-01-13 16:39:20 +00:00
{
MutationCommands result;
for (const auto & alter_cmd : *this)
if (auto mutation_cmd = alter_cmd.tryConvertToMutationCommand(metadata); mutation_cmd)
result.push_back(*mutation_cmd);
2020-05-19 01:53:01 +00:00
if (materialize_ttl)
{
for (const auto & alter_cmd : *this)
{
if (alter_cmd.isTTLAlter(metadata))
{
result.push_back(createMaterializeTTLCommand());
break;
}
}
}
2020-01-13 16:39:20 +00:00
return result;
}
}