ClickHouse/dbms/src/Interpreters/InterpreterAlterQuery.cpp

202 lines
6.7 KiB
C++
Raw Normal View History

2013-08-07 13:07:42 +00:00
#include <DB/Interpreters/InterpreterAlterQuery.h>
#include <DB/Interpreters/InterpreterCreateQuery.h>
2013-08-07 13:07:42 +00:00
#include <DB/Parsers/ASTAlterQuery.h>
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/ASTExpressionList.h>
#include <DB/Parsers/ASTNameTypePair.h>
#include <DB/Parsers/ASTIdentifier.h>
2015-03-02 01:10:58 +00:00
#include <DB/Parsers/ASTLiteral.h>
2013-08-07 13:07:42 +00:00
#include <DB/Parsers/ParserCreateQuery.h>
#include <DB/IO/copyData.h>
2015-04-16 06:12:35 +00:00
#include <DB/IO/ReadBufferFromFile.h>
2013-08-07 13:07:42 +00:00
#include <DB/Common/escapeForFileName.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Parsers/parseQuery.h>
2013-08-07 13:07:42 +00:00
#include <Poco/FileStream.h>
2013-08-07 13:07:42 +00:00
#include <algorithm>
2013-08-09 00:12:59 +00:00
2013-09-23 12:01:19 +00:00
using namespace DB;
2013-08-07 13:07:42 +00:00
InterpreterAlterQuery::InterpreterAlterQuery(ASTPtr query_ptr_, Context & context_)
2013-08-09 00:12:59 +00:00
: query_ptr(query_ptr_), context(context_)
2013-08-07 13:07:42 +00:00
{
}
void InterpreterAlterQuery::execute()
{
auto & alter = typeid_cast<ASTAlterQuery &>(*query_ptr);
const String & table_name = alter.table;
String database_name = alter.database.empty() ? context.getCurrentDatabase() : alter.database;
StoragePtr table = context.getTable(database_name, table_name);
2014-08-07 09:23:55 +00:00
AlterCommands alter_commands;
PartitionCommands partition_commands;
parseAlter(alter.parameters, context.getDataTypeFactory(), alter_commands, partition_commands);
2014-08-07 09:23:55 +00:00
for (const PartitionCommand & command : partition_commands)
{
switch (command.type)
{
case PartitionCommand::DROP_PARTITION:
2015-04-21 13:10:08 +00:00
table->dropPartition(command.partition, command.detach, command.unreplicated, context.getSettingsRef());
break;
case PartitionCommand::ATTACH_PARTITION:
table->attachPartition(command.partition, command.unreplicated, command.part, context.getSettingsRef());
break;
case PartitionCommand::FETCH_PARTITION:
table->fetchPartition(command.partition, command.from, context.getSettingsRef());
break;
case PartitionCommand::FREEZE_PARTITION:
table->freezePartition(command.partition, context.getSettingsRef());
break;
default:
throw Exception("Bad PartitionCommand::Type: " + toString(command.type), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
}
2014-08-07 09:23:55 +00:00
}
if (alter_commands.empty())
return;
alter_commands.validate(table.get(), context);
table->alter(alter_commands, database_name, table_name, context);
}
void InterpreterAlterQuery::parseAlter(
const ASTAlterQuery::ParameterContainer & params_container, const DataTypeFactory & data_type_factory,
2014-08-07 09:23:55 +00:00
AlterCommands & out_alter_commands, PartitionCommands & out_partition_commands)
{
for (const auto & params : params_container)
{
if (params.type == ASTAlterQuery::ADD_COLUMN)
{
AlterCommand command;
command.type = AlterCommand::ADD;
const auto & ast_col_decl = typeid_cast<const ASTColumnDeclaration &>(*params.col_decl);
command.column_name = ast_col_decl.name;
if (ast_col_decl.type)
{
StringRange type_range = ast_col_decl.type->range;
String type_string(type_range.first, type_range.second - type_range.first);
command.data_type = data_type_factory.get(type_string);
}
if (ast_col_decl.default_expression)
{
command.default_type = columnDefaultTypeFromString(ast_col_decl.default_specifier);
command.default_expression = ast_col_decl.default_expression;
}
if (params.column)
command.after_column = typeid_cast<const ASTIdentifier &>(*params.column).name;
2014-08-07 09:23:55 +00:00
out_alter_commands.push_back(command);
}
else if (params.type == ASTAlterQuery::DROP_COLUMN)
{
AlterCommand command;
command.type = AlterCommand::DROP;
command.column_name = typeid_cast<const ASTIdentifier &>(*(params.column)).name;
2014-08-07 09:23:55 +00:00
out_alter_commands.push_back(command);
}
else if (params.type == ASTAlterQuery::MODIFY_COLUMN)
{
AlterCommand command;
command.type = AlterCommand::MODIFY;
const auto & ast_col_decl = typeid_cast<const ASTColumnDeclaration &>(*params.col_decl);
command.column_name = ast_col_decl.name;
if (ast_col_decl.type)
{
StringRange type_range = ast_col_decl.type->range;
String type_string(type_range.first, type_range.second - type_range.first);
command.data_type = data_type_factory.get(type_string);
}
if (ast_col_decl.default_expression)
{
command.default_type = columnDefaultTypeFromString(ast_col_decl.default_specifier);
command.default_expression = ast_col_decl.default_expression;
}
2014-08-07 09:23:55 +00:00
out_alter_commands.push_back(command);
}
else if (params.type == ASTAlterQuery::DROP_PARTITION)
{
const Field & partition = dynamic_cast<const ASTLiteral &>(*params.partition).value;
2015-04-21 13:10:08 +00:00
out_partition_commands.push_back(PartitionCommand::dropPartition(partition, params.detach, params.unreplicated));
}
2014-08-07 11:46:01 +00:00
else if (params.type == ASTAlterQuery::ATTACH_PARTITION)
{
const Field & partition = dynamic_cast<const ASTLiteral &>(*params.partition).value;
out_partition_commands.push_back(PartitionCommand::attachPartition(partition, params.unreplicated, params.part));
}
else if (params.type == ASTAlterQuery::FETCH_PARTITION)
{
const Field & partition = dynamic_cast<const ASTLiteral &>(*params.partition).value;
out_partition_commands.push_back(PartitionCommand::fetchPartition(partition, params.from));
}
else if (params.type == ASTAlterQuery::FREEZE_PARTITION)
{
const Field & partition = dynamic_cast<const ASTLiteral &>(*params.partition).value;
out_partition_commands.push_back(PartitionCommand::freezePartition(partition));
}
else
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
}
}
void InterpreterAlterQuery::updateMetadata(
const String & database_name,
const String & table_name,
const NamesAndTypesList & columns,
const NamesAndTypesList & materialized_columns,
const NamesAndTypesList & alias_columns,
const ColumnDefaults & column_defaults,
Context & context)
2013-08-07 13:07:42 +00:00
{
2013-09-23 12:01:19 +00:00
String path = context.getPath();
2013-08-07 13:07:42 +00:00
String database_name_escaped = escapeForFileName(database_name);
String table_name_escaped = escapeForFileName(table_name);
2013-08-09 00:12:59 +00:00
String metadata_path = path + "metadata/" + database_name_escaped + "/" + table_name_escaped + ".sql";
String metadata_temp_path = metadata_path + ".tmp";
2013-08-07 13:07:42 +00:00
2014-07-11 13:34:12 +00:00
StringPtr query = new String();
2014-07-10 10:16:50 +00:00
{
2014-07-11 13:34:12 +00:00
ReadBufferFromFile in(metadata_path);
WriteBufferFromString out(*query);
copyData(in, out);
2014-07-10 10:16:50 +00:00
}
2014-07-11 13:34:12 +00:00
ParserCreateQuery parser;
ASTPtr ast = parseQuery(parser, query->data(), query->data() + query->size(), "in file " + metadata_path);
2014-07-10 10:16:50 +00:00
2014-07-11 13:34:12 +00:00
ast->query_string = query;
2014-07-10 10:16:50 +00:00
2014-07-11 13:34:12 +00:00
ASTCreateQuery & attach = typeid_cast<ASTCreateQuery &>(*ast);
2014-07-10 10:16:50 +00:00
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(columns, materialized_columns, alias_columns, column_defaults);
*std::find(attach.children.begin(), attach.children.end(), attach.columns) = new_columns;
attach.columns = new_columns;
2014-07-10 10:16:50 +00:00
2013-08-07 13:07:42 +00:00
{
2014-07-11 13:34:12 +00:00
Poco::FileOutputStream ostr(metadata_temp_path);
2014-07-10 10:16:50 +00:00
formatAST(attach, ostr, 0, false);
}
Poco::File(metadata_temp_path).renameTo(metadata_path);
2014-07-10 10:16:50 +00:00
}