ClickHouse/src/Storages/AlterCommands.h

197 lines
5.6 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/IStorage_fwd.h>
2019-12-26 18:17:05 +00:00
#include <Storages/StorageInMemoryMetadata.h>
2020-01-13 16:39:20 +00:00
#include <Storages/MutationCommands.h>
2020-02-19 12:52:27 +00:00
#include <Storages/ColumnsDescription.h>
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
{
/// The AST of the whole command
ASTPtr ast;
2020-01-13 16:39:20 +00:00
enum Type
{
ADD_COLUMN,
DROP_COLUMN,
MODIFY_COLUMN,
COMMENT_COLUMN,
MODIFY_ORDER_BY,
2020-08-27 13:10:10 +00:00
MODIFY_SAMPLE_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,
2019-07-24 12:56:39 +00:00
MODIFY_SETTING,
MODIFY_QUERY,
2020-03-24 17:05:38 +00:00
RENAME_COLUMN,
REMOVE_TTL,
};
/// Which property user wants to remove from column
enum class RemoveProperty
{
NO_PROPERTY,
/// Default specifiers
DEFAULT,
MATERIALIZED,
ALIAS,
/// Other properties
COMMENT,
CODEC,
TTL
};
2019-12-26 18:17:05 +00:00
Type type;
String column_name;
2020-03-17 13:49:50 +00:00
/// For DROP/CLEAR COLUMN/INDEX ... IN PARTITION
ASTPtr partition;
2017-04-16 15:00:33 +00:00
/// For ADD and MODIFY, a new column type.
2019-12-27 19:04:30 +00:00
DataTypePtr data_type = nullptr;
ColumnDefaultKind default_kind{};
ASTPtr default_expression{};
2019-12-26 18:17:05 +00:00
/// For COMMENT column
std::optional<String> comment;
/// For ADD or MODIFY - after which column to add a new one. If an empty string, add to the end.
String after_column;
/// For ADD_COLUMN, MODIFY_COLUMN - Add to the begin if it is true.
bool first = false;
/// 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
2019-12-27 19:04:30 +00:00
ASTPtr order_by = nullptr;
2020-08-27 13:10:10 +00:00
/// For MODIFY_SAMPLE_BY
ASTPtr sample_by = nullptr;
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
2019-12-27 19:04:30 +00:00
ASTPtr index_decl = nullptr;
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
String after_index_name;
/// For ADD/DROP INDEX
String index_name;
// For ADD CONSTRAINT
2019-12-27 19:04:30 +00:00
ASTPtr constraint_decl = nullptr;
// For ADD/DROP CONSTRAINT
String constraint_name;
/// For MODIFY TTL
2019-12-27 19:04:30 +00:00
ASTPtr ttl = nullptr;
/// indicates that this command should not be applied, for example in case of if_exists=true and column doesn't exist.
bool ignore = false;
2020-03-17 13:49:50 +00:00
/// Clear columns or index (don't drop from metadata)
bool clear = false;
2018-12-21 14:40:20 +00:00
/// For ADD and MODIFY
ASTPtr codec = nullptr;
2018-12-21 14:40:20 +00:00
2019-07-24 12:56:39 +00:00
/// For MODIFY SETTING
SettingsChanges settings_changes;
/// For MODIFY_QUERY
ASTPtr select = nullptr;
2020-03-24 17:05:38 +00:00
/// Target column name
String rename_to;
/// What to remove from column (or TTL)
RemoveProperty to_remove = RemoveProperty::NO_PROPERTY;
static std::optional<AlterCommand> parse(const ASTAlterCommand * command);
2020-05-28 12:37:05 +00:00
void apply(StorageInMemoryMetadata & metadata, const Context & context) const;
2019-05-02 23:56:42 +00:00
2020-07-21 15:19:41 +00:00
/// Check that alter command require data modification (mutation) to be
/// executed. For example, cast from Date to UInt16 type can be executed
/// without any data modifications. But column drop or modify from UInt16 to
/// UInt32 require data modification.
2020-01-13 16:39:20 +00:00
bool isRequireMutationStage(const StorageInMemoryMetadata & metadata) const;
2019-12-29 11:25:26 +00:00
/// Checks that only settings changed by alter
2019-08-06 12:52:08 +00:00
bool isSettingsAlter() const;
2019-12-27 14:36:59 +00:00
2019-12-29 11:25:26 +00:00
/// Checks that only comment changed by alter
2019-12-27 14:36:59 +00:00
bool isCommentAlter() const;
2020-01-13 16:39:20 +00:00
2020-05-19 01:53:01 +00:00
/// Checks that any TTL changed by alter
bool isTTLAlter(const StorageInMemoryMetadata & metadata) const;
/// Command removing some property from column or table
bool isRemovingProperty() const;
/// If possible, convert alter command to mutation command. In other case
/// return empty optional. Some storages may execute mutations after
/// metadata changes.
2020-05-28 12:37:05 +00:00
std::optional<MutationCommand> tryConvertToMutationCommand(StorageInMemoryMetadata & metadata, const Context & context) const;
2014-07-11 08:10:45 +00:00
};
2019-12-29 11:25:26 +00:00
/// Return string representation of AlterCommand::Type
2019-12-26 18:17:05 +00:00
String alterTypeToString(const AlterCommand::Type type);
class Context;
2019-12-29 11:25:26 +00:00
/// Vector of AlterCommand with several additional functions
2014-07-11 08:10:45 +00:00
class AlterCommands : public std::vector<AlterCommand>
{
2019-12-26 18:17:05 +00:00
private:
bool prepared = false;
2020-02-19 12:52:27 +00:00
2014-07-11 08:10:45 +00:00
public:
2019-12-29 11:25:26 +00:00
/// Validate that commands can be applied to metadata.
2020-08-08 00:47:03 +00:00
/// Checks that all columns exist and dependencies between them.
2019-12-29 11:25:26 +00:00
/// This check is lightweight and base only on metadata.
/// More accurate check have to be performed with storage->checkAlterIsPossible.
void validate(const StorageInMemoryMetadata & metadata, const Context & context) const;
2019-08-06 12:52:08 +00:00
2020-02-19 15:07:02 +00:00
/// Prepare alter commands. Set ignore flag to some of them and set some
/// parts to commands from storage's metadata (for example, absent default)
void prepare(const StorageInMemoryMetadata & metadata);
2019-12-26 18:17:05 +00:00
2019-12-29 11:25:26 +00:00
/// Apply all alter command in sequential order to storage metadata.
/// Commands have to be prepared before apply.
2020-05-28 12:37:05 +00:00
void apply(StorageInMemoryMetadata & metadata, const Context & context) const;
2019-12-26 18:17:05 +00:00
2019-12-29 11:25:26 +00:00
/// At least one command modify settings.
2019-08-06 12:52:08 +00:00
bool isSettingsAlter() const;
2019-12-27 14:36:59 +00:00
2019-12-29 11:25:26 +00:00
/// At least one command modify comments.
2019-12-27 14:36:59 +00:00
bool isCommentAlter() const;
2020-01-13 16:39:20 +00:00
/// Return mutation commands which some storages may execute as part of
2020-05-22 08:17:57 +00:00
/// alter. If alter can be performed as pure metadata update, than result is
/// empty. If some TTL changes happened than, depending on materialize_ttl
/// additional mutation command (MATERIALIZE_TTL) will be returned.
2020-05-28 12:37:05 +00:00
MutationCommands getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, const Context & context) const;
2014-07-11 08:10:45 +00:00
};
}