ClickHouse/dbms/src/Storages/AlterCommands.h

132 lines
3.9 KiB
C++
Raw Normal View History

2014-07-11 08:10:45 +00:00
#pragma once
2018-06-16 02:13:54 +00:00
#include <optional>
#include <Core/NamesAndTypes.h>
#include <Storages/ColumnsDescription.h>
#include <Storages/IStorage_fwd.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 <Storages/IndicesDescription.h>
#include <Storages/ConstraintsDescription.h>
2014-07-11 08:10:45 +00:00
2019-07-24 12:56:39 +00:00
#include <Common/SettingsChanges.h>
2014-07-11 08:10:45 +00:00
2018-06-16 02:13:54 +00:00
2014-07-11 08:10:45 +00:00
namespace DB
{
class ASTAlterCommand;
2018-11-01 10:35:50 +00:00
/// Operation from the ALTER query (except for manipulation with PART/PARTITION).
/// Adding Nested columns is not expanded to add individual columns.
2014-07-11 08:10:45 +00:00
struct AlterCommand
{
enum Type
{
ADD_COLUMN,
DROP_COLUMN,
MODIFY_COLUMN,
COMMENT_COLUMN,
MODIFY_ORDER_BY,
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
ADD_INDEX,
DROP_INDEX,
ADD_CONSTRAINT,
DROP_CONSTRAINT,
MODIFY_TTL,
2018-11-14 22:46:39 +00:00
UKNOWN_TYPE,
2019-07-24 12:56:39 +00:00
MODIFY_SETTING,
};
2018-11-14 22:46:39 +00:00
Type type = UKNOWN_TYPE;
String column_name;
/// For DROP COLUMN ... FROM PARTITION
String partition_name;
2017-04-16 15:00:33 +00:00
/// For ADD and MODIFY, a new column type.
DataTypePtr data_type;
ColumnDefaultKind default_kind{};
ASTPtr default_expression{};
String comment;
2017-04-16 15:00:33 +00:00
/// For ADD - after which column to add a new one. If an empty string, add to the end. To add to the beginning now it is impossible.
String after_column;
/// For DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN
2018-12-25 23:06:39 +00:00
bool if_exists = false;
/// For ADD_COLUMN
2018-12-25 23:06:39 +00:00
bool if_not_exists = false;
/// For MODIFY_ORDER_BY
ASTPtr order_by;
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
/// For ADD INDEX
ASTPtr index_decl;
String after_index_name;
/// For ADD/DROP INDEX
String index_name;
// For ADD CONSTRAINT
ASTPtr constraint_decl;
// For ADD/DROP CONSTRAINT
String constraint_name;
/// For MODIFY TTL
ASTPtr ttl;
/// indicates that this command should not be applied, for example in case of if_exists=true and column doesn't exist.
bool ignore = false;
2018-12-21 14:40:20 +00:00
/// For ADD and MODIFY
CompressionCodecPtr codec;
2019-07-24 12:56:39 +00:00
/// For MODIFY SETTING
SettingsChanges settings_changes;
AlterCommand() = default;
2019-08-03 11:02:40 +00:00
AlterCommand(const Type type_, const String & column_name_, const DataTypePtr & data_type_,
const ColumnDefaultKind default_kind_, const ASTPtr & default_expression_,
const String & after_column_, const String & comment_,
const bool if_exists_, const bool if_not_exists_)
: type{type_}, column_name{column_name_}, data_type{data_type_}, default_kind{default_kind_},
default_expression{default_expression_}, comment(comment_), after_column{after_column_},
if_exists(if_exists_), if_not_exists(if_not_exists_)
{}
static std::optional<AlterCommand> parse(const ASTAlterCommand * 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
void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
2019-08-26 10:07:32 +00:00
ConstraintsDescription & constraints_description, ASTPtr & order_by_ast,
ASTPtr & primary_key_ast, ASTPtr & ttl_table_ast, SettingsChanges & changes) const;
2019-05-02 23:56:42 +00:00
/// Checks that not only metadata touched by that command
2019-05-02 23:56:42 +00:00
bool isMutable() const;
2019-08-06 12:52:08 +00:00
/// checks that only settings changed by alter
bool isSettingsAlter() const;
2014-07-11 08:10:45 +00:00
};
class Context;
2014-07-11 08:10:45 +00:00
class AlterCommands : public std::vector<AlterCommand>
{
public:
2019-07-29 09:17:37 +00:00
/// Used for primitive table engines, where only columns metadata can be changed
2019-07-24 12:56:39 +00:00
void applyForColumnsOnly(ColumnsDescription & columns_description) const;
void apply(ColumnsDescription & columns_description, IndicesDescription & indices_description,
2019-08-26 10:07:32 +00:00
ConstraintsDescription & constraints_description, ASTPtr & order_by_ast, ASTPtr & primary_key_ast,
ASTPtr & ttl_table_ast, SettingsChanges & changes) const;
2019-08-06 12:52:08 +00:00
/// Apply alter commands only for settings. Exception will be thrown if any other part of table structure will be modified.
void applyForSettingsOnly(SettingsChanges & changes) const;
void validate(const IStorage & table, const Context & context);
2019-05-02 23:56:42 +00:00
bool isMutable() const;
2019-08-06 12:52:08 +00:00
bool isSettingsAlter() const;
2014-07-11 08:10:45 +00:00
};
}