ClickHouse/dbms/src/Interpreters/InterpreterCreateQuery.cpp

438 lines
15 KiB
C++
Raw Normal View History

2011-11-05 23:31:19 +00:00
#include <Poco/File.h>
2011-10-24 12:10:59 +00:00
#include <Poco/FileStream.h>
2011-11-05 23:31:19 +00:00
#include <DB/Common/escapeForFileName.h>
2011-11-01 17:57:37 +00:00
#include <DB/IO/WriteBufferFromString.h>
#include <DB/IO/WriteHelpers.h>
2011-11-06 06:22:52 +00:00
#include <DB/DataStreams/MaterializingBlockInputStream.h>
2011-11-01 15:16:04 +00:00
#include <DB/DataStreams/copyData.h>
2011-08-18 20:33:20 +00:00
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/ASTNameTypePair.h>
#include <DB/Parsers/ASTColumnDeclaration.h>
2011-08-18 20:33:20 +00:00
#include <DB/Storages/StorageLog.h>
#include <DB/Storages/StorageSystemNumbers.h>
2012-08-20 19:21:04 +00:00
#include <DB/Parsers/ParserCreateQuery.h>
2012-05-22 20:18:45 +00:00
#include <DB/Parsers/formatAST.h>
2011-11-01 15:16:04 +00:00
#include <DB/Interpreters/InterpreterSelectQuery.h>
2011-08-18 20:33:20 +00:00
#include <DB/Interpreters/InterpreterCreateQuery.h>
#include <DB/Interpreters/ExpressionAnalyzer.h>
2014-07-10 11:13:45 +00:00
#include <DB/DataTypes/DataTypeNested.h>
2011-08-18 20:33:20 +00:00
namespace DB
{
2012-03-05 00:09:41 +00:00
InterpreterCreateQuery::InterpreterCreateQuery(ASTPtr query_ptr_, Context & context_)
: query_ptr(query_ptr_), context(context_)
2011-11-01 15:16:04 +00:00
{
}
2011-11-01 15:16:04 +00:00
StoragePtr InterpreterCreateQuery::execute(bool assume_metadata_exists)
2011-08-18 20:33:20 +00:00
{
2012-08-02 17:33:31 +00:00
String path = context.getPath();
String current_database = context.getCurrentDatabase();
ASTCreateQuery & create = typeid_cast<ASTCreateQuery &>(*query_ptr);
2011-08-18 20:33:20 +00:00
2012-08-02 17:33:31 +00:00
String database_name = create.database.empty() ? current_database : create.database;
2011-11-05 23:31:19 +00:00
String database_name_escaped = escapeForFileName(database_name);
2011-10-30 05:19:41 +00:00
String table_name = create.table;
2011-11-05 23:31:19 +00:00
String table_name_escaped = escapeForFileName(table_name);
2012-08-02 17:33:31 +00:00
String as_database_name = create.as_database.empty() ? current_database : create.as_database;
2011-10-31 17:30:44 +00:00
String as_table_name = create.as_table;
2012-08-02 17:33:31 +00:00
String data_path = path + "data/" + database_name_escaped + "/";
String metadata_path = path + "metadata/" + database_name_escaped + "/" + (!table_name.empty() ? table_name_escaped + ".sql" : "");
2011-08-18 20:33:20 +00:00
2011-11-05 23:31:19 +00:00
/// CREATE|ATTACH DATABASE
if (!database_name.empty() && table_name.empty())
2011-08-19 18:31:14 +00:00
{
2011-11-05 23:31:19 +00:00
if (create.attach)
{
if (!Poco::File(data_path).exists())
throw Exception("Directory " + data_path + " doesn't exist.", ErrorCodes::DIRECTORY_DOESNT_EXIST);
}
else
{
if (!create.if_not_exists && Poco::File(metadata_path).exists())
throw Exception("Directory " + metadata_path + " already exists.", ErrorCodes::DIRECTORY_ALREADY_EXISTS);
if (!create.if_not_exists && Poco::File(data_path).exists())
throw Exception("Directory " + data_path + " already exists.", ErrorCodes::DIRECTORY_ALREADY_EXISTS);
2011-11-05 23:31:19 +00:00
Poco::File(metadata_path).createDirectory();
Poco::File(data_path).createDirectory();
2011-08-19 18:31:14 +00:00
}
2011-11-05 23:31:19 +00:00
if (!create.if_not_exists || !context.isDatabaseExist(database_name))
context.addDatabase(database_name);
return StoragePtr();
2011-11-05 23:31:19 +00:00
}
2012-08-02 17:33:31 +00:00
SharedPtr<InterpreterSelectQuery> interpreter_select;
Block select_sample;
/// Для таблиц типа вью, чтобы получить столбцы, может понадобиться sample block.
if (create.select && (!create.attach || (!create.columns && (create.is_view || create.is_materialized_view))))
{
interpreter_select = new InterpreterSelectQuery(create.select, context);
select_sample = interpreter_select->getSampleBlock();
}
StoragePtr res;
String storage_name;
NamesAndTypesListPtr columns = new NamesAndTypesList;
NamesAndTypesList materialized_columns{};
NamesAndTypesList alias_columns{};
ColumnDefaults column_defaults{};
2012-08-02 17:33:31 +00:00
StoragePtr as_storage;
IStorage::TableStructureReadLockPtr as_storage_lock;
if (!as_table_name.empty())
{
as_storage = context.getTable(as_database_name, as_table_name);
as_storage_lock = as_storage->lockStructure(false);
}
2011-11-05 23:31:19 +00:00
{
2012-08-02 17:33:31 +00:00
Poco::ScopedLock<Poco::Mutex> lock(context.getMutex());
2011-08-18 20:33:20 +00:00
if (!create.is_temporary)
2012-08-02 17:33:31 +00:00
{
context.assertDatabaseExists(database_name);
if (context.isTableExist(database_name, table_name))
{
if (create.if_not_exists)
return context.getTable(database_name, table_name);
else
throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
}
2012-08-02 17:33:31 +00:00
}
2011-11-01 15:16:04 +00:00
2012-08-02 17:33:31 +00:00
/// Получаем список столбцов
if (create.columns)
2011-10-31 17:30:44 +00:00
{
auto && columns_and_defaults = parseColumns(create.columns);
materialized_columns = removeAndReturnColumns(columns_and_defaults, ColumnDefaultType::Materialized);
alias_columns = removeAndReturnColumns(columns_and_defaults, ColumnDefaultType::Alias);
columns = new NamesAndTypesList{std::move(columns_and_defaults.first)};
column_defaults = std::move(columns_and_defaults.second);
2011-10-31 17:30:44 +00:00
}
2012-08-02 17:33:31 +00:00
else if (!create.as_table.empty())
columns = new NamesAndTypesList(as_storage->getColumnsList());
2012-08-02 17:33:31 +00:00
else if (create.select)
{
columns = new NamesAndTypesList;
for (size_t i = 0; i < select_sample.columns(); ++i)
columns->push_back(NameAndTypePair(select_sample.getByPosition(i).name, select_sample.getByPosition(i).type));
2012-08-02 17:33:31 +00:00
}
else
throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY);
2011-08-18 20:33:20 +00:00
2014-07-10 11:13:45 +00:00
/// Даже если в запросе был список столбцов, на всякий случай приведем его к стандартному виду (развернем Nested).
ASTPtr new_columns = formatColumns(*columns, materialized_columns, alias_columns, column_defaults);
2014-07-10 11:13:45 +00:00
if (create.columns)
{
auto it = std::find(create.children.begin(), create.children.end(), create.columns);
if (it != create.children.end())
*it = new_columns;
else
create.children.push_back(new_columns);
}
2014-07-10 11:13:45 +00:00
else
create.children.push_back(new_columns);
create.columns = new_columns;
2012-08-20 19:21:04 +00:00
2012-08-02 17:33:31 +00:00
/// Выбор нужного движка таблицы
if (create.storage)
{
storage_name = typeid_cast<ASTFunction &>(*create.storage).name;
}
2012-08-02 17:33:31 +00:00
else if (!create.as_table.empty())
2012-08-20 19:21:04 +00:00
{
storage_name = as_storage->getName();
create.storage = typeid_cast<const ASTCreateQuery &>(*context.getCreateQuery(as_database_name, as_table_name)).storage;
}
else if (create.is_temporary)
{
storage_name = "Memory";
ASTFunction * func = new ASTFunction();
func->name = storage_name;
create.storage = func;
}
else if (create.is_view)
{
storage_name = "View";
ASTFunction * func = new ASTFunction();
func->name = storage_name;
create.storage = func;
}
else if (create.is_materialized_view)
{
storage_name = "MaterializedView";
ASTFunction * func = new ASTFunction();
func->name = storage_name;
create.storage = func;
}
else
2012-08-20 19:21:04 +00:00
throw Exception("Incorrect CREATE query: required ENGINE.", ErrorCodes::ENGINE_REQUIRED);
2011-08-18 20:33:20 +00:00
res = context.getStorageFactory().get(
storage_name, data_path, table_name, database_name, context,
context.getGlobalContext(), query_ptr, columns,
materialized_columns, alias_columns, column_defaults, create.attach);
2011-08-18 20:33:20 +00:00
2012-08-02 17:33:31 +00:00
/// Проверка наличия метаданных таблицы на диске и создание метаданных
if (!assume_metadata_exists && !create.is_temporary)
2012-08-02 17:33:31 +00:00
{
if (Poco::File(metadata_path).exists())
{
/** Запрос ATTACH TABLE может использоваться, чтобы создать в оперативке ссылку на уже существующую таблицу.
* Это используется, например, при загрузке сервера.
*/
if (!create.attach)
throw Exception("Metadata for table " + database_name + "." + table_name + " already exists.",
ErrorCodes::TABLE_METADATA_ALREADY_EXISTS);
}
else
{
/// Меняем CREATE на ATTACH и пишем запрос в файл.
ASTPtr attach_ptr = query_ptr->clone();
ASTCreateQuery & attach = typeid_cast<ASTCreateQuery &>(*attach_ptr);
attach.attach = true;
attach.database.clear();
attach.as_database.clear();
attach.as_table.clear();
attach.if_not_exists = false;
attach.is_populate = false;
/// Для engine VIEW необходимо сохранить сам селект запрос, для остальных - наоборот
if (storage_name != "View" && storage_name != "MaterializedView")
2014-04-08 07:31:51 +00:00
attach.select = nullptr;
Poco::FileOutputStream metadata_file(metadata_path);
formatAST(attach, metadata_file, 0, false);
metadata_file << "\n";
}
2012-08-02 17:33:31 +00:00
}
if (create.is_temporary)
{
context.getSessionContext().addExternalTable(table_name, res);
}
else
context.addTable(database_name, table_name, res);
2012-08-02 17:33:31 +00:00
}
2011-11-01 15:16:04 +00:00
/// Если запрос CREATE SELECT, то вставим в таблицу данные
if (create.select && storage_name != "View" && (storage_name != "MaterializedView" || create.is_populate))
2011-11-06 06:22:52 +00:00
{
BlockInputStreamPtr from = new MaterializingBlockInputStream(interpreter_select->execute());
copyData(*from, *res->write(query_ptr));
}
2011-08-18 20:33:20 +00:00
return res;
}
InterpreterCreateQuery::ColumnsAndDefaults InterpreterCreateQuery::parseColumns(ASTPtr expression_list)
2014-07-10 11:13:45 +00:00
{
auto & column_list_ast = typeid_cast<ASTExpressionList &>(*expression_list);
/// list of table columns in correct order
NamesAndTypesList columns{}, known_type_columns{};
ColumnDefaults defaults{};
/// Columns requiring type-deduction or default_expression type-check
std::vector<std::pair<NameAndTypePair *, ASTColumnDeclaration *>> defaulted_columns{};
/** all default_expressions as a single expression list,
* mixed with conversion-columns for each explicitly specified type */
ASTPtr default_expr_list{new ASTExpressionList};
default_expr_list->children.reserve(column_list_ast.children.size());
for (auto & ast : column_list_ast.children)
2014-07-10 11:13:45 +00:00
{
auto & col_decl = typeid_cast<ASTColumnDeclaration &>(*ast);
if (col_decl.type)
{
const auto & type_range = col_decl.type->range;
columns.emplace_back(col_decl.name,
context.getDataTypeFactory().get({ type_range.first, type_range.second }));
known_type_columns.emplace_back(columns.back());
}
else
columns.emplace_back(col_decl.name, nullptr);
/// add column to postprocessing if there is a default_expression specified
if (col_decl.default_expression)
{
defaulted_columns.emplace_back(&columns.back(), &col_decl);
/** for columns with explicitly-specified type create two expressions:
* 1. default_expression aliased as column name with _tmp suffix
* 2. conversion of expression (1) to explicitly-specified type alias as column name */
if (col_decl.type)
{
const auto tmp_column_name = col_decl.name + "_tmp";
const auto & final_column_name = col_decl.name;
const auto conversion_function_name = "to" + columns.back().type->getName();
default_expr_list->children.emplace_back(setAlias(
makeASTFunction(conversion_function_name, ASTPtr{new ASTIdentifier{{}, tmp_column_name}}),
final_column_name));
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), tmp_column_name));
}
else
{
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name));
}
}
2014-07-10 11:13:45 +00:00
}
/// set missing types and wrap default_expression's in a conversion-function if necessary
if (!defaulted_columns.empty())
{
const auto actions = ExpressionAnalyzer{default_expr_list, context, known_type_columns}.getActions(true);
const auto block = actions->getSampleBlock();
for (auto & column : defaulted_columns)
{
const auto name_and_type_ptr = column.first;
const auto col_decl_ptr = column.second;
if (name_and_type_ptr->type)
{
const auto & tmp_column = block.getByName(col_decl_ptr->name + "_tmp");
/// type mismatch between explicitly specified and deduced type, add conversion
if (typeid(*name_and_type_ptr->type) != typeid(*tmp_column.type))
{
col_decl_ptr->default_expression = makeASTFunction(
"to" + name_and_type_ptr->type->getName(),
col_decl_ptr->default_expression);
col_decl_ptr->children.clear();
col_decl_ptr->children.push_back(col_decl_ptr->type);
col_decl_ptr->children.push_back(col_decl_ptr->default_expression);
}
}
else
name_and_type_ptr->type = block.getByName(name_and_type_ptr->name).type;
defaults.emplace(col_decl_ptr->name, ColumnDefault{
columnDefaultTypeFromString(col_decl_ptr->default_specifier),
setAlias(col_decl_ptr->default_expression, col_decl_ptr->name)
});
}
}
return { *DataTypeNested::expandNestedColumns(columns), defaults };
2014-07-10 11:13:45 +00:00
}
NamesAndTypesList InterpreterCreateQuery::removeAndReturnColumns(ColumnsAndDefaults & columns_and_defaults,
const ColumnDefaultType type)
{
auto & columns = columns_and_defaults.first;
auto & defaults = columns_and_defaults.second;
NamesAndTypesList removed{};
for (auto it = std::begin(columns); it != std::end(columns);)
{
const auto jt = defaults.find(it->name);
if (jt != std::end(defaults) && jt->second.type == type)
{
removed.push_back(*it);
it = columns.erase(it);
}
else
++it;
}
return removed;
}
2014-07-10 11:13:45 +00:00
ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
{
ASTPtr columns_list_ptr{new ASTExpressionList};
2014-07-10 11:13:45 +00:00
ASTExpressionList & columns_list = typeid_cast<ASTExpressionList &>(*columns_list_ptr);
for (const auto & column : columns)
2014-07-10 11:13:45 +00:00
{
const auto column_declaration = new ASTColumnDeclaration;
ASTPtr column_declaration_ptr{column_declaration};
2014-07-10 11:13:45 +00:00
column_declaration->name = column.name;
2014-07-10 11:13:45 +00:00
StringPtr type_name{new String(column.type->getName())};
auto pos = type_name->data();
const auto end = pos + type_name->size();
ParserIdentifierWithOptionalParameters storage_p;
Expected expected{""};
if (!storage_p.parse(pos, end, column_declaration->type, expected))
2014-07-10 11:13:45 +00:00
throw Exception("Cannot parse data type.", ErrorCodes::SYNTAX_ERROR);
column_declaration->type->query_string = type_name;
columns_list.children.push_back(column_declaration_ptr);
2014-07-10 11:13:45 +00:00
}
return columns_list_ptr;
}
ASTPtr InterpreterCreateQuery::formatColumns(NamesAndTypesList columns,
const NamesAndTypesList & materialized_columns,
const NamesAndTypesList & alias_columns,
const ColumnDefaults & column_defaults)
{
columns.insert(std::end(columns), std::begin(materialized_columns), std::end(materialized_columns));
columns.insert(std::end(columns), std::begin(alias_columns), std::end(alias_columns));
ASTPtr columns_list_ptr{new ASTExpressionList};
ASTExpressionList & columns_list = typeid_cast<ASTExpressionList &>(*columns_list_ptr);
for (const auto & column : columns)
{
const auto column_declaration = new ASTColumnDeclaration;
ASTPtr column_declaration_ptr{column_declaration};
column_declaration->name = column.name;
StringPtr type_name{new String(column.type->getName())};
auto pos = type_name->data();
const auto end = pos + type_name->size();
ParserIdentifierWithOptionalParameters storage_p;
Expected expected{""};
if (!storage_p.parse(pos, end, column_declaration->type, expected))
throw Exception("Cannot parse data type.", ErrorCodes::SYNTAX_ERROR);
column_declaration->type->query_string = type_name;
const auto it = column_defaults.find(column.name);
if (it != std::end(column_defaults))
{
column_declaration->default_specifier = toString(it->second.type);
column_declaration->default_expression = setAlias(it->second.expression->clone(), "");
}
columns_list.children.push_back(column_declaration_ptr);
}
return columns_list_ptr;
}
2011-08-18 20:33:20 +00:00
}