2013-08-07 13:07:42 +00:00
|
|
|
|
#include <DB/Interpreters/InterpreterAlterQuery.h>
|
2014-07-11 08:12:03 +00:00
|
|
|
|
#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>
|
|
|
|
|
|
|
|
|
|
#include <DB/Parsers/ParserCreateQuery.h>
|
|
|
|
|
#include <DB/IO/copyData.h>
|
|
|
|
|
#include <DB/Common/escapeForFileName.h>
|
|
|
|
|
#include <DB/Parsers/formatAST.h>
|
2013-09-23 12:01:19 +00:00
|
|
|
|
#include <DB/Storages/StorageMerge.h>
|
2014-03-05 18:07:35 +00:00
|
|
|
|
#include <DB/Storages/StorageMergeTree.h>
|
2014-05-21 10:34:27 +00:00
|
|
|
|
#include <DB/Storages/StorageReplicatedMergeTree.h>
|
2013-08-07 13:07:42 +00:00
|
|
|
|
|
2014-06-03 23:09:57 +00:00
|
|
|
|
#include <Poco/FileStream.h>
|
|
|
|
|
|
2013-08-07 13:07:42 +00:00
|
|
|
|
#include <algorithm>
|
|
|
|
|
#include <boost/bind.hpp>
|
|
|
|
|
#include <boost/bind/placeholders.hpp>
|
|
|
|
|
|
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
|
|
|
|
{
|
|
|
|
|
}
|
|
|
|
|
|
2014-07-11 08:12:03 +00:00
|
|
|
|
void InterpreterAlterQuery::execute()
|
2013-12-18 11:19:37 +00:00
|
|
|
|
{
|
2014-07-11 08:12:03 +00:00
|
|
|
|
ASTAlterQuery & alter = typeid_cast<ASTAlterQuery &>(*query_ptr);
|
|
|
|
|
String & table_name = alter.table;
|
|
|
|
|
String database_name = alter.database.empty() ? context.getCurrentDatabase() : alter.database;
|
2014-08-07 09:23:55 +00:00
|
|
|
|
AlterCommands alter_commands;
|
|
|
|
|
PartitionCommands partition_commands;
|
|
|
|
|
parseAlter(alter.parameters, context.getDataTypeFactory(), alter_commands, partition_commands);
|
2013-12-18 11:19:37 +00:00
|
|
|
|
|
2014-07-11 08:12:03 +00:00
|
|
|
|
StoragePtr table = context.getTable(database_name, table_name);
|
2014-08-06 10:26:35 +00:00
|
|
|
|
|
2014-08-07 09:23:55 +00:00
|
|
|
|
for (const PartitionCommand & command : partition_commands)
|
|
|
|
|
{
|
2014-10-09 20:28:33 +00:00
|
|
|
|
switch (command.type)
|
|
|
|
|
{
|
|
|
|
|
case PartitionCommand::DROP_PARTITION:
|
|
|
|
|
table->dropPartition(command.partition, command.detach);
|
|
|
|
|
break;
|
|
|
|
|
|
|
|
|
|
case PartitionCommand::ATTACH_PARTITION:
|
|
|
|
|
table->attachPartition(command.partition, command.unreplicated, command.part);
|
|
|
|
|
break;
|
|
|
|
|
|
|
|
|
|
case PartitionCommand::FETCH_PARTITION:
|
|
|
|
|
table->fetchPartition(command.partition, command.unreplicated, command.from);
|
|
|
|
|
break;
|
|
|
|
|
|
|
|
|
|
default:
|
|
|
|
|
throw Exception("Bad PartitionCommand::Type: " + toString(command.type), ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
|
|
|
|
}
|
2014-08-07 09:23:55 +00:00
|
|
|
|
}
|
2014-08-06 10:26:35 +00:00
|
|
|
|
|
2014-08-07 09:23:55 +00:00
|
|
|
|
if (!alter_commands.empty())
|
|
|
|
|
table->alter(alter_commands, database_name, table_name, context);
|
2013-12-18 11:19:37 +00:00
|
|
|
|
}
|
|
|
|
|
|
2014-08-06 10:26:35 +00:00
|
|
|
|
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)
|
2014-04-02 18:53:30 +00:00
|
|
|
|
{
|
2014-07-11 08:12:03 +00:00
|
|
|
|
for (const auto & params : params_container)
|
2014-04-02 18:53:30 +00:00
|
|
|
|
{
|
2014-08-06 09:24:30 +00:00
|
|
|
|
if (params.type == ASTAlterQuery::ADD_COLUMN)
|
2014-04-02 18:53:30 +00:00
|
|
|
|
{
|
2014-08-06 10:26:35 +00:00
|
|
|
|
AlterCommand command;
|
2014-07-11 08:12:03 +00:00
|
|
|
|
command.type = AlterCommand::ADD;
|
2014-04-02 18:53:30 +00:00
|
|
|
|
|
2014-07-11 08:12:03 +00:00
|
|
|
|
const ASTNameTypePair & ast_name_type = typeid_cast<const ASTNameTypePair &>(*params.name_type);
|
|
|
|
|
StringRange type_range = ast_name_type.type->range;
|
|
|
|
|
String type_string = String(type_range.first, type_range.second - type_range.first);
|
2014-05-20 15:00:13 +00:00
|
|
|
|
|
2014-07-11 08:12:03 +00:00
|
|
|
|
command.column_name = ast_name_type.name;
|
|
|
|
|
command.data_type = data_type_factory.get(type_string);
|
2014-05-20 15:00:13 +00:00
|
|
|
|
|
2014-07-11 08:12:03 +00:00
|
|
|
|
if (params.column)
|
|
|
|
|
command.after_column = typeid_cast<const ASTIdentifier &>(*params.column).name;
|
2014-08-06 10:26:35 +00:00
|
|
|
|
|
2014-08-07 09:23:55 +00:00
|
|
|
|
out_alter_commands.push_back(command);
|
2014-04-02 18:53:30 +00:00
|
|
|
|
}
|
2014-08-06 09:24:30 +00:00
|
|
|
|
else if (params.type == ASTAlterQuery::DROP_COLUMN)
|
2014-05-20 15:44:00 +00:00
|
|
|
|
{
|
2014-08-06 10:26:35 +00:00
|
|
|
|
AlterCommand command;
|
2014-07-11 08:12:03 +00:00
|
|
|
|
command.type = AlterCommand::DROP;
|
|
|
|
|
command.column_name = typeid_cast<const ASTIdentifier &>(*(params.column)).name;
|
2014-08-06 10:26:35 +00:00
|
|
|
|
|
2014-08-07 09:23:55 +00:00
|
|
|
|
out_alter_commands.push_back(command);
|
2014-05-20 15:44:00 +00:00
|
|
|
|
}
|
2014-08-06 09:24:30 +00:00
|
|
|
|
else if (params.type == ASTAlterQuery::MODIFY_COLUMN)
|
2014-05-20 15:00:13 +00:00
|
|
|
|
{
|
2014-08-06 10:26:35 +00:00
|
|
|
|
AlterCommand command;
|
2014-07-11 08:12:03 +00:00
|
|
|
|
command.type = AlterCommand::MODIFY;
|
2014-05-20 15:44:00 +00:00
|
|
|
|
|
2014-07-11 08:12:03 +00:00
|
|
|
|
const ASTNameTypePair & ast_name_type = typeid_cast<const ASTNameTypePair &>(*params.name_type);
|
|
|
|
|
StringRange type_range = ast_name_type.type->range;
|
|
|
|
|
String type_string = String(type_range.first, type_range.second - type_range.first);
|
2014-05-20 15:00:13 +00:00
|
|
|
|
|
2014-07-11 08:12:03 +00:00
|
|
|
|
command.column_name = ast_name_type.name;
|
|
|
|
|
command.data_type = data_type_factory.get(type_string);
|
2014-08-06 10:26:35 +00:00
|
|
|
|
|
2014-08-07 09:23:55 +00:00
|
|
|
|
out_alter_commands.push_back(command);
|
2014-08-06 10:26:35 +00:00
|
|
|
|
}
|
|
|
|
|
else if (params.type == ASTAlterQuery::DROP_PARTITION)
|
|
|
|
|
{
|
2014-08-06 10:28:36 +00:00
|
|
|
|
const Field & partition = dynamic_cast<const ASTLiteral &>(*params.partition).value;
|
2014-08-07 09:23:55 +00:00
|
|
|
|
out_partition_commands.push_back(PartitionCommand::dropPartition(partition, params.detach));
|
2014-05-20 15:00:13 +00:00
|
|
|
|
}
|
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));
|
|
|
|
|
}
|
2014-10-09 20:28:33 +00:00
|
|
|
|
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.unreplicated, params.from));
|
|
|
|
|
}
|
2014-05-20 15:00:13 +00:00
|
|
|
|
else
|
2014-07-11 08:12:03 +00:00
|
|
|
|
throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR);
|
2014-05-20 15:00:13 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2014-07-11 08:12:03 +00:00
|
|
|
|
void InterpreterAlterQuery::updateMetadata(
|
2014-07-14 15:49:03 +00:00
|
|
|
|
const String & database_name, const String & table_name, const NamesAndTypesList & columns, Context & context)
|
2013-08-07 13:07:42 +00:00
|
|
|
|
{
|
2013-09-23 12:01:19 +00:00
|
|
|
|
String path = context.getPath();
|
2014-03-05 18:07:35 +00:00
|
|
|
|
|
2013-08-07 13:07:42 +00:00
|
|
|
|
String database_name_escaped = escapeForFileName(database_name);
|
|
|
|
|
String table_name_escaped = escapeForFileName(table_name);
|
2014-03-05 18:07:35 +00:00
|
|
|
|
|
2013-08-09 00:12:59 +00:00
|
|
|
|
String metadata_path = path + "metadata/" + database_name_escaped + "/" + table_name_escaped + ".sql";
|
2014-07-11 08:12:03 +00:00
|
|
|
|
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
|
|
|
|
const char * begin = query->data();
|
|
|
|
|
const char * end = begin + query->size();
|
|
|
|
|
const char * pos = begin;
|
2014-07-10 10:16:50 +00:00
|
|
|
|
|
2014-07-11 13:34:12 +00:00
|
|
|
|
ParserCreateQuery parser;
|
|
|
|
|
ASTPtr ast;
|
|
|
|
|
Expected expected = "";
|
|
|
|
|
bool parse_res = parser.parse(pos, end, ast, expected);
|
2014-07-10 10:16:50 +00:00
|
|
|
|
|
2014-07-11 13:34:12 +00:00
|
|
|
|
/// Распарсенный запрос должен заканчиваться на конец входных данных или на точку с запятой.
|
|
|
|
|
if (!parse_res || (pos != end && *pos != ';'))
|
|
|
|
|
throw Exception(getSyntaxErrorMessage(parse_res, begin, end, pos, expected, "in file " + metadata_path),
|
|
|
|
|
DB::ErrorCodes::SYNTAX_ERROR);
|
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
|
|
|
|
|
2014-07-11 08:12:03 +00:00
|
|
|
|
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(columns);
|
|
|
|
|
*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);
|
|
|
|
|
}
|
2014-07-11 08:12:03 +00:00
|
|
|
|
|
|
|
|
|
Poco::File(metadata_temp_path).renameTo(metadata_path);
|
2014-07-10 10:16:50 +00:00
|
|
|
|
}
|