ClickHouse/src/Parsers/ASTAlterQuery.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

260 lines
6.8 KiB
C++
Raw Normal View History

2013-08-07 13:07:42 +00:00
#pragma once
#include <Parsers/ASTExpressionList.h>
2017-04-21 12:39:28 +00:00
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Parsers/ASTQueryWithTableAndOutput.h>
2019-10-09 13:02:05 +00:00
#include <Parsers/ASTTTLElement.h>
#include <Parsers/IAST.h>
2013-08-07 13:07:42 +00:00
namespace DB
{
2017-04-21 12:39:28 +00:00
/** ALTER query:
2013-08-07 13:07:42 +00:00
* ALTER TABLE [db.]name_type
* ADD COLUMN col_name type [AFTER col_after],
2017-04-21 12:39:28 +00:00
* DROP COLUMN col_drop [FROM PARTITION partition],
* MODIFY COLUMN col_name type,
* DROP PARTITION partition,
* COMMENT_COLUMN col_name 'comment',
* ALTER LIVE VIEW [db.]name_type
* REFRESH
2013-08-07 13:07:42 +00:00
*/
class ASTAlterCommand : public IAST
2013-08-07 13:07:42 +00:00
{
public:
enum Type
2013-08-07 13:07:42 +00:00
{
ADD_COLUMN,
DROP_COLUMN,
MODIFY_COLUMN,
COMMENT_COLUMN,
2020-03-24 17:05:38 +00:00
RENAME_COLUMN,
2021-07-31 18:17:06 +00:00
MATERIALIZE_COLUMN,
MODIFY_ORDER_BY,
2020-08-27 13:10:10 +00:00
MODIFY_SAMPLE_BY,
MODIFY_TTL,
2020-01-22 13:24:20 +00:00
MATERIALIZE_TTL,
2019-07-24 12:56:39 +00:00
MODIFY_SETTING,
RESET_SETTING,
MODIFY_QUERY,
REMOVE_TTL,
2021-10-14 13:44:28 +00:00
REMOVE_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,
2019-04-09 15:36:33 +00:00
MATERIALIZE_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
ADD_CONSTRAINT,
DROP_CONSTRAINT,
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_PROJECTION,
DROP_PROJECTION,
MATERIALIZE_PROJECTION,
DROP_PARTITION,
2019-07-22 11:23:11 +00:00
DROP_DETACHED_PARTITION,
2014-08-07 11:46:01 +00:00
ATTACH_PARTITION,
MOVE_PARTITION,
REPLACE_PARTITION,
FETCH_PARTITION,
FREEZE_PARTITION,
FREEZE_ALL,
2021-03-02 20:28:42 +00:00
UNFREEZE_PARTITION,
UNFREEZE_ALL,
DELETE,
UPDATE,
NO_TYPE,
LIVE_VIEW_REFRESH,
2021-08-27 06:30:21 +00:00
MODIFY_DATABASE_SETTING,
MODIFY_COMMENT,
2013-08-07 13:07:42 +00:00
};
Type type = NO_TYPE;
2016-01-28 01:00:27 +00:00
/** The ADD COLUMN query stores the name and type of the column to add
* This field is not used in the DROP query
* In MODIFY query, the column name and the new type are stored here
*/
ASTPtr col_decl;
2013-08-07 13:07:42 +00:00
/** The ADD COLUMN and MODIFY COLUMN query here optionally stores the name of the column following AFTER
* The DROP query stores the column name for deletion here
2020-03-24 17:05:38 +00:00
* Also used for RENAME COLUMN.
*/
ASTPtr column;
2014-07-10 10:16:50 +00:00
/** For MODIFY ORDER BY
*/
ASTPtr order_by;
2020-08-27 13:10:10 +00:00
/** For MODIFY SAMPLE BY
*/
ASTPtr 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
/** The ADD INDEX query stores the IndexDeclaration there.
*/
ASTPtr index_decl;
/** The ADD INDEX query stores the name of the index following AFTER.
* The DROP INDEX query stores the name for deletion.
2019-05-05 17:01:54 +00:00
* The MATERIALIZE INDEX query stores the name of the index to materialize.
2019-05-09 14:25:18 +00:00
* The CLEAR INDEX query stores the name of the index to clear.
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-07-24 12:56:39 +00:00
ASTPtr 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
/** The ADD CONSTRAINT query stores the ConstraintDeclaration there.
*/
ASTPtr constraint_decl;
/** The DROP CONSTRAINT query stores the name for deletion.
*/
ASTPtr constraint;
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
/** The ADD PROJECTION query stores the ProjectionDeclaration there.
*/
ASTPtr projection_decl;
/** The ADD PROJECTION query stores the name of the projection following AFTER.
* The DROP PROJECTION query stores the name for deletion.
* The MATERIALIZE PROJECTION query stores the name of the projection to materialize.
* The CLEAR PROJECTION query stores the name of the projection to clear.
*/
ASTPtr projection;
/** Used in DROP PARTITION, ATTACH PARTITION FROM, UPDATE, DELETE queries.
* The value or ID of the partition is stored here.
*/
ASTPtr partition;
/// For DELETE/UPDATE WHERE: the predicate that filters the rows to delete/update.
ASTPtr predicate;
/// A list of expressions of the form `column = expr` for the UPDATE command.
ASTPtr update_assignments;
/// A column comment
ASTPtr comment;
/// For MODIFY TTL query
ASTPtr ttl;
2019-07-24 12:56:39 +00:00
/// FOR MODIFY_SETTING
ASTPtr settings_changes;
/// FOR RESET_SETTING
ASTPtr settings_resets;
/// For MODIFY_QUERY
ASTPtr select;
/** In ALTER CHANNEL, ADD, DROP, SUSPEND, RESUME, REFRESH, MODIFY queries, the list of live views is stored here
*/
ASTPtr values;
bool detach = false; /// true for DETACH PARTITION
2019-08-29 16:17:47 +00:00
bool part = false; /// true for ATTACH PART, DROP DETACHED PART and MOVE
bool clear_column = false; /// for CLEAR COLUMN (do not drop column from metadata)
2019-05-09 14:25:18 +00:00
bool clear_index = false; /// for CLEAR INDEX (do not drop index from metadata)
bool clear_projection = false; /// for CLEAR PROJECTION (do not drop projection from metadata)
2019-05-09 14:25:18 +00:00
bool if_not_exists = false; /// option for ADD_COLUMN
bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN
bool first = false; /// option for ADD_COLUMN, MODIFY_COLUMN
DataDestinationType move_destination_type; /// option for MOVE PART/PARTITION
2019-07-18 15:19:03 +00:00
String move_destination_name; /// option for MOVE PART/PARTITION
2019-07-18 15:19:03 +00:00
/** For FETCH PARTITION - the path in ZK to the shard, from which to download the partition.
*/
String from;
2016-03-25 11:48:45 +00:00
2021-02-24 14:26:46 +00:00
/**
* For FREEZE PARTITION - place local backup to directory with specified name.
* For UNFREEZE - delete local backup at directory with specified name.
*/
String with_name;
/// REPLACE(ATTACH) PARTITION partition FROM db.table
String from_database;
String from_table;
/// To distinguish REPLACE and ATTACH PARTITION partition FROM db.table
bool replace = true;
/// MOVE PARTITION partition TO TABLE db.table
String to_database;
String to_table;
2019-07-29 03:14:53 +00:00
2020-03-24 17:05:38 +00:00
/// Target column name
ASTPtr rename_to;
/// Which property user want to remove
String remove_property;
String getID(char delim) const override;
ASTPtr clone() const override;
static const char * typeToString(Type type);
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
};
class ASTAlterQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster
{
public:
2021-08-20 17:59:57 +00:00
enum class AlterObjectType
{
TABLE,
DATABASE,
2021-08-27 06:30:21 +00:00
LIVE_VIEW,
UNKNOWN,
2021-08-20 17:59:57 +00:00
};
2021-08-27 06:30:21 +00:00
AlterObjectType alter_object = AlterObjectType::UNKNOWN;
ASTExpressionList * command_list = nullptr;
2013-08-07 13:07:42 +00:00
bool isSettingsAlter() const;
2021-01-26 17:51:25 +00:00
bool isFreezeAlter() const;
bool isAttachAlter() const;
bool isFetchAlter() const;
bool isDropPartitionAlter() const;
String getID(char) const override;
2013-08-07 13:07:42 +00:00
2016-01-28 01:00:27 +00:00
ASTPtr clone() const override;
ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams & params) const override
{
return removeOnCluster<ASTAlterQuery>(clone(), params.default_database);
}
2017-04-21 12:39:28 +00:00
QueryKind getQueryKind() const override { return QueryKind::Alter; }
2021-08-04 14:09:23 +00:00
protected:
void formatQueryImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override;
2021-01-26 17:51:25 +00:00
bool isOneCommandTypeOnly(const ASTAlterCommand::Type & type) const;
2013-08-07 13:07:42 +00:00
};
2013-08-07 13:07:42 +00:00
}