mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
dbms: development [#CONV-2944].
This commit is contained in:
parent
67b02812f6
commit
26c9af828e
@ -13,8 +13,15 @@ namespace DB
|
||||
class InterpreterCreateQuery
|
||||
{
|
||||
public:
|
||||
InterpreterCreateQuery(ASTPtr query_ptr_, Context & context_, size_t max_block_size_ = DEFAULT_BLOCK_SIZE);
|
||||
|
||||
/// Добавляет созданную таблицу в контекст, а также возвращает её.
|
||||
StoragePtr execute(ASTPtr query, Context & context);
|
||||
StoragePtr execute();
|
||||
|
||||
private:
|
||||
ASTPtr query_ptr;
|
||||
Context context;
|
||||
size_t max_block_size;
|
||||
};
|
||||
|
||||
|
||||
|
@ -21,6 +21,7 @@ public:
|
||||
ASTPtr storage;
|
||||
String as_database;
|
||||
String as_table;
|
||||
ASTPtr select;
|
||||
|
||||
ASTCreateQuery() {}
|
||||
ASTCreateQuery(StringRange range_) : IAST(range_), attach(false) {}
|
||||
|
@ -27,6 +27,15 @@ protected:
|
||||
};
|
||||
|
||||
|
||||
/** ENGINE = name. */
|
||||
class ParserEngine : public IParserBase
|
||||
{
|
||||
protected:
|
||||
String getName() { return "ENGINE"; }
|
||||
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected);
|
||||
};
|
||||
|
||||
|
||||
/** Запрос типа такого:
|
||||
* CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name
|
||||
* (
|
||||
@ -36,7 +45,10 @@ protected:
|
||||
* ) ENGINE = engine
|
||||
*
|
||||
* Или:
|
||||
* CREATE TABLE [db.]name AS [db2.]name2
|
||||
* CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name AS [db2.]name2
|
||||
*
|
||||
* Или:
|
||||
* CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name AS SELECT ...
|
||||
*/
|
||||
class ParserCreateQuery : public IParserBase
|
||||
{
|
||||
|
@ -1,11 +1,14 @@
|
||||
#include <Poco/FileStream.h>
|
||||
|
||||
#include <DB/DataStreams/copyData.h>
|
||||
|
||||
#include <DB/Parsers/ASTCreateQuery.h>
|
||||
#include <DB/Parsers/ASTNameTypePair.h>
|
||||
|
||||
#include <DB/Storages/StorageLog.h>
|
||||
#include <DB/Storages/StorageSystemNumbers.h>
|
||||
|
||||
#include <DB/Interpreters/InterpreterSelectQuery.h>
|
||||
#include <DB/Interpreters/InterpreterCreateQuery.h>
|
||||
|
||||
|
||||
@ -13,9 +16,15 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
StoragePtr InterpreterCreateQuery::execute(ASTPtr query, Context & context)
|
||||
InterpreterCreateQuery::InterpreterCreateQuery(ASTPtr query_ptr_, Context & context_, size_t max_block_size_)
|
||||
: query_ptr(query_ptr_), context(context_), max_block_size(max_block_size_)
|
||||
{
|
||||
ASTCreateQuery & create = dynamic_cast<ASTCreateQuery &>(*query);
|
||||
}
|
||||
|
||||
|
||||
StoragePtr InterpreterCreateQuery::execute()
|
||||
{
|
||||
ASTCreateQuery & create = dynamic_cast<ASTCreateQuery &>(*query_ptr);
|
||||
|
||||
String database_name = create.database.empty() ? context.current_database : create.database;
|
||||
String table_name = create.table;
|
||||
@ -43,6 +52,11 @@ StoragePtr InterpreterCreateQuery::execute(ASTPtr query, Context & context)
|
||||
|| (*context.databases)[as_database_name].end() == (*context.databases)[as_database_name].find(as_table_name)))
|
||||
throw Exception("Table " + as_database_name + "." + as_table_name + " doesn't exist.", ErrorCodes::UNKNOWN_TABLE);
|
||||
|
||||
SharedPtr<InterpreterSelectQuery> interpreter_select;
|
||||
if (create.select)
|
||||
interpreter_select = new InterpreterSelectQuery(create.select, context, max_block_size);
|
||||
|
||||
/// Получаем список столбцов
|
||||
if (create.columns)
|
||||
{
|
||||
ASTExpressionList & columns_list = dynamic_cast<ASTExpressionList &>(*create.columns);
|
||||
@ -55,8 +69,15 @@ StoragePtr InterpreterCreateQuery::execute(ASTPtr query, Context & context)
|
||||
}
|
||||
else if (!create.as_table.empty())
|
||||
columns = new NamesAndTypes((*context.databases)[as_database_name][as_table_name]->getColumns());
|
||||
else if (create.select)
|
||||
{
|
||||
Block sample = interpreter_select->getSampleBlock();
|
||||
columns = new NamesAndTypes;
|
||||
for (size_t i = 0; i < sample.columns(); ++i)
|
||||
(*columns)[sample.getByPosition(i).name] = sample.getByPosition(i).type;
|
||||
}
|
||||
else
|
||||
throw Exception("Incorrect CREATE query: required list of column descriptions or AS section.", ErrorCodes::INCORRECT_QUERY);
|
||||
throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
/// Выбор нужного движка таблицы
|
||||
|
||||
@ -99,6 +120,10 @@ StoragePtr InterpreterCreateQuery::execute(ASTPtr query, Context & context)
|
||||
(*context.databases)[database_name][table_name] = res;
|
||||
}
|
||||
|
||||
/// Если запрос CREATE SELECT, то вставим в таблицу данные
|
||||
if (create.select)
|
||||
copyData(*interpreter_select->execute(), *res->write(query_ptr));
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -32,8 +32,8 @@ void InterpreterQuery::execute(WriteBuffer & ostr, SharedPtr<ReadBuffer> remaini
|
||||
}
|
||||
else if (dynamic_cast<ASTCreateQuery *>(&*query_ptr))
|
||||
{
|
||||
InterpreterCreateQuery interpreter;
|
||||
interpreter.execute(query_ptr, context);
|
||||
InterpreterCreateQuery interpreter(query_ptr, context, max_block_size);
|
||||
interpreter.execute();
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown type of query: " + query_ptr->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
|
||||
|
@ -35,8 +35,8 @@ static void executeCreateQuery(const String & query, Context & context, const St
|
||||
ast_create_query.attach = true;
|
||||
ast_create_query.database = database;
|
||||
|
||||
InterpreterCreateQuery interpreter;
|
||||
interpreter.execute(ast, context);
|
||||
InterpreterCreateQuery interpreter(ast, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
|
||||
|
||||
|
@ -98,8 +98,8 @@ int main(int argc, char ** argv)
|
||||
(*context.databases)["test"];
|
||||
context.current_database = "test";
|
||||
|
||||
DB::InterpreterCreateQuery interpreter;
|
||||
interpreter.execute(ast, context);
|
||||
DB::InterpreterCreateQuery interpreter(ast, context);
|
||||
interpreter.execute();
|
||||
}
|
||||
catch (const DB::Exception & e)
|
||||
{
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <DB/Parsers/CommonParsers.h>
|
||||
#include <DB/Parsers/ExpressionListParsers.h>
|
||||
#include <DB/Parsers/ParserCreateQuery.h>
|
||||
#include <DB/Parsers/ParserSelectQuery.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -66,6 +67,35 @@ bool ParserNameTypePair::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & e
|
||||
}
|
||||
|
||||
|
||||
bool ParserEngine::parseImpl(Pos & pos, Pos end, ASTPtr & storage, String & expected)
|
||||
{
|
||||
ParserWhiteSpaceOrComments ws;
|
||||
ParserString s_engine("ENGINE", true);
|
||||
ParserString s_eq("=");
|
||||
ParserIdentifierWithOptionalParameters storage_p;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
/// ENGINE
|
||||
if (s_engine.ignore(pos, end, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!s_eq.ignore(pos, end, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!storage_p.parse(pos, end, storage, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & expected)
|
||||
{
|
||||
Pos begin = pos;
|
||||
@ -77,15 +107,14 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & ex
|
||||
ParserString s_dot(".");
|
||||
ParserString s_lparen("(");
|
||||
ParserString s_rparen(")");
|
||||
ParserString s_engine("ENGINE", true);
|
||||
ParserString s_eq("=");
|
||||
ParserString s_if("IF", true);
|
||||
ParserString s_not("NOT", true);
|
||||
ParserString s_exists("EXISTS", true);
|
||||
ParserString s_as("AS", true);
|
||||
ParserString s_select("SELECT", true);
|
||||
ParserEngine engine_p;
|
||||
ParserIdentifier name_p;
|
||||
ParserList columns_p(new ParserNameTypePair, new ParserString(","), false);
|
||||
ParserIdentifierWithOptionalParameters storage_p;
|
||||
|
||||
ASTPtr database;
|
||||
ASTPtr table;
|
||||
@ -93,6 +122,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & ex
|
||||
ASTPtr storage;
|
||||
ASTPtr as_database;
|
||||
ASTPtr as_table;
|
||||
ASTPtr select;
|
||||
bool attach = false;
|
||||
bool if_not_exists = false;
|
||||
|
||||
@ -145,12 +175,28 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & ex
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!engine_p.parse(pos, end, storage, expected))
|
||||
return false;
|
||||
}
|
||||
else if (s_as.ignore(pos, end, expected))
|
||||
{
|
||||
/// Или AS другая таблица
|
||||
if (!engine_p.parse(pos, end, storage, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
/// AS SELECT ...
|
||||
Pos before_select = pos;
|
||||
if (s_select.ignore(pos, end, expected))
|
||||
{
|
||||
pos = before_select;
|
||||
ParserSelectQuery select_p;
|
||||
select_p.parse(pos, end, select, expected);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// AS [db.]table
|
||||
if (!name_p.parse(pos, end, as_table, expected))
|
||||
return false;
|
||||
|
||||
@ -165,25 +211,10 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & ex
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
return false;
|
||||
|
||||
/// ENGINE
|
||||
if (s_engine.ignore(pos, end, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!s_eq.ignore(pos, end, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!storage_p.parse(pos, end, storage, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
ASTCreateQuery * query = new ASTCreateQuery(StringRange(begin, pos));
|
||||
node = query;
|
||||
|
||||
@ -198,9 +229,13 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & ex
|
||||
query->as_database = dynamic_cast<ASTIdentifier &>(*as_database).name;
|
||||
if (as_table)
|
||||
query->as_table = dynamic_cast<ASTIdentifier &>(*as_table).name;
|
||||
query->select = select;
|
||||
|
||||
query->children.push_back(columns);
|
||||
if (storage)
|
||||
query->children.push_back(storage);
|
||||
if (select)
|
||||
query->children.push_back(select);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user