mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-11 17:02:25 +00:00
Remove non-alter move command.
This commit is contained in:
parent
52bca488dc
commit
2925657f5e
@ -6,7 +6,6 @@
|
|||||||
#include <Parsers/ASTKillQueryQuery.h>
|
#include <Parsers/ASTKillQueryQuery.h>
|
||||||
#include <Parsers/ASTOptimizeQuery.h>
|
#include <Parsers/ASTOptimizeQuery.h>
|
||||||
#include <Parsers/ASTRenameQuery.h>
|
#include <Parsers/ASTRenameQuery.h>
|
||||||
#include <Parsers/ASTMoveQuery.h>
|
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
#include <Parsers/ASTSelectWithUnionQuery.h>
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||||
#include <Parsers/ASTSetQuery.h>
|
#include <Parsers/ASTSetQuery.h>
|
||||||
@ -26,7 +25,6 @@
|
|||||||
#include <Interpreters/InterpreterFactory.h>
|
#include <Interpreters/InterpreterFactory.h>
|
||||||
#include <Interpreters/InterpreterInsertQuery.h>
|
#include <Interpreters/InterpreterInsertQuery.h>
|
||||||
#include <Interpreters/InterpreterKillQueryQuery.h>
|
#include <Interpreters/InterpreterKillQueryQuery.h>
|
||||||
#include <Interpreters/InterpreterMoveQuery.h>
|
|
||||||
#include <Interpreters/InterpreterOptimizeQuery.h>
|
#include <Interpreters/InterpreterOptimizeQuery.h>
|
||||||
#include <Interpreters/InterpreterRenameQuery.h>
|
#include <Interpreters/InterpreterRenameQuery.h>
|
||||||
#include <Interpreters/InterpreterSelectQuery.h>
|
#include <Interpreters/InterpreterSelectQuery.h>
|
||||||
@ -115,10 +113,6 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
|
|||||||
throwIfNoAccess(context);
|
throwIfNoAccess(context);
|
||||||
return std::make_unique<InterpreterRenameQuery>(query, context);
|
return std::make_unique<InterpreterRenameQuery>(query, context);
|
||||||
}
|
}
|
||||||
else if (query->as<ASTMoveQuery>())
|
|
||||||
{
|
|
||||||
return std::make_unique<InterpreterMoveQuery>(query, context);
|
|
||||||
}
|
|
||||||
else if (query->as<ASTShowTablesQuery>())
|
else if (query->as<ASTShowTablesQuery>())
|
||||||
{
|
{
|
||||||
return std::make_unique<InterpreterShowTablesQuery>(query, context);
|
return std::make_unique<InterpreterShowTablesQuery>(query, context);
|
||||||
|
@ -1,55 +0,0 @@
|
|||||||
#include <Parsers/ASTMoveQuery.h>
|
|
||||||
#include <Databases/IDatabase.h>
|
|
||||||
#include <Interpreters/Context.h>
|
|
||||||
#include <Interpreters/InterpreterMoveQuery.h>
|
|
||||||
#include <Storages/IStorage.h>
|
|
||||||
#include <Interpreters/DDLWorker.h>
|
|
||||||
#include <Common/typeid_cast.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
|
|
||||||
InterpreterMoveQuery::InterpreterMoveQuery(const ASTPtr & query_ptr_, const Context & context_)
|
|
||||||
: query_ptr(query_ptr_), context(context_)
|
|
||||||
{
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
BlockIO InterpreterMoveQuery::execute()
|
|
||||||
{
|
|
||||||
const auto & move = query_ptr->as<ASTMoveQuery &>();
|
|
||||||
|
|
||||||
if (!move.cluster.empty())
|
|
||||||
{
|
|
||||||
///@TODO_IGR ASK WHAT SHOULD I DO HERE
|
|
||||||
// NameSet databases;
|
|
||||||
// for (const auto & elem : rename.elements)
|
|
||||||
// {
|
|
||||||
// databases.emplace(elem.from.database);
|
|
||||||
// databases.emplace(elem.to.database);
|
|
||||||
// }
|
|
||||||
//
|
|
||||||
// return executeDDLQueryOnCluster(query_ptr, context, std::move(databases));
|
|
||||||
}
|
|
||||||
|
|
||||||
String path = context.getPath();
|
|
||||||
String current_database = context.getCurrentDatabase();
|
|
||||||
|
|
||||||
auto table = context.tryGetTable(move.table.database, move.table.table);
|
|
||||||
// if (!table) ///@TODO_IGR
|
|
||||||
|
|
||||||
TableStructureWriteLockHolder table_lock(table->lockExclusively(context.getCurrentQueryId()));
|
|
||||||
|
|
||||||
decltype(context.getLock()) lock;
|
|
||||||
|
|
||||||
lock = context.getLock();
|
|
||||||
|
|
||||||
table->move(move.part_name, move.destination_disk_name);
|
|
||||||
|
|
||||||
return {};
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
}
|
|
@ -1,27 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <Interpreters/IInterpreter.h>
|
|
||||||
#include <Parsers/IAST_fwd.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
class Context;
|
|
||||||
|
|
||||||
/** Allows you a in the table.
|
|
||||||
*/
|
|
||||||
class InterpreterMoveQuery : public IInterpreter
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
InterpreterMoveQuery(const ASTPtr & query_ptr_, const Context & context_);
|
|
||||||
|
|
||||||
BlockIO execute() override;
|
|
||||||
|
|
||||||
private:
|
|
||||||
ASTPtr query_ptr;
|
|
||||||
|
|
||||||
const Context & context;
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
@ -130,19 +130,18 @@ void ASTAlterCommand::formatImpl(
|
|||||||
<< (part ? "PART " : "PARTITION ") << (settings.hilite ? hilite_none : "");
|
<< (part ? "PART " : "PARTITION ") << (settings.hilite ? hilite_none : "");
|
||||||
partition->formatImpl(settings, state, frame);
|
partition->formatImpl(settings, state, frame);
|
||||||
settings.ostr << "TO ";
|
settings.ostr << "TO ";
|
||||||
switch (space_to_move)
|
switch (move_destination_type)
|
||||||
{
|
{
|
||||||
case SpaceToMove::DISK:
|
case MoveDestinationType::DISK:
|
||||||
settings.ostr << "DISK ";
|
settings.ostr << "DISK ";
|
||||||
break;
|
break;
|
||||||
case SpaceToMove::VOLUME:
|
case MoveDestinationType::VOLUME:
|
||||||
settings.ostr << "VOLUME ";
|
settings.ostr << "VOLUME ";
|
||||||
break;
|
break;
|
||||||
case SpaceToMove::NONE:
|
case MoveDestinationType::NONE:
|
||||||
default:
|
throw Exception("Unexpected MOVE destination (NONE)", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||||
break;
|
|
||||||
}
|
}
|
||||||
settings.ostr << space_to_move_name;
|
settings.ostr << move_destination_name;
|
||||||
}
|
}
|
||||||
else if (type == ASTAlterCommand::REPLACE_PARTITION)
|
else if (type == ASTAlterCommand::REPLACE_PARTITION)
|
||||||
{
|
{
|
||||||
|
@ -99,16 +99,16 @@ public:
|
|||||||
|
|
||||||
bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN
|
bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN
|
||||||
|
|
||||||
enum SpaceToMove
|
enum MoveDestinationType
|
||||||
{
|
{
|
||||||
DISK,
|
DISK,
|
||||||
VOLUME,
|
VOLUME,
|
||||||
NONE,
|
NONE,
|
||||||
};
|
};
|
||||||
|
|
||||||
SpaceToMove space_to_move = SpaceToMove::NONE;
|
MoveDestinationType move_destination_type = MoveDestinationType::NONE;
|
||||||
|
|
||||||
String space_to_move_name;
|
String move_destination_name;
|
||||||
|
|
||||||
/** For FETCH PARTITION - the path in ZK to the shard, from which to download the partition.
|
/** For FETCH PARTITION - the path in ZK to the shard, from which to download the partition.
|
||||||
*/
|
*/
|
||||||
|
@ -1,64 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <Parsers/IAST.h>
|
|
||||||
#include <Parsers/ASTQueryWithOutput.h>
|
|
||||||
#include <Parsers/ASTQueryWithOnCluster.h>
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
|
|
||||||
/** MOVE query
|
|
||||||
*/
|
|
||||||
class ASTMoveQuery : public ASTQueryWithOutput, public ASTQueryWithOnCluster
|
|
||||||
{
|
|
||||||
public:
|
|
||||||
struct Table
|
|
||||||
{
|
|
||||||
String database;
|
|
||||||
String table;
|
|
||||||
};
|
|
||||||
|
|
||||||
Table table;
|
|
||||||
String part_name;
|
|
||||||
String destination_disk_name;
|
|
||||||
|
|
||||||
/** Get the text that identifies this element. */
|
|
||||||
String getID(char) const override { return "Move"; }
|
|
||||||
|
|
||||||
ASTPtr clone() const override
|
|
||||||
{
|
|
||||||
auto res = std::make_shared<ASTMoveQuery>(*this);
|
|
||||||
cloneOutputOptions(*res);
|
|
||||||
return res;
|
|
||||||
}
|
|
||||||
|
|
||||||
ASTPtr getRewrittenASTWithoutOnCluster(const std::string & /*new_database*/) const override
|
|
||||||
{
|
|
||||||
///@TODO_IGR ASK
|
|
||||||
auto query_ptr = clone();
|
|
||||||
auto & query = query_ptr->as<ASTMoveQuery &>();
|
|
||||||
|
|
||||||
query.cluster.clear();
|
|
||||||
|
|
||||||
query.table = table;
|
|
||||||
query.part_name = part_name;
|
|
||||||
query.destination_disk_name = destination_disk_name;
|
|
||||||
|
|
||||||
return query_ptr;
|
|
||||||
}
|
|
||||||
|
|
||||||
protected:
|
|
||||||
void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override
|
|
||||||
{
|
|
||||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << "TABLE"
|
|
||||||
<< (settings.hilite ? hilite_keyword : "")
|
|
||||||
<< (table.database.empty() ? backQuoteIfNeed(table.database) + "." : "") << backQuoteIfNeed(table.table)
|
|
||||||
<< (settings.hilite ? hilite_keyword : "") << "MOVE PART"
|
|
||||||
<< (settings.hilite ? hilite_keyword : "") << part_name
|
|
||||||
<< (settings.hilite ? hilite_keyword : "") << "TO"
|
|
||||||
<< (settings.hilite ? hilite_none : "") << destination_disk_name;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
@ -203,9 +203,9 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
|||||||
command->type = ASTAlterCommand::MOVE_PARTITION;
|
command->type = ASTAlterCommand::MOVE_PARTITION;
|
||||||
|
|
||||||
if (s_to_disk.ignore(pos))
|
if (s_to_disk.ignore(pos))
|
||||||
command->space_to_move = ASTAlterCommand::SpaceToMove::DISK;
|
command->move_destination_type = ASTAlterCommand::MoveDestinationType::DISK;
|
||||||
else if (s_to_volume.ignore(pos))
|
else if (s_to_volume.ignore(pos))
|
||||||
command->space_to_move = ASTAlterCommand::SpaceToMove::VOLUME;
|
command->move_destination_type = ASTAlterCommand::MoveDestinationType::VOLUME;
|
||||||
else
|
else
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
@ -213,7 +213,7 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
|||||||
if (!parser_string_literal.parse(pos, ast_space_name, expected))
|
if (!parser_string_literal.parse(pos, ast_space_name, expected))
|
||||||
return false;
|
return false;
|
||||||
|
|
||||||
command->space_to_move_name = ast_space_name->as<ASTLiteral &>().value.get<const String &>();
|
command->move_destination_name = ast_space_name->as<ASTLiteral &>().value.get<const String &>();
|
||||||
}
|
}
|
||||||
else if (s_fetch_partition.ignore(pos, expected))
|
else if (s_fetch_partition.ignore(pos, expected))
|
||||||
{
|
{
|
||||||
|
@ -1,96 +0,0 @@
|
|||||||
#include <Parsers/ASTIdentifier.h>
|
|
||||||
#include <Parsers/ASTMoveQuery.h>
|
|
||||||
|
|
||||||
#include <Parsers/CommonParsers.h>
|
|
||||||
#include <Parsers/ParserMoveQuery.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
|
|
||||||
/// Parse database.table or table.
|
|
||||||
static bool parseDatabaseAndTable(
|
|
||||||
ASTMoveQuery::Table & db_and_table, IParser::Pos & pos, Expected & expected)
|
|
||||||
{
|
|
||||||
ParserIdentifier name_p;
|
|
||||||
ParserToken s_dot(TokenType::Dot);
|
|
||||||
|
|
||||||
ASTPtr database;
|
|
||||||
ASTPtr table;
|
|
||||||
|
|
||||||
if (!name_p.parse(pos, table, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
if (s_dot.ignore(pos, expected))
|
|
||||||
{
|
|
||||||
database = table;
|
|
||||||
if (!name_p.parse(pos, table, expected))
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
db_and_table.database.clear();
|
|
||||||
getIdentifierName(database, db_and_table.database);
|
|
||||||
getIdentifierName(table, db_and_table.table);
|
|
||||||
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
bool ParserMoveQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
|
||||||
{
|
|
||||||
ParserIdentifier name_p;
|
|
||||||
ParserKeyword s_table("TABLE");
|
|
||||||
ParserKeyword s_move_part("MOVE PART");
|
|
||||||
ParserKeyword s_to("TO");
|
|
||||||
ParserToken s_comma(TokenType::Comma);
|
|
||||||
|
|
||||||
if (!s_table.ignore(pos, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
ASTMoveQuery::Table table;
|
|
||||||
|
|
||||||
if (!parseDatabaseAndTable(table, pos, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
if (!s_move_part.ignore(pos, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
String part_name;
|
|
||||||
String destination_disk_name;
|
|
||||||
|
|
||||||
ASTPtr part_name_ast_ptr;
|
|
||||||
ASTPtr destination_disk_name_ast_ptr;
|
|
||||||
|
|
||||||
if (!name_p.parse(pos, part_name_ast_ptr, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
if (!s_to.ignore(pos, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
if (!name_p.parse(pos, destination_disk_name_ast_ptr, expected))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
|
|
||||||
getIdentifierName(part_name_ast_ptr, part_name);
|
|
||||||
getIdentifierName(destination_disk_name_ast_ptr, destination_disk_name);
|
|
||||||
|
|
||||||
String cluster_str;
|
|
||||||
if (ParserKeyword{"ON"}.ignore(pos, expected))
|
|
||||||
{
|
|
||||||
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
auto query = std::make_shared<ASTMoveQuery>();
|
|
||||||
query->cluster = cluster_str;
|
|
||||||
node = query;
|
|
||||||
|
|
||||||
query->table = table;
|
|
||||||
query->part_name = part_name;
|
|
||||||
query->destination_disk_name = destination_disk_name;
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
}
|
|
@ -1,21 +0,0 @@
|
|||||||
#pragma once
|
|
||||||
|
|
||||||
#include <Parsers/IParserBase.h>
|
|
||||||
#include <Parsers/ExpressionElementParsers.h>
|
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
|
||||||
{
|
|
||||||
|
|
||||||
/** Query like this:
|
|
||||||
* RENAME TABLE [db.]name TO [db.]name, [db.]name TO [db.]name, ...
|
|
||||||
* (An arbitrary number of tables can be renamed.)
|
|
||||||
*/
|
|
||||||
class ParserMoveQuery : public IParserBase
|
|
||||||
{
|
|
||||||
protected:
|
|
||||||
const char * getName() const { return "MOVE query"; }
|
|
||||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
|
||||||
};
|
|
||||||
|
|
||||||
}
|
|
@ -7,7 +7,6 @@
|
|||||||
#include <Parsers/ParserCheckQuery.h>
|
#include <Parsers/ParserCheckQuery.h>
|
||||||
#include <Parsers/ParserCreateQuery.h>
|
#include <Parsers/ParserCreateQuery.h>
|
||||||
#include <Parsers/ParserRenameQuery.h>
|
#include <Parsers/ParserRenameQuery.h>
|
||||||
#include <Parsers/ParserMoveQuery.h>
|
|
||||||
#include <Parsers/ParserAlterQuery.h>
|
#include <Parsers/ParserAlterQuery.h>
|
||||||
#include <Parsers/ParserDropQuery.h>
|
#include <Parsers/ParserDropQuery.h>
|
||||||
#include <Parsers/ParserKillQueryQuery.h>
|
#include <Parsers/ParserKillQueryQuery.h>
|
||||||
@ -33,7 +32,6 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
|||||||
ParserCheckQuery check_p;
|
ParserCheckQuery check_p;
|
||||||
ParserOptimizeQuery optimize_p;
|
ParserOptimizeQuery optimize_p;
|
||||||
ParserKillQueryQuery kill_query_p;
|
ParserKillQueryQuery kill_query_p;
|
||||||
ParserMoveQuery move_p;
|
|
||||||
|
|
||||||
ASTPtr query;
|
ASTPtr query;
|
||||||
|
|
||||||
@ -59,8 +57,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
|
|||||||
|| drop_p.parse(pos, query, expected)
|
|| drop_p.parse(pos, query, expected)
|
||||||
|| check_p.parse(pos, query, expected)
|
|| check_p.parse(pos, query, expected)
|
||||||
|| kill_query_p.parse(pos, query, expected)
|
|| kill_query_p.parse(pos, query, expected)
|
||||||
|| optimize_p.parse(pos, query, expected)
|
|| optimize_p.parse(pos, query, expected);
|
||||||
|| move_p.parse(pos, query, expected);
|
|
||||||
|
|
||||||
if (!parsed)
|
if (!parsed)
|
||||||
return false;
|
return false;
|
||||||
|
@ -37,19 +37,19 @@ std::optional<PartitionCommand> PartitionCommand::parse(const ASTAlterCommand *
|
|||||||
res.type = MOVE_PARTITION;
|
res.type = MOVE_PARTITION;
|
||||||
res.partition = command_ast->partition;
|
res.partition = command_ast->partition;
|
||||||
res.part = command_ast->part;
|
res.part = command_ast->part;
|
||||||
switch (command_ast->space_to_move)
|
switch (command_ast->move_destination_type)
|
||||||
{
|
{
|
||||||
case ASTAlterCommand::SpaceToMove::DISK:
|
case ASTAlterCommand::MoveDestinationType::DISK:
|
||||||
res.space_to_move = PartitionCommand::SpaceToMove::DISK;
|
res.space_to_move = PartitionCommand::SpaceToMove::DISK;
|
||||||
break;
|
break;
|
||||||
case ASTAlterCommand::SpaceToMove::VOLUME:
|
case ASTAlterCommand::MoveDestinationType::VOLUME:
|
||||||
res.space_to_move = PartitionCommand::SpaceToMove::VOLUME;
|
res.space_to_move = PartitionCommand::SpaceToMove::VOLUME;
|
||||||
break;
|
break;
|
||||||
case ASTAlterCommand::SpaceToMove::NONE:
|
case ASTAlterCommand::MoveDestinationType::NONE:
|
||||||
res.space_to_move = PartitionCommand::SpaceToMove::NONE;
|
res.space_to_move = PartitionCommand::SpaceToMove::NONE;
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
res.space_to_move_name = command_ast->space_to_move_name;
|
res.space_to_move_name = command_ast->move_destination_name;
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
else if (command_ast->type == ASTAlterCommand::REPLACE_PARTITION)
|
else if (command_ast->type == ASTAlterCommand::REPLACE_PARTITION)
|
||||||
|
Loading…
Reference in New Issue
Block a user