ClickHouse/dbms/include/DB/Interpreters/InterpreterAlterQuery.h

113 lines
2.9 KiB
C++
Raw Normal View History

2013-08-07 13:07:42 +00:00
#pragma once
#include <DB/Storages/IStorage.h>
#include <DB/Storages/AlterCommands.h>
2013-08-07 13:07:42 +00:00
#include <DB/Interpreters/Context.h>
2015-06-18 02:11:05 +00:00
#include <DB/Interpreters/IInterpreter.h>
2013-08-07 13:07:42 +00:00
namespace DB
{
2013-08-07 13:07:42 +00:00
2013-08-08 23:49:59 +00:00
/** Позволяет добавить или удалить столбец в таблице.
* Также позволяет осуществить манипуляции с партициями таблиц семейства MergeTree.
2013-08-07 13:07:42 +00:00
*/
2015-06-18 02:11:05 +00:00
class InterpreterAlterQuery : public IInterpreter
2013-08-07 13:07:42 +00:00
{
public:
2016-01-28 01:00:27 +00:00
InterpreterAlterQuery(ASTPtr query_ptr_, const Context & context_);
2013-08-07 13:07:42 +00:00
2015-06-18 02:11:05 +00:00
BlockIO execute() override;
2013-08-07 13:07:42 +00:00
private:
2014-08-07 09:23:55 +00:00
struct PartitionCommand
{
enum Type
{
DROP_PARTITION,
2014-08-07 11:46:01 +00:00
ATTACH_PARTITION,
FETCH_PARTITION,
FREEZE_PARTITION,
2016-01-28 01:00:27 +00:00
RESHARD_PARTITION
2014-08-07 09:23:55 +00:00
};
Type type;
Field partition;
bool detach = false; /// true для DETACH PARTITION.
2014-08-07 09:23:55 +00:00
bool unreplicated = false;
bool part = false;
2014-08-07 11:46:01 +00:00
String from; /// Для FETCH PARTITION - путь в ZK к шарду, с которого скачивать партицию.
2016-01-28 01:00:27 +00:00
/// Для RESHARD PARTITION.
Field last_partition;
WeightedZooKeeperPaths weighted_zookeeper_paths;
2016-01-28 01:00:42 +00:00
ASTPtr sharding_key_expr;
bool do_copy = false;
2016-03-01 17:47:53 +00:00
Field coordinator;
2016-01-28 01:00:27 +00:00
/// For FREEZE PARTITION
String with_name;
2015-04-21 13:10:08 +00:00
static PartitionCommand dropPartition(const Field & partition, bool detach, bool unreplicated)
2014-08-07 09:23:55 +00:00
{
PartitionCommand res;
res.type = DROP_PARTITION;
res.partition = partition;
res.detach = detach;
res.unreplicated = unreplicated;
return res;
2014-08-07 09:23:55 +00:00
}
2014-08-07 11:46:01 +00:00
static PartitionCommand attachPartition(const Field & partition, bool unreplicated, bool part)
{
PartitionCommand res;
res.type = ATTACH_PARTITION;
res.partition = partition;
res.unreplicated = unreplicated;
res.part = part;
return res;
2014-08-07 11:46:01 +00:00
}
static PartitionCommand fetchPartition(const Field & partition, const String & from)
{
PartitionCommand res;
res.type = FETCH_PARTITION;
res.partition = partition;
res.from = from;
return res;
}
static PartitionCommand freezePartition(const Field & partition, const String & with_name)
{
PartitionCommand res;
res.type = FREEZE_PARTITION;
res.partition = partition;
res.with_name = with_name;
return res;
}
2016-01-28 01:00:27 +00:00
static PartitionCommand reshardPartitions(const Field & first_partition_, const Field & last_partition_,
2016-03-25 11:48:45 +00:00
const WeightedZooKeeperPaths & weighted_zookeeper_paths_, const ASTPtr & sharding_key_expr_,
bool do_copy_, const Field & coordinator_)
2016-01-28 01:00:27 +00:00
{
2016-03-01 17:47:53 +00:00
return {RESHARD_PARTITION, first_partition_, false, false, false, {},
2016-03-25 11:48:45 +00:00
last_partition_, weighted_zookeeper_paths_, sharding_key_expr_, do_copy_, coordinator_};
2016-01-28 01:00:27 +00:00
}
2014-08-07 09:23:55 +00:00
};
using PartitionCommands = std::vector<PartitionCommand>;
2013-08-07 13:07:42 +00:00
ASTPtr query_ptr;
2013-08-07 13:07:42 +00:00
Context context;
static void parseAlter(const ASTAlterQuery::ParameterContainer & params,
2014-08-07 09:23:55 +00:00
AlterCommands & out_alter_commands, PartitionCommands & out_partition_commands);
2013-08-07 13:07:42 +00:00
};
2013-08-07 13:07:42 +00:00
}