mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 09:32:06 +00:00
dbms: development [#CONV-2944].
This commit is contained in:
parent
b1b0d3836a
commit
169ef6bf2e
@ -85,6 +85,7 @@ namespace ErrorCodes
|
||||
CANNOT_CLOSE_FILE,
|
||||
UNKNOWN_TYPE_OF_QUERY,
|
||||
INCORRECT_FILE_NAME,
|
||||
INCORRECT_QUERY,
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -11,6 +11,7 @@
|
||||
#include <DB/Functions/IFunction.h>
|
||||
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <DB/DataTypes/DataTypeFactory.h>
|
||||
#include <DB/Storages/StorageFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -38,6 +39,7 @@ struct Context
|
||||
SharedPtr<Functions> functions; /// Обычные функции.
|
||||
AggregateFunctionFactoryPtr aggregate_function_factory; /// Агрегатные функции.
|
||||
DataTypeFactoryPtr data_type_factory; /// Типы данных.
|
||||
StorageFactoryPtr storage_factory; /// Движки таблиц.
|
||||
NamesAndTypes columns; /// Столбцы текущей обрабатываемой таблицы.
|
||||
|
||||
SharedPtr<Poco::FastMutex> mutex; /// Для доступа и модификации разделяемых объектов.
|
||||
|
@ -19,6 +19,8 @@ public:
|
||||
String table;
|
||||
ASTPtr columns;
|
||||
ASTPtr storage;
|
||||
String as_database;
|
||||
String as_table;
|
||||
|
||||
ASTCreateQuery() {}
|
||||
ASTCreateQuery(StringRange range_) : IAST(range_), attach(false) {}
|
||||
|
27
dbms/include/DB/Storages/StorageFactory.h
Normal file
27
dbms/include/DB/Storages/StorageFactory.h
Normal file
@ -0,0 +1,27 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Storages/IStorage.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
using Poco::SharedPtr;
|
||||
|
||||
|
||||
/** Позволяет создать таблицу по имени движка.
|
||||
*/
|
||||
class StorageFactory
|
||||
{
|
||||
public:
|
||||
StoragePtr get(
|
||||
const String & name,
|
||||
const String & data_path,
|
||||
const String & table_name,
|
||||
SharedPtr<NamesAndTypes> columns) const;
|
||||
};
|
||||
|
||||
typedef SharedPtr<StorageFactory> StorageFactoryPtr;
|
||||
|
||||
|
||||
}
|
@ -19,6 +19,9 @@ StoragePtr InterpreterCreateQuery::execute(ASTPtr query, Context & context)
|
||||
|
||||
String database_name = create.database.empty() ? context.current_database : create.database;
|
||||
String table_name = create.table;
|
||||
String as_database_name = create.as_database.empty() ? context.current_database : create.as_database;
|
||||
String as_table_name = create.as_table;
|
||||
|
||||
SharedPtr<NamesAndTypes> columns = new NamesAndTypes;
|
||||
String data_path = context.path + "data/" + database_name + "/"; /// TODO: эскейпинг
|
||||
String metadata_path = context.path + "metadata/" + database_name + "/" + table_name + ".sql";
|
||||
@ -34,39 +37,38 @@ StoragePtr InterpreterCreateQuery::execute(ASTPtr query, Context & context)
|
||||
throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
||||
}
|
||||
}
|
||||
|
||||
ASTExpressionList & columns_list = dynamic_cast<ASTExpressionList &>(*create.columns);
|
||||
for (ASTs::iterator it = columns_list.children.begin(); it != columns_list.children.end(); ++it)
|
||||
{
|
||||
ASTNameTypePair & name_and_type_pair = dynamic_cast<ASTNameTypePair &>(**it);
|
||||
StringRange type_range = name_and_type_pair.type->range;
|
||||
(*columns)[name_and_type_pair.name] = context.data_type_factory->get(String(type_range.first, type_range.second - type_range.first));
|
||||
}
|
||||
|
||||
ASTFunction & storage_expr = dynamic_cast<ASTFunction &>(*create.storage);
|
||||
String storage_str(storage_expr.range.first, storage_expr.range.second - storage_expr.range.first);
|
||||
StoragePtr res;
|
||||
if (!create.as_table.empty()
|
||||
&& ((*context.databases).end() == (*context.databases).find(as_database_name)
|
||||
|| (*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);
|
||||
|
||||
if (create.columns)
|
||||
{
|
||||
ASTExpressionList & columns_list = dynamic_cast<ASTExpressionList &>(*create.columns);
|
||||
for (ASTs::iterator it = columns_list.children.begin(); it != columns_list.children.end(); ++it)
|
||||
{
|
||||
ASTNameTypePair & name_and_type_pair = dynamic_cast<ASTNameTypePair &>(**it);
|
||||
StringRange type_range = name_and_type_pair.type->range;
|
||||
(*columns)[name_and_type_pair.name] = context.data_type_factory->get(String(type_range.first, type_range.second - type_range.first));
|
||||
}
|
||||
}
|
||||
else if (!create.as_table.empty())
|
||||
columns = new NamesAndTypes((*context.databases)[as_database_name][as_table_name]->getColumns());
|
||||
else
|
||||
throw Exception("Incorrect CREATE query: required list of column descriptions or AS section.", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
/// Выбор нужного движка таблицы
|
||||
|
||||
if (storage_expr.name == "Log")
|
||||
{
|
||||
if (storage_expr.arguments)
|
||||
throw Exception("Storage Log doesn't allow parameters", ErrorCodes::STORAGE_DOESNT_ALLOW_PARAMETERS);
|
||||
|
||||
res = new StorageLog(data_path, table_name, columns);
|
||||
}
|
||||
else if (storage_expr.name == "SystemNumbers")
|
||||
{
|
||||
if (storage_expr.arguments)
|
||||
throw Exception("Storage SystemNumbers doesn't allow parameters", ErrorCodes::STORAGE_DOESNT_ALLOW_PARAMETERS);
|
||||
if (columns->size() != 1 || columns->begin()->first != "number" || columns->begin()->second->getName() != "UInt64")
|
||||
throw Exception("Storage SystemNumbers only allows one column with name 'number' and type 'UInt64'", ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
res = new StorageSystemNumbers(table_name);
|
||||
}
|
||||
String storage_name;
|
||||
if (create.storage)
|
||||
storage_name = dynamic_cast<ASTFunction &>(*create.storage).name;
|
||||
else if (!create.as_table.empty())
|
||||
storage_name = (*context.databases)[as_database_name][as_table_name]->getName();
|
||||
else
|
||||
throw Exception("Unknown storage " + storage_str, ErrorCodes::UNKNOWN_STORAGE);
|
||||
throw Exception("Incorrect CREATE query: required ENGINE or AS section.", ErrorCodes::INCORRECT_QUERY);
|
||||
|
||||
StoragePtr res = context.storage_factory->get(storage_name, data_path, table_name, columns);
|
||||
|
||||
/// Проверка наличия метаданных таблицы на диске и создание метаданных
|
||||
|
||||
@ -82,7 +84,13 @@ StoragePtr InterpreterCreateQuery::execute(ASTPtr query, Context & context)
|
||||
else
|
||||
{
|
||||
Poco::FileOutputStream metadata_file(metadata_path);
|
||||
metadata_file << String(create.range.first, create.range.second - create.range.first) << std::endl;
|
||||
metadata_file << "ATTACH TABLE " << database_name << "." << table_name << "\n"
|
||||
<< "(\n";
|
||||
|
||||
for (NamesAndTypes::const_iterator it = columns->begin(); it != columns->end(); ++it)
|
||||
metadata_file << (it != columns->begin() ? ",\n" : "") << "\t" << it->first << " " << it->second->getName();
|
||||
|
||||
metadata_file << "\n) ENGINE = " << storage_name << "\n";
|
||||
}
|
||||
|
||||
{
|
||||
|
@ -13,6 +13,7 @@
|
||||
#include <DB/Storages/StorageLog.h>
|
||||
#include <DB/Storages/StorageSystemNumbers.h>
|
||||
#include <DB/Storages/StorageSystemOne.h>
|
||||
#include <DB/Storages/StorageFactory.h>
|
||||
|
||||
#include <DB/DataStreams/TabSeparatedRowOutputStream.h>
|
||||
#include <DB/DataStreams/copyData.h>
|
||||
@ -188,6 +189,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
context.aggregate_function_factory = new DB::AggregateFunctionFactory;
|
||||
context.data_type_factory = new DB::DataTypeFactory;
|
||||
context.storage_factory = new DB::StorageFactory;
|
||||
|
||||
DB::loadMetadata(context);
|
||||
|
||||
|
@ -82,6 +82,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & ex
|
||||
ParserString s_if("IF", true);
|
||||
ParserString s_not("NOT", true);
|
||||
ParserString s_exists("EXISTS", true);
|
||||
ParserString s_as("AS", true);
|
||||
ParserIdentifier name_p;
|
||||
ParserList columns_p(new ParserNameTypePair, new ParserString(","), false);
|
||||
ParserIdentifierWithOptionalParameters storage_p;
|
||||
@ -90,6 +91,8 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & ex
|
||||
ASTPtr table;
|
||||
ASTPtr columns;
|
||||
ASTPtr storage;
|
||||
ASTPtr as_database;
|
||||
ASTPtr as_table;
|
||||
bool attach = false;
|
||||
bool if_not_exists = false;
|
||||
|
||||
@ -128,35 +131,58 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & ex
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
if (!s_lparen.ignore(pos, end, expected))
|
||||
/// Список столбцов
|
||||
if (s_lparen.ignore(pos, end, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!columns_p.parse(pos, end, columns, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!s_rparen.ignore(pos, end, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
else if (s_as.ignore(pos, end, expected))
|
||||
{
|
||||
/// Или AS другая таблица
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!name_p.parse(pos, end, as_table, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (s_dot.ignore(pos, end, expected))
|
||||
{
|
||||
as_database = as_table;
|
||||
if (!name_p.parse(pos, end, as_table, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
}
|
||||
else
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
/// ENGINE
|
||||
if (s_engine.ignore(pos, end, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!columns_p.parse(pos, end, columns, expected))
|
||||
return false;
|
||||
if (!s_eq.ignore(pos, end, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!s_rparen.ignore(pos, end, expected))
|
||||
return false;
|
||||
if (!storage_p.parse(pos, end, storage, expected))
|
||||
return false;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
if (!s_engine.ignore(pos, end, expected))
|
||||
return false;
|
||||
|
||||
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);
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
ASTCreateQuery * query = new ASTCreateQuery(StringRange(begin, pos));
|
||||
node = query;
|
||||
@ -168,6 +194,10 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, String & ex
|
||||
query->table = dynamic_cast<ASTIdentifier &>(*table).name;
|
||||
query->columns = columns;
|
||||
query->storage = storage;
|
||||
if (as_database)
|
||||
query->as_database = dynamic_cast<ASTIdentifier &>(*as_database).name;
|
||||
if (as_table)
|
||||
query->as_table = dynamic_cast<ASTIdentifier &>(*as_table).name;
|
||||
|
||||
query->children.push_back(columns);
|
||||
query->children.push_back(storage);
|
||||
|
40
dbms/src/Storages/StorageFactory.cpp
Normal file
40
dbms/src/Storages/StorageFactory.cpp
Normal file
@ -0,0 +1,40 @@
|
||||
#include <DB/Storages/StorageLog.h>
|
||||
#include <DB/Storages/StorageSystemNumbers.h>
|
||||
#include <DB/Storages/StorageSystemOne.h>
|
||||
#include <DB/Storages/StorageFactory.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
StoragePtr StorageFactory::get(
|
||||
const String & name,
|
||||
const String & data_path,
|
||||
const String & table_name,
|
||||
SharedPtr<NamesAndTypes> columns) const
|
||||
{
|
||||
if (name == "Log")
|
||||
return new StorageLog(data_path, table_name, columns);
|
||||
else if (name == "SystemNumbers")
|
||||
{
|
||||
if (columns->size() != 1 || columns->begin()->first != "number" || columns->begin()->second->getName() != "UInt64")
|
||||
throw Exception("Storage SystemNumbers only allows one column with name 'number' and type 'UInt64'",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return new StorageSystemNumbers(table_name);
|
||||
}
|
||||
else if (name == "SystemOne")
|
||||
{
|
||||
if (columns->size() != 1 || columns->begin()->first != "dummy" || columns->begin()->second->getName() != "UInt8")
|
||||
throw Exception("Storage SystemOne only allows one column with name 'dummy' and type 'UInt8'",
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
|
||||
return new StorageSystemOne(table_name);
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown storage " + name, ErrorCodes::UNKNOWN_STORAGE);
|
||||
}
|
||||
|
||||
|
||||
}
|
Loading…
Reference in New Issue
Block a user