2016-03-19 01:18:49 +00:00
|
|
|
#include <memory>
|
|
|
|
|
2017-12-25 21:10:46 +00:00
|
|
|
#include <boost/range/join.hpp>
|
|
|
|
|
2011-11-05 23:31:19 +00:00
|
|
|
#include <Poco/File.h>
|
2011-10-24 12:10:59 +00:00
|
|
|
#include <Poco/FileStream.h>
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Common/escapeForFileName.h>
|
2018-11-08 12:03:42 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
|
|
|
#include <IO/WriteBufferFromFile.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
|
|
|
|
|
|
|
#include <Parsers/ASTColumnDeclaration.h>
|
2017-12-20 07:39:52 +00:00
|
|
|
#include <Parsers/ASTCreateQuery.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTIdentifier.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
2017-12-20 07:39:52 +00:00
|
|
|
#include <Parsers/ASTNameTypePair.h>
|
2018-02-19 00:45:32 +00:00
|
|
|
#include <Parsers/ASTInsertQuery.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ParserCreateQuery.h>
|
2017-12-20 07:39:52 +00:00
|
|
|
#include <Parsers/formatAST.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/parseQuery.h>
|
|
|
|
|
|
|
|
#include <Storages/StorageFactory.h>
|
|
|
|
#include <Storages/StorageLog.h>
|
|
|
|
|
2017-05-23 18:33:48 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2017-04-13 16:12:56 +00:00
|
|
|
#include <Interpreters/DDLWorker.h>
|
2017-12-20 07:39:52 +00:00
|
|
|
#include <Interpreters/ExpressionAnalyzer.h>
|
2018-11-08 15:43:14 +00:00
|
|
|
#include <Interpreters/SyntaxAnalyzer.h>
|
2017-12-20 07:39:52 +00:00
|
|
|
#include <Interpreters/InterpreterCreateQuery.h>
|
2018-02-27 19:38:59 +00:00
|
|
|
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
2018-02-19 00:45:32 +00:00
|
|
|
#include <Interpreters/InterpreterInsertQuery.h>
|
2018-02-27 19:38:59 +00:00
|
|
|
#include <Interpreters/ExpressionActions.h>
|
2018-11-01 14:49:37 +00:00
|
|
|
#include <Interpreters/AddDefaultDatabaseVisitor.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
|
|
|
#include <DataTypes/DataTypeFactory.h>
|
2017-12-25 18:58:39 +00:00
|
|
|
#include <DataTypes/NestedUtils.h>
|
2017-12-20 07:39:52 +00:00
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2018-08-14 12:51:45 +00:00
|
|
|
#include <DataTypes/DataTypesDecimal.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
|
|
|
#include <Databases/DatabaseFactory.h>
|
|
|
|
#include <Databases/IDatabase.h>
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-06-19 20:06:35 +00:00
|
|
|
#include <Common/ZooKeeper/ZooKeeper.h>
|
2011-08-18 20:33:20 +00:00
|
|
|
|
2018-10-11 02:57:48 +00:00
|
|
|
#include <Compression/CompressionFactory.h>
|
|
|
|
|
2018-02-19 00:45:32 +00:00
|
|
|
|
2011-08-18 20:33:20 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
2018-02-19 00:45:32 +00:00
|
|
|
|
2016-01-11 21:46:36 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int TABLE_ALREADY_EXISTS;
|
|
|
|
extern const int EMPTY_LIST_OF_COLUMNS_PASSED;
|
|
|
|
extern const int INCORRECT_QUERY;
|
|
|
|
extern const int ENGINE_REQUIRED;
|
|
|
|
extern const int UNKNOWN_DATABASE_ENGINE;
|
|
|
|
extern const int DUPLICATE_COLUMN;
|
2017-12-20 07:39:52 +00:00
|
|
|
extern const int READONLY;
|
2018-08-13 16:53:23 +00:00
|
|
|
extern const int ILLEGAL_COLUMN;
|
2018-09-11 14:33:06 +00:00
|
|
|
extern const int DATABASE_ALREADY_EXISTS;
|
2018-09-11 18:37:19 +00:00
|
|
|
extern const int QUERY_IS_PROHIBITED;
|
2018-10-10 13:15:17 +00:00
|
|
|
extern const int THERE_IS_NO_DEFAULT_VALUE;
|
2019-02-11 14:46:09 +00:00
|
|
|
extern const int BAD_DATABASE_FOR_TEMPORARY_TABLE;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2011-08-18 20:33:20 +00:00
|
|
|
|
2017-12-22 19:20:18 +00:00
|
|
|
InterpreterCreateQuery::InterpreterCreateQuery(const ASTPtr & query_ptr_, Context & context_)
|
|
|
|
: query_ptr(query_ptr_), context(context_)
|
|
|
|
{
|
|
|
|
}
|
2014-06-26 00:58:14 +00:00
|
|
|
|
2011-11-01 15:16:04 +00:00
|
|
|
|
2017-04-25 15:21:03 +00:00
|
|
|
BlockIO InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
2011-08-18 20:33:20 +00:00
|
|
|
{
|
2017-04-25 15:21:03 +00:00
|
|
|
if (!create.cluster.empty())
|
2018-04-17 19:33:58 +00:00
|
|
|
return executeDDLQueryOnCluster(query_ptr, context, {create.database});
|
2017-04-25 15:21:03 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String database_name = create.database;
|
|
|
|
|
2018-09-18 18:33:15 +00:00
|
|
|
auto guard = context.getDDLGuard(database_name, "");
|
|
|
|
|
|
|
|
/// Database can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard
|
|
|
|
if (context.isDatabaseExist(database_name))
|
2018-09-11 14:33:06 +00:00
|
|
|
{
|
|
|
|
if (create.if_not_exists)
|
|
|
|
return {};
|
|
|
|
else
|
|
|
|
throw Exception("Database " + database_name + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS);
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
String database_engine_name;
|
|
|
|
if (!create.storage)
|
|
|
|
{
|
2017-12-20 07:39:52 +00:00
|
|
|
database_engine_name = "Ordinary"; /// Default database engine.
|
2017-09-17 18:49:43 +00:00
|
|
|
auto engine = std::make_shared<ASTFunction>();
|
|
|
|
engine->name = database_engine_name;
|
|
|
|
auto storage = std::make_shared<ASTStorage>();
|
|
|
|
storage->set(storage->engine, engine);
|
|
|
|
create.set(create.storage, storage);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2017-09-18 14:18:29 +00:00
|
|
|
const ASTStorage & storage = *create.storage;
|
|
|
|
const ASTFunction & engine = *storage.engine;
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Currently, there are no database engines, that support any arguments.
|
2018-12-25 19:37:39 +00:00
|
|
|
if (engine.arguments || engine.parameters || storage.partition_by || storage.primary_key
|
2019-01-20 15:02:19 +00:00
|
|
|
|| storage.order_by || storage.sample_by || storage.settings ||
|
|
|
|
(create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty()))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
std::stringstream ostr;
|
2017-12-01 18:36:55 +00:00
|
|
|
formatAST(storage, ostr, false, false);
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
|
|
|
}
|
|
|
|
|
2017-09-17 18:49:43 +00:00
|
|
|
database_engine_name = engine.name;
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
String database_name_escaped = escapeForFileName(database_name);
|
|
|
|
|
2017-11-03 19:53:10 +00:00
|
|
|
/// Create directories for tables metadata.
|
2017-04-01 07:20:54 +00:00
|
|
|
String path = context.getPath();
|
|
|
|
String metadata_path = path + "metadata/" + database_name_escaped + "/";
|
|
|
|
Poco::File(metadata_path).createDirectory();
|
|
|
|
|
|
|
|
DatabasePtr database = DatabaseFactory::get(database_engine_name, database_name, metadata_path, context);
|
|
|
|
|
|
|
|
/// Will write file with database metadata, if needed.
|
|
|
|
String metadata_file_tmp_path = path + "metadata/" + database_name_escaped + ".sql.tmp";
|
|
|
|
String metadata_file_path = path + "metadata/" + database_name_escaped + ".sql";
|
|
|
|
|
|
|
|
bool need_write_metadata = !create.attach;
|
|
|
|
|
|
|
|
if (need_write_metadata)
|
|
|
|
{
|
|
|
|
create.attach = true;
|
|
|
|
create.if_not_exists = false;
|
|
|
|
|
|
|
|
std::ostringstream statement_stream;
|
2017-12-01 18:36:55 +00:00
|
|
|
formatAST(create, statement_stream, false);
|
2017-04-01 07:20:54 +00:00
|
|
|
statement_stream << '\n';
|
|
|
|
String statement = statement_stream.str();
|
|
|
|
|
|
|
|
/// Exclusive flag guarantees, that database is not created right now in another thread.
|
|
|
|
WriteBufferFromFile out(metadata_file_tmp_path, statement.size(), O_WRONLY | O_CREAT | O_EXCL);
|
|
|
|
writeString(statement, out);
|
|
|
|
|
|
|
|
out.next();
|
|
|
|
if (context.getSettingsRef().fsync_metadata)
|
|
|
|
out.sync();
|
|
|
|
out.close();
|
|
|
|
}
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
|
|
|
context.addDatabase(database_name, database);
|
|
|
|
|
|
|
|
if (need_write_metadata)
|
|
|
|
Poco::File(metadata_file_tmp_path).renameTo(metadata_file_path);
|
|
|
|
|
|
|
|
database->loadTables(context, thread_pool, has_force_restore_data_flag);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
if (need_write_metadata)
|
|
|
|
Poco::File(metadata_file_tmp_path).remove();
|
|
|
|
|
|
|
|
throw;
|
|
|
|
}
|
2017-04-25 15:21:03 +00:00
|
|
|
|
|
|
|
return {};
|
2011-08-18 20:33:20 +00:00
|
|
|
}
|
|
|
|
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-12-25 21:57:29 +00:00
|
|
|
using ColumnsAndDefaults = std::pair<NamesAndTypesList, ColumnDefaults>;
|
2018-10-11 02:57:48 +00:00
|
|
|
using ColumnsDeclarationAndModifiers = std::tuple<NamesAndTypesList, ColumnDefaults, ColumnCodecs, ColumnComments>;
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-04-02 17:37:49 +00:00
|
|
|
/// AST to the list of columns with types. Columns of Nested type are expanded into a list of real columns.
|
2018-10-11 02:57:48 +00:00
|
|
|
static ColumnsDeclarationAndModifiers parseColumns(const ASTExpressionList & column_list_ast, const Context & context)
|
2014-07-10 11:13:45 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
/// list of table columns in correct order
|
2017-12-25 21:57:29 +00:00
|
|
|
NamesAndTypesList columns{};
|
2017-04-01 07:20:54 +00:00
|
|
|
ColumnDefaults defaults{};
|
2018-10-11 02:57:48 +00:00
|
|
|
ColumnCodecs codecs{};
|
2018-10-14 15:30:06 +00:00
|
|
|
ColumnComments comments{};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// Columns requiring type-deduction or default_expression type-check
|
2017-12-25 21:57:29 +00:00
|
|
|
std::vector<std::pair<NameAndTypePair *, ASTColumnDeclaration *>> defaulted_columns{};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/** all default_expressions as a single expression list,
|
|
|
|
* mixed with conversion-columns for each explicitly specified type */
|
|
|
|
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
|
|
|
|
default_expr_list->children.reserve(column_list_ast.children.size());
|
|
|
|
|
2017-09-17 18:49:43 +00:00
|
|
|
for (const auto & ast : column_list_ast.children)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
auto & col_decl = typeid_cast<ASTColumnDeclaration &>(*ast);
|
|
|
|
|
2019-01-15 14:20:34 +00:00
|
|
|
DataTypePtr column_type = nullptr;
|
2017-04-01 07:20:54 +00:00
|
|
|
if (col_decl.type)
|
|
|
|
{
|
2019-01-15 14:20:34 +00:00
|
|
|
column_type = DataTypeFactory::instance().get(col_decl.type);
|
|
|
|
columns.emplace_back(col_decl.name, column_type);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions
|
|
|
|
columns.emplace_back(col_decl.name, std::make_shared<DataTypeUInt8>());
|
|
|
|
|
|
|
|
/// 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 & final_column_name = col_decl.name;
|
|
|
|
const auto tmp_column_name = final_column_name + "_tmp";
|
|
|
|
const auto data_type_ptr = columns.back().type.get();
|
|
|
|
|
|
|
|
default_expr_list->children.emplace_back(setAlias(
|
2018-02-26 03:37:08 +00:00
|
|
|
makeASTFunction("CAST", std::make_shared<ASTIdentifier>(tmp_column_name),
|
2018-10-22 08:54:54 +00:00
|
|
|
std::make_shared<ASTLiteral>(data_type_ptr->getName())), final_column_name));
|
2017-04-01 07:20:54 +00:00
|
|
|
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));
|
|
|
|
}
|
2018-10-14 15:30:06 +00:00
|
|
|
|
2018-10-11 02:57:48 +00:00
|
|
|
if (col_decl.codec)
|
|
|
|
{
|
2019-01-15 14:20:34 +00:00
|
|
|
auto codec = CompressionCodecFactory::instance().get(col_decl.codec, column_type);
|
2018-10-11 02:57:48 +00:00
|
|
|
codecs.emplace(col_decl.name, codec);
|
|
|
|
}
|
|
|
|
|
2018-11-08 12:03:42 +00:00
|
|
|
if (col_decl.comment)
|
2018-10-14 15:30:06 +00:00
|
|
|
{
|
2018-11-22 13:51:43 +00:00
|
|
|
if (auto comment_str = typeid_cast<ASTLiteral &>(*col_decl.comment).value.get<String>(); !comment_str.empty())
|
|
|
|
comments.emplace(col_decl.name, comment_str);
|
2018-10-14 15:30:06 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// set missing types and wrap default_expression's in a conversion-function if necessary
|
|
|
|
if (!defaulted_columns.empty())
|
|
|
|
{
|
2019-01-09 16:16:59 +00:00
|
|
|
auto syntax_analyzer_result = SyntaxAnalyzer(context).analyze(default_expr_list, columns);
|
2018-11-08 17:28:52 +00:00
|
|
|
const auto actions = ExpressionAnalyzer(default_expr_list, syntax_analyzer_result, context).getActions(true);
|
2017-04-01 07:20:54 +00:00
|
|
|
const auto block = actions->getSampleBlock();
|
|
|
|
|
2018-10-10 12:50:29 +00:00
|
|
|
for (auto action : actions->getActions())
|
|
|
|
if (action.type == ExpressionAction::Type::JOIN || action.type == ExpressionAction::Type::ARRAY_JOIN)
|
2018-10-10 13:23:06 +00:00
|
|
|
throw Exception("Cannot CREATE table. Unsupported default value that requires ARRAY JOIN or JOIN action", ErrorCodes::THERE_IS_NO_DEFAULT_VALUE);
|
2018-10-10 12:50:29 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (auto & column : defaulted_columns)
|
|
|
|
{
|
|
|
|
const auto name_and_type_ptr = column.first;
|
|
|
|
const auto col_decl_ptr = column.second;
|
|
|
|
|
|
|
|
const auto & column_name = col_decl_ptr->name;
|
|
|
|
const auto has_explicit_type = nullptr != col_decl_ptr->type;
|
|
|
|
auto & explicit_type = name_and_type_ptr->type;
|
|
|
|
|
|
|
|
/// if column declaration contains explicit type, name_and_type_ptr->type is not null
|
|
|
|
if (has_explicit_type)
|
|
|
|
{
|
|
|
|
const auto & tmp_column = block.getByName(column_name + "_tmp");
|
|
|
|
const auto & deduced_type = tmp_column.type;
|
|
|
|
|
|
|
|
/// type mismatch between explicitly specified and deduced type, add conversion for non-array types
|
|
|
|
if (!explicit_type->equals(*deduced_type))
|
|
|
|
{
|
|
|
|
col_decl_ptr->default_expression = makeASTFunction("CAST", col_decl_ptr->default_expression,
|
2018-02-26 03:37:08 +00:00
|
|
|
std::make_shared<ASTLiteral>(explicit_type->getName()));
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
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
|
|
|
|
/// no explicit type, name_and_type_ptr->type is null, set to deduced type
|
|
|
|
explicit_type = block.getByName(column_name).type;
|
|
|
|
|
|
|
|
defaults.emplace(column_name, ColumnDefault{
|
2018-03-12 13:47:01 +00:00
|
|
|
columnDefaultKindFromString(col_decl_ptr->default_specifier),
|
2017-04-01 07:20:54 +00:00
|
|
|
col_decl_ptr->default_expression
|
|
|
|
});
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-12-25 10:04:38 +00:00
|
|
|
std::unordered_map<std::string, std::vector<std::string>> mapping;
|
|
|
|
auto new_columns = Nested::flattenWithMapping(columns, mapping);
|
|
|
|
for (const auto & [old_name, new_names] : mapping)
|
|
|
|
{
|
2018-12-25 11:33:54 +00:00
|
|
|
auto codec_it = codecs.find(old_name);
|
|
|
|
if ((new_names.size() == 1 && old_name == new_names.back()) || codec_it == codecs.end())
|
2018-12-25 10:04:38 +00:00
|
|
|
continue;
|
|
|
|
|
2018-12-25 11:33:54 +00:00
|
|
|
auto codec = codec_it->second;
|
|
|
|
codecs.erase(codec_it);
|
2018-12-25 10:04:38 +00:00
|
|
|
for (const auto & new_name : new_names)
|
|
|
|
codecs.emplace(new_name, codec);
|
|
|
|
}
|
|
|
|
|
|
|
|
return {new_columns, defaults, codecs, comments};
|
2014-07-10 11:13:45 +00:00
|
|
|
}
|
|
|
|
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2018-12-14 11:29:12 +00:00
|
|
|
static NamesAndTypesList removeAndReturnColumns(ColumnsAndDefaults & columns_declare, const ColumnDefaultKind kind)
|
2014-07-10 11:13:45 +00:00
|
|
|
{
|
2018-10-11 02:57:48 +00:00
|
|
|
auto & columns = std::get<0>(columns_declare);
|
|
|
|
auto & defaults = std::get<1>(columns_declare);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-12-25 21:57:29 +00:00
|
|
|
NamesAndTypesList removed{};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
for (auto it = std::begin(columns); it != std::end(columns);)
|
|
|
|
{
|
|
|
|
const auto jt = defaults.find(it->name);
|
2018-03-12 13:47:01 +00:00
|
|
|
if (jt != std::end(defaults) && jt->second.kind == kind)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
removed.push_back(*it);
|
|
|
|
it = columns.erase(it);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
++it;
|
|
|
|
}
|
|
|
|
|
|
|
|
return removed;
|
2014-07-10 11:13:45 +00:00
|
|
|
}
|
|
|
|
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-12-25 21:57:29 +00:00
|
|
|
ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
|
2014-07-10 11:13:45 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
auto columns_list = std::make_shared<ASTExpressionList>();
|
2014-07-10 11:13:45 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
for (const auto & column : columns)
|
|
|
|
{
|
|
|
|
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
|
|
|
|
column_declaration->name = column.name;
|
2014-07-10 11:13:45 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ParserIdentifierWithOptionalParameters storage_p;
|
2019-01-18 16:30:35 +00:00
|
|
|
String type_name = column.type->getName();
|
|
|
|
auto pos = type_name.data();
|
|
|
|
const auto end = pos + type_name.size();
|
2018-04-16 15:11:13 +00:00
|
|
|
column_declaration->type = parseQuery(storage_p, pos, end, "data type", 0);
|
2017-04-01 07:20:54 +00:00
|
|
|
columns_list->children.emplace_back(column_declaration);
|
|
|
|
}
|
2014-07-10 11:13:45 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return columns_list;
|
2014-07-10 11:13:45 +00:00
|
|
|
}
|
|
|
|
|
2018-03-06 20:18:34 +00:00
|
|
|
ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
|
2014-09-25 15:01:09 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
auto columns_list = std::make_shared<ASTExpressionList>();
|
2014-09-30 03:08:47 +00:00
|
|
|
|
2018-03-10 17:03:57 +00:00
|
|
|
for (const auto & column : columns.getAll())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
|
|
|
|
ASTPtr column_declaration_ptr{column_declaration};
|
2014-09-30 03:08:47 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
column_declaration->name = column.name;
|
2014-09-30 03:08:47 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ParserIdentifierWithOptionalParameters storage_p;
|
2019-01-18 16:30:35 +00:00
|
|
|
String type_name = column.type->getName();
|
|
|
|
auto type_name_pos = type_name.data();
|
|
|
|
const auto type_name_end = type_name_pos + type_name.size();
|
2019-01-04 12:10:00 +00:00
|
|
|
column_declaration->type = parseQuery(storage_p, type_name_pos, type_name_end, "data type", 0);
|
2014-09-30 03:08:47 +00:00
|
|
|
|
2018-10-14 15:30:06 +00:00
|
|
|
const auto defaults_it = columns.defaults.find(column.name);
|
|
|
|
if (defaults_it != std::end(columns.defaults))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-10-14 15:30:06 +00:00
|
|
|
column_declaration->default_specifier = toString(defaults_it->second.kind);
|
|
|
|
column_declaration->default_expression = defaults_it->second.expression->clone();
|
|
|
|
}
|
|
|
|
|
|
|
|
const auto comments_it = columns.comments.find(column.name);
|
|
|
|
if (comments_it != std::end(columns.comments))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-11-08 12:03:42 +00:00
|
|
|
column_declaration->comment = std::make_shared<ASTLiteral>(Field(comments_it->second));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2014-09-25 15:01:09 +00:00
|
|
|
|
2018-10-11 02:57:48 +00:00
|
|
|
const auto ct = columns.codecs.find(column.name);
|
|
|
|
if (ct != std::end(columns.codecs))
|
|
|
|
{
|
2018-12-19 17:20:18 +00:00
|
|
|
String codec_desc = ct->second->getCodecDesc();
|
2018-10-11 02:57:48 +00:00
|
|
|
codec_desc = "CODEC(" + codec_desc + ")";
|
2019-01-04 12:10:00 +00:00
|
|
|
auto codec_desc_pos = codec_desc.data();
|
|
|
|
const auto codec_desc_end = codec_desc_pos + codec_desc.size();
|
2018-10-11 02:57:48 +00:00
|
|
|
ParserIdentifierWithParameters codec_p;
|
2019-01-04 12:10:00 +00:00
|
|
|
column_declaration->codec = parseQuery(codec_p, codec_desc_pos, codec_desc_end, "column codec", 0);
|
2018-10-11 02:57:48 +00:00
|
|
|
}
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
columns_list->children.push_back(column_declaration_ptr);
|
|
|
|
}
|
2014-07-10 11:13:45 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return columns_list;
|
2014-07-10 11:13:45 +00:00
|
|
|
}
|
|
|
|
|
2019-01-21 09:47:35 +00:00
|
|
|
ASTPtr InterpreterCreateQuery::formatIndices(const IndicesDescription & indices)
|
|
|
|
{
|
|
|
|
auto res = std::make_shared<ASTExpressionList>();
|
|
|
|
|
|
|
|
for (const auto & index : indices.indices)
|
2019-01-22 15:58:11 +00:00
|
|
|
res->children.push_back(index->clone());
|
2019-01-21 09:47:35 +00:00
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
2018-03-06 20:18:34 +00:00
|
|
|
ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpressionList & columns, const Context & context)
|
2016-03-19 01:18:49 +00:00
|
|
|
{
|
2018-03-06 20:18:34 +00:00
|
|
|
ColumnsDescription res;
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2018-10-14 15:30:06 +00:00
|
|
|
auto && parsed_columns = parseColumns(columns, context);
|
|
|
|
auto columns_and_defaults = std::make_pair(std::move(std::get<0>(parsed_columns)), std::move(std::get<1>(parsed_columns)));
|
2018-03-12 13:47:01 +00:00
|
|
|
res.aliases = removeAndReturnColumns(columns_and_defaults, ColumnDefaultKind::Alias);
|
2018-12-14 11:29:12 +00:00
|
|
|
res.materialized = removeAndReturnColumns(columns_and_defaults, ColumnDefaultKind::Materialized);
|
2018-03-06 20:18:34 +00:00
|
|
|
res.ordinary = std::move(columns_and_defaults.first);
|
|
|
|
res.defaults = std::move(columns_and_defaults.second);
|
2018-10-11 02:57:48 +00:00
|
|
|
res.codecs = std::move(std::get<2>(parsed_columns));
|
|
|
|
res.comments = std::move(std::get<3>(parsed_columns));
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2018-03-06 20:18:34 +00:00
|
|
|
if (res.ordinary.size() + res.materialized.size() == 0)
|
2017-04-01 07:20:54 +00:00
|
|
|
throw Exception{"Cannot CREATE table without physical columns", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED};
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return res;
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2018-03-06 20:18:34 +00:00
|
|
|
ColumnsDescription InterpreterCreateQuery::setColumns(
|
2017-04-01 07:20:54 +00:00
|
|
|
ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const
|
2016-03-19 01:18:49 +00:00
|
|
|
{
|
2018-03-06 20:18:34 +00:00
|
|
|
ColumnsDescription res;
|
2019-02-05 13:52:45 +00:00
|
|
|
IndicesDescription indices;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-02-05 13:52:45 +00:00
|
|
|
if (create.columns_list)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-02-05 13:52:45 +00:00
|
|
|
if (create.columns_list->columns)
|
|
|
|
res = getColumnsDescription(*create.columns_list->columns, context);
|
|
|
|
if (create.columns_list->indices)
|
|
|
|
for (const auto & index : create.columns_list->indices->children)
|
|
|
|
indices.indices.push_back(
|
|
|
|
std::dynamic_pointer_cast<ASTIndexDeclaration>(index->clone()));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else if (!create.as_table.empty())
|
|
|
|
{
|
2018-03-13 14:18:11 +00:00
|
|
|
res = as_storage->getColumns();
|
2019-02-05 13:52:45 +00:00
|
|
|
indices = as_storage->getIndicesDescription();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else if (create.select)
|
|
|
|
{
|
|
|
|
for (size_t i = 0; i < as_select_sample.columns(); ++i)
|
2018-03-06 20:18:34 +00:00
|
|
|
res.ordinary.emplace_back(as_select_sample.safeGetByPosition(i).name, as_select_sample.safeGetByPosition(i).type);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY);
|
|
|
|
|
|
|
|
/// Even if query has list of columns, canonicalize it (unfold Nested columns).
|
2018-03-06 20:18:34 +00:00
|
|
|
ASTPtr new_columns = formatColumns(res);
|
2019-02-05 13:52:45 +00:00
|
|
|
ASTPtr new_indices = formatIndices(indices);
|
|
|
|
|
2019-01-20 15:02:19 +00:00
|
|
|
if (!create.columns_list)
|
|
|
|
{
|
|
|
|
auto new_columns_list = std::make_shared<ASTColumns>();
|
|
|
|
create.set(create.columns_list, new_columns_list);
|
|
|
|
}
|
2019-01-20 11:03:21 +00:00
|
|
|
|
2019-01-20 15:02:19 +00:00
|
|
|
if (create.columns_list->columns)
|
|
|
|
create.columns_list->replace(create.columns_list->columns, new_columns);
|
2017-04-01 07:20:54 +00:00
|
|
|
else
|
2019-01-20 15:02:19 +00:00
|
|
|
create.columns_list->set(create.columns_list->columns, new_columns);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-02-05 13:52:45 +00:00
|
|
|
if (new_indices && create.columns_list->indices)
|
|
|
|
create.columns_list->replace(create.columns_list->indices, new_indices);
|
|
|
|
else if (new_indices)
|
|
|
|
create.columns_list->set(create.columns_list->indices, new_indices);
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Check for duplicates
|
|
|
|
std::set<String> all_columns;
|
2017-12-25 21:57:29 +00:00
|
|
|
auto check_column_already_exists = [&all_columns](const NameAndTypePair & column_name_and_type)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
if (!all_columns.emplace(column_name_and_type.name).second)
|
|
|
|
throw Exception("Column " + backQuoteIfNeed(column_name_and_type.name) + " already exists", ErrorCodes::DUPLICATE_COLUMN);
|
|
|
|
};
|
|
|
|
|
2018-03-06 20:18:34 +00:00
|
|
|
for (const auto & elem : res.ordinary)
|
2017-04-01 07:20:54 +00:00
|
|
|
check_column_already_exists(elem);
|
2018-03-06 20:18:34 +00:00
|
|
|
for (const auto & elem : res.materialized)
|
2017-04-01 07:20:54 +00:00
|
|
|
check_column_already_exists(elem);
|
2018-03-06 20:18:34 +00:00
|
|
|
for (const auto & elem : res.aliases)
|
2017-04-01 07:20:54 +00:00
|
|
|
check_column_already_exists(elem);
|
|
|
|
|
|
|
|
return res;
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-10-25 19:52:32 +00:00
|
|
|
void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
|
2016-03-19 01:18:49 +00:00
|
|
|
{
|
2017-10-25 19:52:32 +00:00
|
|
|
if (create.storage)
|
2017-10-26 12:15:33 +00:00
|
|
|
{
|
2018-10-18 15:03:14 +00:00
|
|
|
if (create.temporary && create.storage->engine->name != "Memory")
|
2017-10-26 12:15:33 +00:00
|
|
|
throw Exception(
|
|
|
|
"Temporary tables can only be created with ENGINE = Memory, not " + create.storage->engine->name,
|
|
|
|
ErrorCodes::INCORRECT_QUERY);
|
|
|
|
|
2017-10-25 19:52:32 +00:00
|
|
|
return;
|
2017-10-26 12:15:33 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-10-18 15:03:14 +00:00
|
|
|
if (create.temporary)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-09-17 18:49:43 +00:00
|
|
|
auto engine_ast = std::make_shared<ASTFunction>();
|
2017-10-25 19:52:32 +00:00
|
|
|
engine_ast->name = "Memory";
|
2017-09-17 18:49:43 +00:00
|
|
|
auto storage_ast = std::make_shared<ASTStorage>();
|
|
|
|
storage_ast->set(storage_ast->engine, engine_ast);
|
|
|
|
create.set(create.storage, storage_ast);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else if (!create.as_table.empty())
|
|
|
|
{
|
2017-04-02 17:37:49 +00:00
|
|
|
/// NOTE Getting the structure from the table specified in the AS is done not atomically with the creation of the table.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
String as_database_name = create.as_database.empty() ? context.getCurrentDatabase() : create.as_database;
|
|
|
|
String as_table_name = create.as_table;
|
|
|
|
|
2018-03-13 13:28:32 +00:00
|
|
|
ASTPtr as_create_ptr = context.getCreateTableQuery(as_database_name, as_table_name);
|
2017-09-17 18:49:43 +00:00
|
|
|
const auto & as_create = typeid_cast<const ASTCreateQuery &>(*as_create_ptr);
|
2017-08-18 20:56:19 +00:00
|
|
|
|
2017-10-25 19:52:32 +00:00
|
|
|
if (as_create.is_view)
|
|
|
|
throw Exception(
|
2017-12-22 19:20:18 +00:00
|
|
|
"Cannot CREATE a table AS " + as_database_name + "." + as_table_name + ", it is a View",
|
2017-10-25 19:52:32 +00:00
|
|
|
ErrorCodes::INCORRECT_QUERY);
|
2017-08-18 20:56:19 +00:00
|
|
|
|
2017-10-25 19:52:32 +00:00
|
|
|
create.set(create.storage, as_create.storage->ptr());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
|
2017-04-21 12:39:28 +00:00
|
|
|
BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
2016-03-19 01:18:49 +00:00
|
|
|
{
|
2017-04-21 12:39:28 +00:00
|
|
|
if (!create.cluster.empty())
|
2018-04-17 19:33:58 +00:00
|
|
|
{
|
|
|
|
NameSet databases{create.database};
|
|
|
|
if (!create.to_table.empty())
|
|
|
|
databases.emplace(create.to_database);
|
|
|
|
|
2018-10-25 09:40:30 +00:00
|
|
|
return executeDDLQueryOnCluster(query_ptr, context, std::move(databases));
|
2018-04-17 19:33:58 +00:00
|
|
|
}
|
2017-04-21 12:39:28 +00:00
|
|
|
|
2019-02-11 14:46:09 +00:00
|
|
|
/// Temporary tables are created out of databases.
|
|
|
|
if (create.temporary && !create.database.empty())
|
|
|
|
throw Exception("Temporary tables cannot be inside a database. You should not specify a database for a temporary table.",
|
|
|
|
ErrorCodes::BAD_DATABASE_FOR_TEMPORARY_TABLE);
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String path = context.getPath();
|
|
|
|
String current_database = context.getCurrentDatabase();
|
|
|
|
|
|
|
|
String database_name = create.database.empty() ? current_database : create.database;
|
|
|
|
String table_name = create.table;
|
|
|
|
String table_name_escaped = escapeForFileName(table_name);
|
|
|
|
|
2017-10-21 20:38:39 +00:00
|
|
|
// If this is a stub ATTACH query, read the query definition from the database
|
2019-01-20 11:03:21 +00:00
|
|
|
if (create.attach && !create.storage && !create.columns_list)
|
2017-10-21 20:38:39 +00:00
|
|
|
{
|
2017-10-25 00:55:31 +00:00
|
|
|
// Table SQL definition is available even if the table is detached
|
2018-03-13 13:28:32 +00:00
|
|
|
auto query = context.getCreateTableQuery(database_name, table_name);
|
2017-10-21 20:38:39 +00:00
|
|
|
auto & as_create = typeid_cast<const ASTCreateQuery &>(*query);
|
|
|
|
create = as_create; // Copy the saved create query, but use ATTACH instead of CREATE
|
|
|
|
create.attach = true;
|
|
|
|
}
|
|
|
|
|
2017-10-30 17:53:01 +00:00
|
|
|
if (create.to_database.empty())
|
|
|
|
create.to_database = current_database;
|
|
|
|
|
2017-11-21 13:30:45 +00:00
|
|
|
if (create.select && (create.is_view || create.is_materialized_view))
|
2018-11-01 14:49:37 +00:00
|
|
|
{
|
|
|
|
AddDefaultDatabaseVisitor visitor(current_database);
|
|
|
|
visitor.visit(*create.select);
|
|
|
|
}
|
2017-11-21 13:30:45 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
Block as_select_sample;
|
2019-01-20 11:03:21 +00:00
|
|
|
if (create.select && (!create.attach || !create.columns_list))
|
2018-02-27 19:38:59 +00:00
|
|
|
as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), context);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
String as_database_name = create.as_database.empty() ? current_database : create.as_database;
|
|
|
|
String as_table_name = create.as_table;
|
|
|
|
|
|
|
|
StoragePtr as_storage;
|
|
|
|
TableStructureReadLockPtr as_storage_lock;
|
|
|
|
if (!as_table_name.empty())
|
|
|
|
{
|
|
|
|
as_storage = context.getTable(as_database_name, as_table_name);
|
2018-11-28 15:50:52 +00:00
|
|
|
as_storage_lock = as_storage->lockStructure(false);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
/// Set and retrieve list of columns.
|
2018-03-06 20:18:34 +00:00
|
|
|
ColumnsDescription columns = setColumns(create, as_select_sample, as_storage);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2017-10-25 19:52:32 +00:00
|
|
|
/// Set the table engine if it was not specified explicitly.
|
|
|
|
setEngine(create);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
StoragePtr res;
|
|
|
|
|
|
|
|
{
|
|
|
|
std::unique_ptr<DDLGuard> guard;
|
|
|
|
|
2017-11-03 19:53:10 +00:00
|
|
|
String data_path;
|
|
|
|
DatabasePtr database;
|
|
|
|
|
2018-10-18 15:03:14 +00:00
|
|
|
if (!create.temporary)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2017-11-03 19:53:10 +00:00
|
|
|
database = context.getDatabase(database_name);
|
2018-02-21 19:26:59 +00:00
|
|
|
data_path = database->getDataPath();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-09-18 18:33:15 +00:00
|
|
|
/** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing).
|
|
|
|
* If table doesnt exist, one thread is creating table, while others wait in DDLGuard.
|
2017-04-01 07:20:54 +00:00
|
|
|
*/
|
2018-09-18 18:33:15 +00:00
|
|
|
guard = context.getDDLGuard(database_name, table_name);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-09-18 18:33:15 +00:00
|
|
|
/// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard.
|
|
|
|
if (database->isTableExist(context, table_name))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
if (create.if_not_exists)
|
|
|
|
return {};
|
|
|
|
else
|
|
|
|
throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
|
|
|
|
}
|
|
|
|
}
|
2018-03-12 15:36:33 +00:00
|
|
|
else if (context.tryGetExternalTable(table_name) && create.if_not_exists)
|
|
|
|
return {};
|
2018-05-07 02:01:11 +00:00
|
|
|
|
2017-12-20 07:39:52 +00:00
|
|
|
res = StorageFactory::instance().get(create,
|
|
|
|
data_path,
|
|
|
|
table_name,
|
|
|
|
database_name,
|
|
|
|
context,
|
|
|
|
context.getGlobalContext(),
|
2018-03-06 20:18:34 +00:00
|
|
|
columns,
|
2017-12-20 07:39:52 +00:00
|
|
|
create.attach,
|
|
|
|
false);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-10-18 15:03:14 +00:00
|
|
|
if (create.temporary)
|
2018-02-14 04:00:37 +00:00
|
|
|
context.getSessionContext().addExternalTable(table_name, res, query_ptr);
|
2017-04-01 07:20:54 +00:00
|
|
|
else
|
2017-11-03 19:53:10 +00:00
|
|
|
database->createTable(context, table_name, res, query_ptr);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-09-07 01:39:48 +00:00
|
|
|
/// We must call "startup" and "shutdown" while holding DDLGuard.
|
|
|
|
/// Because otherwise method "shutdown" (from InterpreterDropQuery) can be called before startup
|
|
|
|
/// (in case when table was created and instantly dropped before started up)
|
|
|
|
///
|
|
|
|
/// Method "startup" may create background tasks and method "shutdown" will wait for them.
|
|
|
|
/// But if "shutdown" is called before "startup", it will exit early, because there are no background tasks to wait.
|
|
|
|
/// Then background task is created by "startup" method. And when destructor of a table object is called, background task is still active,
|
|
|
|
/// and the task will use references to freed data.
|
|
|
|
|
|
|
|
res->startup();
|
|
|
|
}
|
2017-06-06 17:06:14 +00:00
|
|
|
|
2017-11-21 13:30:45 +00:00
|
|
|
/// If the query is a CREATE SELECT, insert the data into the table.
|
|
|
|
if (create.select && !create.attach
|
|
|
|
&& !create.is_view && (!create.is_materialized_view || create.is_populate))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2018-02-19 00:45:32 +00:00
|
|
|
auto insert = std::make_shared<ASTInsertQuery>();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-10-18 15:03:14 +00:00
|
|
|
if (!create.temporary)
|
2018-02-19 03:00:16 +00:00
|
|
|
insert->database = database_name;
|
|
|
|
|
2018-02-19 00:45:32 +00:00
|
|
|
insert->table = table_name;
|
|
|
|
insert->select = create.select->clone();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2018-10-18 15:03:14 +00:00
|
|
|
if (create.temporary && !context.getSessionContext().hasQueryContext())
|
2018-09-10 11:29:23 +00:00
|
|
|
context.getSessionContext().setQueryContext(context.getSessionContext());
|
2018-09-13 03:34:58 +00:00
|
|
|
|
2018-03-07 18:53:21 +00:00
|
|
|
return InterpreterInsertQuery(insert,
|
2018-10-18 15:03:14 +00:00
|
|
|
create.temporary ? context.getSessionContext() : context,
|
2018-03-07 18:53:21 +00:00
|
|
|
context.getSettingsRef().insert_allow_materialized_columns).execute();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return {};
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
2017-04-13 13:42:29 +00:00
|
|
|
|
2016-03-19 01:18:49 +00:00
|
|
|
BlockIO InterpreterCreateQuery::execute()
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
ASTCreateQuery & create = typeid_cast<ASTCreateQuery &>(*query_ptr);
|
2017-12-22 19:20:18 +00:00
|
|
|
checkAccess(create);
|
2018-02-12 18:41:53 +00:00
|
|
|
ASTQueryWithOutput::resetOutputASTIfExist(create);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// CREATE|ATTACH DATABASE
|
|
|
|
if (!create.database.empty() && create.table.empty())
|
|
|
|
{
|
2017-04-25 15:21:03 +00:00
|
|
|
return createDatabase(create);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else
|
2017-04-21 12:39:28 +00:00
|
|
|
return createTable(create);
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
2017-12-22 19:20:18 +00:00
|
|
|
|
2017-12-20 07:39:52 +00:00
|
|
|
void InterpreterCreateQuery::checkAccess(const ASTCreateQuery & create)
|
|
|
|
{
|
2018-01-18 23:40:32 +00:00
|
|
|
/// Internal queries (initiated by the server itself) always have access to everything.
|
|
|
|
if (internal)
|
|
|
|
return;
|
|
|
|
|
2017-12-20 07:39:52 +00:00
|
|
|
const Settings & settings = context.getSettingsRef();
|
2018-03-11 00:15:26 +00:00
|
|
|
auto readonly = settings.readonly;
|
2018-09-11 18:37:19 +00:00
|
|
|
auto allow_ddl = settings.allow_ddl;
|
2017-12-20 07:39:52 +00:00
|
|
|
|
2018-09-11 18:37:19 +00:00
|
|
|
if (!readonly && allow_ddl)
|
2017-12-20 07:39:52 +00:00
|
|
|
return;
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2017-12-20 07:39:52 +00:00
|
|
|
/// CREATE|ATTACH DATABASE
|
|
|
|
if (!create.database.empty() && create.table.empty())
|
|
|
|
{
|
2018-09-11 18:37:19 +00:00
|
|
|
if (readonly)
|
|
|
|
throw Exception("Cannot create database in readonly mode", ErrorCodes::READONLY);
|
|
|
|
|
2018-09-13 03:34:58 +00:00
|
|
|
throw Exception("Cannot create database. DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
|
2017-12-20 07:39:52 +00:00
|
|
|
}
|
|
|
|
|
2018-10-18 15:03:14 +00:00
|
|
|
if (create.temporary && readonly >= 2)
|
2017-12-20 07:39:52 +00:00
|
|
|
return;
|
|
|
|
|
2018-09-11 18:37:19 +00:00
|
|
|
if (readonly)
|
|
|
|
throw Exception("Cannot create table in readonly mode", ErrorCodes::READONLY);
|
|
|
|
|
2018-09-13 03:34:58 +00:00
|
|
|
throw Exception("Cannot create table. DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
|
2017-12-20 07:39:52 +00:00
|
|
|
}
|
2011-08-18 20:33:20 +00:00
|
|
|
}
|