2016-03-19 01:18:49 +00:00
|
|
|
#include <memory>
|
|
|
|
|
2020-07-02 20:39:31 +00:00
|
|
|
#include <filesystem>
|
2011-10-24 12:10:59 +00:00
|
|
|
|
2019-09-17 21:25:19 +00:00
|
|
|
#include <Common/StringUtils/StringUtils.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>
|
2020-09-23 18:28:59 +00:00
|
|
|
#include <Common/Macros.h>
|
2020-11-02 18:37:23 +00:00
|
|
|
#include <Common/randomSeed.h>
|
2020-12-08 14:13:35 +00:00
|
|
|
#include <Common/renameat2.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
2020-04-15 20:28:05 +00:00
|
|
|
#include <Core/Defines.h>
|
2020-05-20 01:22:32 +00:00
|
|
|
#include <Core/Settings.h>
|
2020-04-15 20:28:05 +00:00
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <IO/WriteBufferFromFile.h>
|
|
|
|
#include <IO/WriteHelpers.h>
|
2020-01-15 16:17:04 +00:00
|
|
|
#include <IO/ReadHelpers.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
|
|
|
#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>
|
2019-10-07 14:18:18 +00:00
|
|
|
#include <Parsers/ASTIndexDeclaration.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Parsers/ASTLiteral.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>
|
2020-05-28 12:37:05 +00:00
|
|
|
#include <Storages/StorageInMemoryMetadata.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
2017-05-23 18:33:48 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2020-11-03 13:47:26 +00:00
|
|
|
#include <Interpreters/executeDDLQueryOnCluster.h>
|
|
|
|
#include <Interpreters/Cluster.h>
|
2017-12-20 07:39:52 +00:00
|
|
|
#include <Interpreters/ExpressionAnalyzer.h>
|
|
|
|
#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-11-01 14:49:37 +00:00
|
|
|
#include <Interpreters/AddDefaultDatabaseVisitor.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
2020-01-26 09:49:53 +00:00
|
|
|
#include <Access/AccessRightsElement.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>
|
2019-05-28 15:59:37 +00:00
|
|
|
#include <DataTypes/DataTypeLowCardinality.h>
|
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
|
|
|
#include <Databases/DatabaseFactory.h>
|
2020-07-04 16:32:23 +00:00
|
|
|
#include <Databases/DatabaseReplicated.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
#include <Databases/IDatabase.h>
|
2020-07-06 08:30:11 +00:00
|
|
|
#include <Databases/DatabaseOnDisk.h>
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2020-04-12 20:50:32 +00:00
|
|
|
#include <Dictionaries/getDictionaryConfigurationFromAST.h>
|
|
|
|
|
2018-10-11 02:57:48 +00:00
|
|
|
#include <Compression/CompressionFactory.h>
|
2019-06-16 16:47:47 +00:00
|
|
|
|
2019-03-11 16:50:31 +00:00
|
|
|
#include <Interpreters/InterpreterDropQuery.h>
|
2020-12-14 03:30:39 +00:00
|
|
|
#include <Interpreters/QueryLog.h>
|
2019-06-16 16:47:47 +00:00
|
|
|
#include <Interpreters/addTypeConversionToAST.h>
|
2018-10-11 02:57:48 +00:00
|
|
|
|
2019-07-18 18:29:49 +00:00
|
|
|
#include <TableFunctions/TableFunctionFactory.h>
|
2020-12-08 14:13:35 +00:00
|
|
|
#include <common/logger_useful.h>
|
2019-07-18 18:29:49 +00:00
|
|
|
|
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 UNKNOWN_DATABASE_ENGINE;
|
|
|
|
extern const int DUPLICATE_COLUMN;
|
2018-09-11 14:33:06 +00:00
|
|
|
extern const int DATABASE_ALREADY_EXISTS;
|
2020-08-31 15:58:31 +00:00
|
|
|
extern const int BAD_ARGUMENTS;
|
2019-02-11 14:46:09 +00:00
|
|
|
extern const int BAD_DATABASE_FOR_TEMPORARY_TABLE;
|
2019-05-28 15:59:37 +00:00
|
|
|
extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY;
|
2019-10-11 13:21:52 +00:00
|
|
|
extern const int DICTIONARY_ALREADY_EXISTS;
|
2020-05-25 19:09:14 +00:00
|
|
|
extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE;
|
2020-05-30 15:26:19 +00:00
|
|
|
extern const int ILLEGAL_COLUMN;
|
2020-10-22 15:08:00 +00:00
|
|
|
extern const int LOGICAL_ERROR;
|
2020-11-05 09:52:23 +00:00
|
|
|
extern const int UNKNOWN_DATABASE;
|
2020-12-08 14:13:35 +00:00
|
|
|
extern const int PATH_ACCESS_DENIED;
|
2020-12-11 12:06:19 +00:00
|
|
|
extern const int NOT_IMPLEMENTED;
|
2020-12-25 17:43:03 +00:00
|
|
|
extern const int UNKNOWN_TABLE;
|
2016-01-11 21:46:36 +00:00
|
|
|
}
|
|
|
|
|
2020-07-02 20:39:31 +00:00
|
|
|
namespace fs = std::filesystem;
|
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-01 07:20:54 +00:00
|
|
|
String database_name = create.database;
|
|
|
|
|
2020-02-10 18:31:52 +00:00
|
|
|
auto guard = DatabaseCatalog::instance().getDDLGuard(database_name, "");
|
2018-09-18 18:33:15 +00:00
|
|
|
|
|
|
|
/// Database can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard
|
2020-02-10 13:10:17 +00:00
|
|
|
if (DatabaseCatalog::instance().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
|
|
|
|
2020-07-06 08:30:11 +00:00
|
|
|
/// Will write file with database metadata, if needed.
|
|
|
|
String database_name_escaped = escapeForFileName(database_name);
|
|
|
|
fs::path metadata_path = fs::canonical(context.getPath());
|
|
|
|
fs::path metadata_file_tmp_path = metadata_path / "metadata" / (database_name_escaped + ".sql.tmp");
|
|
|
|
fs::path metadata_file_path = metadata_path / "metadata" / (database_name_escaped + ".sql");
|
|
|
|
|
|
|
|
if (!create.storage && create.attach)
|
|
|
|
{
|
|
|
|
if (!fs::exists(metadata_file_path))
|
|
|
|
throw Exception("Database engine must be specified for ATTACH DATABASE query", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
|
|
|
/// Short syntax: try read database definition from file
|
|
|
|
auto ast = DatabaseOnDisk::parseQueryFromMetadata(nullptr, context, metadata_file_path);
|
|
|
|
create = ast->as<ASTCreateQuery &>();
|
|
|
|
if (!create.table.empty() || !create.storage)
|
2020-11-22 17:13:40 +00:00
|
|
|
throw Exception(ErrorCodes::INCORRECT_QUERY, "Metadata file {} contains incorrect CREATE DATABASE query", metadata_file_path.string());
|
2020-07-06 08:30:11 +00:00
|
|
|
create.attach = true;
|
|
|
|
create.attach_short_syntax = true;
|
2020-07-17 17:54:24 +00:00
|
|
|
create.database = database_name;
|
2020-07-06 08:30:11 +00:00
|
|
|
}
|
|
|
|
else if (!create.storage)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-10-23 13:46:38 +00:00
|
|
|
/// For new-style databases engine is explicitly specified in .sql
|
|
|
|
/// When attaching old-style database during server startup, we must always use Ordinary engine
|
2019-11-28 19:40:51 +00:00
|
|
|
if (create.attach)
|
|
|
|
throw Exception("Database engine must be specified for ATTACH DATABASE query", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
|
|
|
bool old_style_database = context.getSettingsRef().default_database_engine.value == DefaultDatabaseEngine::Ordinary;
|
2017-09-17 18:49:43 +00:00
|
|
|
auto engine = std::make_shared<ASTFunction>();
|
|
|
|
auto storage = std::make_shared<ASTStorage>();
|
2021-02-15 10:26:34 +00:00
|
|
|
engine->name = old_style_database ? "Ordinary" : "Atomic";
|
2020-12-04 02:15:44 +00:00
|
|
|
engine->no_empty_args = true;
|
2017-09-17 18:49:43 +00:00
|
|
|
storage->set(storage->engine, engine);
|
|
|
|
create.set(create.storage, storage);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2019-11-26 19:06:19 +00:00
|
|
|
else if ((create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty()))
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
|
|
|
/// Currently, there are no database engines, that support any arguments.
|
2020-11-09 16:05:40 +00:00
|
|
|
throw Exception(ErrorCodes::UNKNOWN_DATABASE_ENGINE, "Unknown database engine: {}", serializeAST(*create.storage));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
|
2020-10-26 15:12:16 +00:00
|
|
|
if (create.storage->engine->name == "Atomic" || create.storage->engine->name == "Replicated")
|
2020-07-02 20:39:31 +00:00
|
|
|
{
|
|
|
|
if (create.attach && create.uuid == UUIDHelpers::Nil)
|
2020-12-08 14:13:35 +00:00
|
|
|
throw Exception(ErrorCodes::INCORRECT_QUERY, "UUID must be specified for ATTACH. "
|
|
|
|
"If you want to attach existing database, use just ATTACH DATABASE {};", create.database);
|
2020-07-02 20:39:31 +00:00
|
|
|
else if (create.uuid == UUIDHelpers::Nil)
|
|
|
|
create.uuid = UUIDHelpers::generateV4();
|
2019-11-26 19:06:19 +00:00
|
|
|
|
2020-07-02 20:39:31 +00:00
|
|
|
metadata_path = metadata_path / "store" / DatabaseCatalog::getPathForUUID(create.uuid);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-07-02 20:39:31 +00:00
|
|
|
if (!create.attach && fs::exists(metadata_path))
|
2020-11-22 17:13:40 +00:00
|
|
|
throw Exception(ErrorCodes::DATABASE_ALREADY_EXISTS, "Metadata directory {} already exists", metadata_path.string());
|
2020-07-02 20:39:31 +00:00
|
|
|
}
|
2020-09-14 19:25:02 +00:00
|
|
|
else if (create.storage->engine->name == "MaterializeMySQL")
|
|
|
|
{
|
|
|
|
/// It creates nested database with Ordinary or Atomic engine depending on UUID in query and default engine setting.
|
|
|
|
/// Do nothing if it's an internal ATTACH on server startup or short-syntax ATTACH query from user,
|
|
|
|
/// because we got correct query from the metadata file in this case.
|
|
|
|
/// If we got query from user, then normalize it first.
|
|
|
|
bool attach_from_user = create.attach && !internal && !create.attach_short_syntax;
|
|
|
|
bool create_from_user = !create.attach;
|
|
|
|
|
|
|
|
if (create_from_user)
|
|
|
|
{
|
2020-09-17 18:19:02 +00:00
|
|
|
const auto & default_engine = context.getSettingsRef().default_database_engine.value;
|
2020-09-14 19:25:02 +00:00
|
|
|
if (create.uuid == UUIDHelpers::Nil && default_engine == DefaultDatabaseEngine::Atomic)
|
|
|
|
create.uuid = UUIDHelpers::generateV4(); /// Will enable Atomic engine for nested database
|
|
|
|
}
|
|
|
|
else if (attach_from_user && create.uuid == UUIDHelpers::Nil)
|
|
|
|
{
|
|
|
|
/// Ambiguity is possible: should we attach nested database as Ordinary
|
|
|
|
/// or throw "UUID must be specified" for Atomic? So we suggest short syntax for Ordinary.
|
|
|
|
throw Exception("Use short attach syntax ('ATTACH DATABASE name;' without engine) to attach existing database "
|
|
|
|
"or specify UUID to attach new database with Atomic engine", ErrorCodes::INCORRECT_QUERY);
|
|
|
|
}
|
|
|
|
|
|
|
|
/// Set metadata path according to nested engine
|
|
|
|
if (create.uuid == UUIDHelpers::Nil)
|
|
|
|
metadata_path = metadata_path / "metadata" / database_name_escaped;
|
|
|
|
else
|
|
|
|
metadata_path = metadata_path / "store" / DatabaseCatalog::getPathForUUID(create.uuid);
|
|
|
|
}
|
2020-07-02 20:39:31 +00:00
|
|
|
else
|
|
|
|
{
|
2020-07-10 09:19:32 +00:00
|
|
|
bool is_on_cluster = context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
|
|
|
|
if (create.uuid != UUIDHelpers::Nil && !is_on_cluster)
|
2020-07-02 20:39:31 +00:00
|
|
|
throw Exception("Ordinary database engine does not support UUID", ErrorCodes::INCORRECT_QUERY);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-07-10 09:19:32 +00:00
|
|
|
/// Ignore UUID if it's ON CLUSTER query
|
|
|
|
create.uuid = UUIDHelpers::Nil;
|
2020-07-02 20:39:31 +00:00
|
|
|
metadata_path = metadata_path / "metadata" / database_name_escaped;
|
|
|
|
}
|
|
|
|
|
2020-08-11 21:59:46 +00:00
|
|
|
if (create.storage->engine->name == "MaterializeMySQL" && !context.getSettingsRef().allow_experimental_database_materialize_mysql && !internal)
|
|
|
|
{
|
|
|
|
throw Exception("MaterializeMySQL is an experimental database engine. "
|
|
|
|
"Enable allow_experimental_database_materialize_mysql to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
|
|
|
}
|
|
|
|
|
2021-02-15 10:26:34 +00:00
|
|
|
if (create.storage->engine->name == "Replicated" && !context.getSettingsRef().allow_experimental_database_replicated && !internal)
|
|
|
|
{
|
|
|
|
throw Exception("Replicated is an experimental database engine. "
|
|
|
|
"Enable allow_experimental_database_replicated to use it.", ErrorCodes::UNKNOWN_DATABASE_ENGINE);
|
|
|
|
}
|
|
|
|
|
2020-07-17 13:11:44 +00:00
|
|
|
DatabasePtr database = DatabaseFactory::get(create, metadata_path / "", context);
|
|
|
|
|
|
|
|
if (create.uuid != UUIDHelpers::Nil)
|
|
|
|
create.database = TABLE_WITH_UUID_NAME_PLACEHOLDER;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-07-02 20:39:31 +00:00
|
|
|
bool need_write_metadata = !create.attach || !fs::exists(metadata_file_path);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (need_write_metadata)
|
|
|
|
{
|
|
|
|
create.attach = true;
|
|
|
|
create.if_not_exists = false;
|
|
|
|
|
2020-11-09 16:05:40 +00:00
|
|
|
WriteBufferFromOwnString statement_buf;
|
|
|
|
formatAST(create, statement_buf, false);
|
|
|
|
writeChar('\n', statement_buf);
|
|
|
|
String statement = statement_buf.str();
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
/// 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();
|
|
|
|
}
|
|
|
|
|
2020-08-18 15:15:27 +00:00
|
|
|
/// We attach database before loading it's tables, so do not allow concurrent DDL queries
|
|
|
|
auto db_guard = DatabaseCatalog::instance().getExclusiveDDLGuardForDatabase(database_name);
|
|
|
|
|
2019-11-26 19:06:19 +00:00
|
|
|
bool added = false;
|
|
|
|
bool renamed = false;
|
2017-04-01 07:20:54 +00:00
|
|
|
try
|
|
|
|
{
|
2020-04-06 23:22:44 +00:00
|
|
|
/// TODO Attach db only after it was loaded. Now it's not possible because of view dependencies
|
2020-02-10 13:10:17 +00:00
|
|
|
DatabaseCatalog::instance().attachDatabase(database_name, database);
|
2019-11-26 19:06:19 +00:00
|
|
|
added = true;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
if (need_write_metadata)
|
2019-11-26 19:06:19 +00:00
|
|
|
{
|
2020-12-08 14:13:35 +00:00
|
|
|
/// Prevents from overwriting metadata of detached database
|
|
|
|
renameNoReplace(metadata_file_tmp_path, metadata_file_path);
|
2019-11-26 19:06:19 +00:00
|
|
|
renamed = true;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-08-06 11:56:49 +00:00
|
|
|
database->loadStoredObjects(context, has_force_restore_data_flag, create.attach && force_attach);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
2019-11-26 19:06:19 +00:00
|
|
|
if (renamed)
|
2020-07-02 20:39:31 +00:00
|
|
|
{
|
2020-10-14 05:26:28 +00:00
|
|
|
[[maybe_unused]] bool removed = fs::remove(metadata_file_path);
|
2020-07-02 20:39:31 +00:00
|
|
|
assert(removed);
|
|
|
|
}
|
2019-11-26 19:06:19 +00:00
|
|
|
if (added)
|
2020-02-17 19:28:25 +00:00
|
|
|
DatabaseCatalog::instance().detachDatabase(database_name, false, false);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
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
|
|
|
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
|
|
|
|
2020-08-26 21:57:42 +00:00
|
|
|
ParserDataType type_parser;
|
2019-01-18 16:30:35 +00:00
|
|
|
String type_name = column.type->getName();
|
2020-04-22 06:01:33 +00:00
|
|
|
const char * pos = type_name.data();
|
|
|
|
const char * end = pos + type_name.size();
|
2020-08-26 21:57:42 +00:00
|
|
|
column_declaration->type = parseQuery(type_parser, pos, end, "data type", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
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
|
|
|
|
2019-03-14 15:20:51 +00:00
|
|
|
for (const auto & column : columns)
|
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
|
|
|
|
2020-08-26 21:57:42 +00:00
|
|
|
ParserDataType type_parser;
|
2019-01-18 16:30:35 +00:00
|
|
|
String type_name = column.type->getName();
|
2020-04-22 06:01:33 +00:00
|
|
|
const char * type_name_pos = type_name.data();
|
|
|
|
const char * type_name_end = type_name_pos + type_name.size();
|
2020-08-26 21:57:42 +00:00
|
|
|
column_declaration->type = parseQuery(type_parser, type_name_pos, type_name_end, "data type", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH);
|
2014-09-30 03:08:47 +00:00
|
|
|
|
2019-03-14 15:20:51 +00:00
|
|
|
if (column.default_desc.expression)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-03-14 15:20:51 +00:00
|
|
|
column_declaration->default_specifier = toString(column.default_desc.kind);
|
|
|
|
column_declaration->default_expression = column.default_desc.expression->clone();
|
2018-10-14 15:30:06 +00:00
|
|
|
}
|
|
|
|
|
2019-03-14 15:20:51 +00:00
|
|
|
if (!column.comment.empty())
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-03-14 15:20:51 +00:00
|
|
|
column_declaration->comment = std::make_shared<ASTLiteral>(Field(column.comment));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2014-09-25 15:01:09 +00:00
|
|
|
|
2019-03-14 15:20:51 +00:00
|
|
|
if (column.codec)
|
2020-08-28 17:40:45 +00:00
|
|
|
column_declaration->codec = column.codec;
|
2018-10-11 02:57:48 +00:00
|
|
|
|
2019-04-15 09:30:45 +00:00
|
|
|
if (column.ttl)
|
|
|
|
column_declaration->ttl = column.ttl;
|
|
|
|
|
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>();
|
|
|
|
|
2020-05-28 12:47:17 +00:00
|
|
|
for (const auto & index : indices)
|
2020-05-28 12:37:05 +00:00
|
|
|
res->children.push_back(index.definition_ast->clone());
|
2019-01-21 09:47:35 +00:00
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
2019-05-17 04:08:03 +00:00
|
|
|
ASTPtr InterpreterCreateQuery::formatConstraints(const ConstraintsDescription & constraints)
|
|
|
|
{
|
|
|
|
auto res = std::make_shared<ASTExpressionList>();
|
|
|
|
|
|
|
|
for (const auto & constraint : constraints.constraints)
|
|
|
|
res->children.push_back(constraint->clone());
|
|
|
|
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
2020-05-04 00:11:49 +00:00
|
|
|
ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
|
|
|
|
const ASTExpressionList & columns_ast, const Context & context, bool sanity_check_compression_codecs)
|
2016-03-19 01:18:49 +00:00
|
|
|
{
|
2019-03-14 15:20:51 +00:00
|
|
|
/// First, deduce implicit types.
|
|
|
|
|
|
|
|
/** all default_expressions as a single expression list,
|
|
|
|
* mixed with conversion-columns for each explicitly specified type */
|
2020-05-20 01:22:32 +00:00
|
|
|
|
2019-03-14 15:20:51 +00:00
|
|
|
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
|
|
|
|
NamesAndTypesList column_names_and_types;
|
|
|
|
|
|
|
|
for (const auto & ast : columns_ast.children)
|
|
|
|
{
|
2019-03-16 21:46:53 +00:00
|
|
|
const auto & col_decl = ast->as<ASTColumnDeclaration &>();
|
2019-03-14 15:20:51 +00:00
|
|
|
|
|
|
|
DataTypePtr column_type = nullptr;
|
2020-05-20 01:22:32 +00:00
|
|
|
|
2019-03-14 15:20:51 +00:00
|
|
|
if (col_decl.type)
|
|
|
|
{
|
|
|
|
column_type = DataTypeFactory::instance().get(col_decl.type);
|
2020-05-20 01:22:32 +00:00
|
|
|
|
2020-06-14 16:17:22 +00:00
|
|
|
if (col_decl.null_modifier)
|
2020-05-27 12:32:39 +00:00
|
|
|
{
|
2020-05-23 14:32:47 +00:00
|
|
|
if (column_type->isNullable())
|
2020-08-08 01:01:47 +00:00
|
|
|
throw Exception("Can't use [NOT] NULL modifier with Nullable type", ErrorCodes::ILLEGAL_SYNTAX_FOR_DATA_TYPE);
|
2020-06-14 16:17:22 +00:00
|
|
|
if (*col_decl.null_modifier)
|
2020-05-20 01:22:32 +00:00
|
|
|
column_type = makeNullable(column_type);
|
|
|
|
}
|
2020-06-14 16:17:22 +00:00
|
|
|
else if (context.getSettingsRef().data_type_default_nullable)
|
|
|
|
{
|
2020-05-20 01:22:32 +00:00
|
|
|
column_type = makeNullable(column_type);
|
2020-06-14 16:17:22 +00:00
|
|
|
}
|
2020-05-20 01:22:32 +00:00
|
|
|
|
2019-03-14 15:20:51 +00:00
|
|
|
column_names_and_types.emplace_back(col_decl.name, column_type);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions
|
|
|
|
column_names_and_types.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)
|
|
|
|
{
|
2019-06-16 16:47:47 +00:00
|
|
|
/** 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
|
|
|
|
*/
|
2019-03-14 15:20:51 +00:00
|
|
|
if (col_decl.type)
|
|
|
|
{
|
|
|
|
const auto & final_column_name = col_decl.name;
|
2020-11-02 18:37:23 +00:00
|
|
|
const auto tmp_column_name = final_column_name + "_tmp_alter" + toString(randomSeed());
|
2020-04-22 06:01:33 +00:00
|
|
|
const auto * data_type_ptr = column_names_and_types.back().type.get();
|
2019-03-14 15:20:51 +00:00
|
|
|
|
2019-06-16 16:47:47 +00:00
|
|
|
default_expr_list->children.emplace_back(
|
|
|
|
setAlias(addTypeConversionToAST(std::make_shared<ASTIdentifier>(tmp_column_name), data_type_ptr->getName()),
|
|
|
|
final_column_name));
|
|
|
|
|
|
|
|
default_expr_list->children.emplace_back(
|
|
|
|
setAlias(
|
|
|
|
col_decl.default_expression->clone(),
|
|
|
|
tmp_column_name));
|
2019-03-14 15:20:51 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
Block defaults_sample_block;
|
|
|
|
/// set missing types and wrap default_expression's in a conversion-function if necessary
|
|
|
|
if (!default_expr_list->children.empty())
|
2020-03-03 09:05:17 +00:00
|
|
|
defaults_sample_block = validateColumnsDefaultsAndGetSampleBlock(default_expr_list, column_names_and_types, context);
|
2019-03-14 15:20:51 +00:00
|
|
|
|
2018-03-06 20:18:34 +00:00
|
|
|
ColumnsDescription res;
|
2019-03-14 15:20:51 +00:00
|
|
|
auto name_type_it = column_names_and_types.begin();
|
|
|
|
for (auto ast_it = columns_ast.children.begin(); ast_it != columns_ast.children.end(); ++ast_it, ++name_type_it)
|
|
|
|
{
|
|
|
|
ColumnDescription column;
|
|
|
|
|
2019-03-16 21:46:53 +00:00
|
|
|
auto & col_decl = (*ast_it)->as<ASTColumnDeclaration &>();
|
2019-03-14 15:20:51 +00:00
|
|
|
|
|
|
|
column.name = col_decl.name;
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2019-03-14 15:20:51 +00:00
|
|
|
if (col_decl.default_expression)
|
|
|
|
{
|
|
|
|
ASTPtr default_expr = col_decl.default_expression->clone();
|
|
|
|
if (col_decl.type)
|
|
|
|
column.type = name_type_it->type;
|
|
|
|
else
|
|
|
|
column.type = defaults_sample_block.getByName(column.name).type;
|
|
|
|
|
|
|
|
column.default_desc.kind = columnDefaultKindFromString(col_decl.default_specifier);
|
|
|
|
column.default_desc.expression = default_expr;
|
|
|
|
}
|
|
|
|
else if (col_decl.type)
|
|
|
|
column.type = name_type_it->type;
|
|
|
|
else
|
|
|
|
throw Exception();
|
|
|
|
|
|
|
|
if (col_decl.comment)
|
2019-03-16 21:46:53 +00:00
|
|
|
column.comment = col_decl.comment->as<ASTLiteral &>().value.get<String>();
|
2019-03-14 15:20:51 +00:00
|
|
|
|
|
|
|
if (col_decl.codec)
|
2020-08-31 07:39:58 +00:00
|
|
|
{
|
|
|
|
if (col_decl.default_specifier == "ALIAS")
|
2020-08-31 15:58:31 +00:00
|
|
|
throw Exception{"Cannot specify codec for column type ALIAS", ErrorCodes::BAD_ARGUMENTS};
|
2020-08-31 07:39:58 +00:00
|
|
|
column.codec = CompressionCodecFactory::instance().validateCodecAndGetPreprocessedAST(
|
|
|
|
col_decl.codec, column.type, sanity_check_compression_codecs);
|
2020-08-31 06:24:19 +00:00
|
|
|
}
|
2019-03-14 15:20:51 +00:00
|
|
|
|
2019-04-15 09:30:45 +00:00
|
|
|
if (col_decl.ttl)
|
|
|
|
column.ttl = col_decl.ttl;
|
|
|
|
|
2019-03-14 15:20:51 +00:00
|
|
|
res.add(std::move(column));
|
|
|
|
}
|
|
|
|
|
2020-11-10 17:32:00 +00:00
|
|
|
if (context.getSettingsRef().flatten_nested)
|
|
|
|
res.flattenNested();
|
2019-03-14 15:20:51 +00:00
|
|
|
|
|
|
|
if (res.getAllPhysical().empty())
|
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
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-08-24 21:20:20 +00:00
|
|
|
ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(const ASTExpressionList * constraints)
|
|
|
|
{
|
|
|
|
ConstraintsDescription res;
|
|
|
|
if (constraints)
|
|
|
|
for (const auto & constraint : constraints->children)
|
|
|
|
res.constraints.push_back(std::dynamic_pointer_cast<ASTConstraintDeclaration>(constraint->clone()));
|
|
|
|
return res;
|
|
|
|
}
|
|
|
|
|
|
|
|
|
2019-10-23 18:39:07 +00:00
|
|
|
InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(ASTCreateQuery & create) const
|
2016-03-19 01:18:49 +00:00
|
|
|
{
|
2019-10-23 18:39:07 +00:00
|
|
|
TableProperties properties;
|
2020-06-18 16:10:47 +00:00
|
|
|
TableLockHolder as_storage_lock;
|
2019-05-17 04:08:03 +00:00
|
|
|
|
2019-02-05 13:52:45 +00:00
|
|
|
if (create.columns_list)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2020-10-14 12:19:29 +00:00
|
|
|
if (create.as_table_function && (create.columns_list->indices || create.columns_list->constraints))
|
|
|
|
throw Exception("Indexes and constraints are not supported for table functions", ErrorCodes::INCORRECT_QUERY);
|
|
|
|
|
2019-02-05 13:52:45 +00:00
|
|
|
if (create.columns_list->columns)
|
2020-05-04 00:11:49 +00:00
|
|
|
{
|
|
|
|
bool sanity_check_compression_codecs = !create.attach && !context.getSettingsRef().allow_suspicious_codecs;
|
|
|
|
properties.columns = getColumnsDescription(*create.columns_list->columns, context, sanity_check_compression_codecs);
|
|
|
|
}
|
2019-08-24 21:20:20 +00:00
|
|
|
|
2019-02-05 13:52:45 +00:00
|
|
|
if (create.columns_list->indices)
|
|
|
|
for (const auto & index : create.columns_list->indices->children)
|
2020-05-28 12:47:17 +00:00
|
|
|
properties.indices.push_back(
|
2020-06-01 12:02:36 +00:00
|
|
|
IndexDescription::getIndexFromAST(index->clone(), properties.columns, context));
|
2019-08-24 21:20:20 +00:00
|
|
|
|
2019-10-23 18:39:07 +00:00
|
|
|
properties.constraints = getConstraintsDescription(create.columns_list->constraints);
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else if (!create.as_table.empty())
|
|
|
|
{
|
2020-02-17 19:28:25 +00:00
|
|
|
String as_database_name = context.resolveDatabase(create.as_database);
|
2020-05-28 23:01:18 +00:00
|
|
|
StoragePtr as_storage = DatabaseCatalog::instance().getTable({as_database_name, create.as_table}, context);
|
2019-10-23 18:39:07 +00:00
|
|
|
|
|
|
|
/// as_storage->getColumns() and setEngine(...) must be called under structure lock of other_table for CREATE ... AS other_table.
|
2020-06-18 16:10:47 +00:00
|
|
|
as_storage_lock = as_storage->lockForShare(context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout);
|
2020-06-16 16:55:04 +00:00
|
|
|
auto as_storage_metadata = as_storage->getInMemoryMetadataPtr();
|
|
|
|
properties.columns = as_storage_metadata->getColumns();
|
2019-09-20 20:53:14 +00:00
|
|
|
|
|
|
|
/// Secondary indices make sense only for MergeTree family of storage engines.
|
|
|
|
/// We should not copy them for other storages.
|
2019-09-17 21:25:19 +00:00
|
|
|
if (create.storage && endsWith(create.storage->engine->name, "MergeTree"))
|
2020-06-16 16:55:04 +00:00
|
|
|
properties.indices = as_storage_metadata->getSecondaryIndices();
|
2019-09-20 20:53:14 +00:00
|
|
|
|
2020-06-16 16:55:04 +00:00
|
|
|
properties.constraints = as_storage_metadata->getConstraints();
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
|
|
|
else if (create.select)
|
|
|
|
{
|
2019-10-23 18:39:07 +00:00
|
|
|
Block as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), context);
|
|
|
|
properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList());
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2019-10-23 18:39:07 +00:00
|
|
|
else if (create.as_table_function)
|
2020-10-14 12:19:29 +00:00
|
|
|
{
|
|
|
|
/// Table function without columns list.
|
|
|
|
auto table_function = TableFunctionFactory::instance().get(create.as_table_function, context);
|
|
|
|
properties.columns = table_function->getActualTableStructure(context);
|
|
|
|
assert(!properties.columns.empty());
|
|
|
|
}
|
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);
|
|
|
|
|
2019-02-05 13:52:45 +00:00
|
|
|
|
2019-10-23 18:39:07 +00:00
|
|
|
/// Even if query has list of columns, canonicalize it (unfold Nested columns).
|
2019-01-20 15:02:19 +00:00
|
|
|
if (!create.columns_list)
|
2019-10-23 18:39:07 +00:00
|
|
|
create.set(create.columns_list, std::make_shared<ASTColumns>());
|
2019-01-20 11:03:21 +00:00
|
|
|
|
2019-10-23 18:39:07 +00:00
|
|
|
ASTPtr new_columns = formatColumns(properties.columns);
|
|
|
|
ASTPtr new_indices = formatIndices(properties.indices);
|
|
|
|
ASTPtr new_constraints = formatConstraints(properties.constraints);
|
|
|
|
|
|
|
|
create.columns_list->setOrReplace(create.columns_list->columns, new_columns);
|
|
|
|
create.columns_list->setOrReplace(create.columns_list->indices, new_indices);
|
|
|
|
create.columns_list->setOrReplace(create.columns_list->constraints, new_constraints);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-10-23 18:39:07 +00:00
|
|
|
validateTableStructure(create, properties);
|
|
|
|
/// Set the table engine if it was not specified explicitly.
|
|
|
|
setEngine(create);
|
2021-02-01 19:29:47 +00:00
|
|
|
|
2021-02-08 19:36:17 +00:00
|
|
|
assert(as_database_saved.empty() && as_table_saved.empty());
|
|
|
|
std::swap(create.as_database, as_database_saved);
|
|
|
|
std::swap(create.as_table, as_table_saved);
|
2021-02-01 19:29:47 +00:00
|
|
|
|
2019-10-23 18:39:07 +00:00
|
|
|
return properties;
|
|
|
|
}
|
2019-05-17 04:08:03 +00:00
|
|
|
|
2019-10-23 18:39:07 +00:00
|
|
|
void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & create,
|
|
|
|
const InterpreterCreateQuery::TableProperties & properties) const
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Check for duplicates
|
|
|
|
std::set<String> all_columns;
|
2019-10-23 18:39:07 +00:00
|
|
|
for (const auto & column : properties.columns)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-03-14 15:20:51 +00:00
|
|
|
if (!all_columns.emplace(column.name).second)
|
|
|
|
throw Exception("Column " + backQuoteIfNeed(column.name) + " already exists", ErrorCodes::DUPLICATE_COLUMN);
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-08-19 11:52:17 +00:00
|
|
|
const auto & settings = context.getSettingsRef();
|
|
|
|
|
2019-10-23 18:39:07 +00:00
|
|
|
/// Check low cardinality types in creating table if it was not allowed in setting
|
2020-08-19 11:52:17 +00:00
|
|
|
if (!create.attach && !settings.allow_suspicious_low_cardinality_types && !create.is_materialized_view)
|
2019-10-23 18:39:07 +00:00
|
|
|
{
|
|
|
|
for (const auto & name_and_type_pair : properties.columns.getAllPhysical())
|
|
|
|
{
|
|
|
|
if (const auto * current_type_ptr = typeid_cast<const DataTypeLowCardinality *>(name_and_type_pair.type.get()))
|
|
|
|
{
|
|
|
|
if (!isStringOrFixedString(*removeNullable(current_type_ptr->getDictionaryType())))
|
|
|
|
throw Exception("Creating columns of type " + current_type_ptr->getName() + " is prohibited by default "
|
|
|
|
"due to expected negative impact on performance. "
|
|
|
|
"It can be enabled with the \"allow_suspicious_low_cardinality_types\" setting.",
|
|
|
|
ErrorCodes::SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2020-05-30 15:26:19 +00:00
|
|
|
|
2020-08-19 11:52:17 +00:00
|
|
|
if (!create.attach && !settings.allow_experimental_geo_types)
|
2020-05-30 15:26:19 +00:00
|
|
|
{
|
|
|
|
for (const auto & name_and_type_pair : properties.columns.getAllPhysical())
|
|
|
|
{
|
2020-08-19 11:52:17 +00:00
|
|
|
const auto & type = name_and_type_pair.type->getName();
|
2020-05-30 15:26:19 +00:00
|
|
|
if (type == "MultiPolygon" || type == "Polygon" || type == "Ring" || type == "Point")
|
|
|
|
{
|
|
|
|
String message = "Cannot create table with column '" + name_and_type_pair.name + "' which type is '"
|
|
|
|
+ type + "' because experimental geo types are not allowed. "
|
|
|
|
+ "Set setting allow_experimental_geo_types = 1 in order to allow it.";
|
|
|
|
throw Exception(message, ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2020-08-19 11:52:17 +00:00
|
|
|
|
|
|
|
if (!create.attach && !settings.allow_experimental_bigint_types)
|
|
|
|
{
|
|
|
|
for (const auto & name_and_type_pair : properties.columns.getAllPhysical())
|
|
|
|
{
|
|
|
|
WhichDataType which(*name_and_type_pair.type);
|
|
|
|
if (which.IsBigIntOrDeimal())
|
|
|
|
{
|
|
|
|
const auto & type_name = name_and_type_pair.type->getName();
|
|
|
|
String message = "Cannot create table with column '" + name_and_type_pair.name + "' which type is '"
|
|
|
|
+ type_name + "' because experimental bigint types are not allowed. "
|
|
|
|
+ "Set 'allow_experimental_bigint_types' setting to enable.";
|
|
|
|
throw Exception(message, ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2020-12-14 22:16:04 +00:00
|
|
|
|
|
|
|
if (!create.attach && !settings.allow_experimental_map_type)
|
|
|
|
{
|
|
|
|
for (const auto & name_and_type_pair : properties.columns.getAllPhysical())
|
|
|
|
{
|
|
|
|
WhichDataType which(*name_and_type_pair.type);
|
|
|
|
if (which.isMap())
|
|
|
|
{
|
|
|
|
const auto & type_name = name_and_type_pair.type->getName();
|
|
|
|
String message = "Cannot create table with column '" + name_and_type_pair.name + "' which type is '"
|
|
|
|
+ type_name + "' because experimental Map type is not allowed. "
|
|
|
|
+ "Set 'allow_experimental_map_type = 1' setting to enable";
|
|
|
|
throw Exception(message, ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|
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
|
|
|
{
|
2020-10-14 12:19:29 +00:00
|
|
|
if (create.as_table_function)
|
|
|
|
return;
|
|
|
|
|
2020-12-16 03:19:38 +00:00
|
|
|
if (create.storage || create.is_dictionary || create.isView())
|
2017-10-26 12:15:33 +00:00
|
|
|
{
|
2020-10-14 12:19:29 +00:00
|
|
|
if (create.temporary && create.storage && create.storage->engine && 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
|
|
|
|
2020-02-14 18:20:20 +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";
|
2020-12-04 02:15:44 +00:00
|
|
|
engine_ast->no_empty_args = true;
|
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
|
|
|
|
2020-02-10 13:10:17 +00:00
|
|
|
String as_database_name = context.resolveDatabase(create.as_database);
|
2017-04-01 07:20:54 +00:00
|
|
|
String as_table_name = create.as_table;
|
|
|
|
|
2020-05-28 20:10:45 +00:00
|
|
|
ASTPtr as_create_ptr = DatabaseCatalog::instance().getDatabase(as_database_name)->getCreateTableQuery(as_table_name, context);
|
2019-03-15 16:14:13 +00:00
|
|
|
const auto & as_create = as_create_ptr->as<ASTCreateQuery &>();
|
2017-08-18 20:56:19 +00:00
|
|
|
|
2020-01-02 23:05:44 +00:00
|
|
|
const String qualified_name = backQuoteIfNeed(as_database_name) + "." + backQuoteIfNeed(as_table_name);
|
|
|
|
|
2020-12-16 04:07:50 +00:00
|
|
|
if (as_create.is_ordinary_view)
|
2017-10-25 19:52:32 +00:00
|
|
|
throw Exception(
|
2020-01-02 23:05:44 +00:00
|
|
|
"Cannot CREATE a table AS " + qualified_name + ", it is a View",
|
2017-10-25 19:52:32 +00:00
|
|
|
ErrorCodes::INCORRECT_QUERY);
|
2017-08-18 20:56:19 +00:00
|
|
|
|
2019-05-29 21:52:13 +00:00
|
|
|
if (as_create.is_live_view)
|
|
|
|
throw Exception(
|
2020-01-02 23:05:44 +00:00
|
|
|
"Cannot CREATE a table AS " + qualified_name + ", it is a Live View",
|
2019-05-29 21:52:13 +00:00
|
|
|
ErrorCodes::INCORRECT_QUERY);
|
|
|
|
|
2020-01-02 22:45:39 +00:00
|
|
|
if (as_create.is_dictionary)
|
|
|
|
throw Exception(
|
2020-01-02 23:05:44 +00:00
|
|
|
"Cannot CREATE a table AS " + qualified_name + ", it is a Dictionary",
|
2019-05-29 21:52:13 +00:00
|
|
|
ErrorCodes::INCORRECT_QUERY);
|
|
|
|
|
2020-11-16 10:14:12 +00:00
|
|
|
if (as_create.storage)
|
|
|
|
create.set(create.storage, as_create.storage->ptr());
|
|
|
|
else if (as_create.as_table_function)
|
|
|
|
create.as_table_function = as_create.as_table_function->clone();
|
|
|
|
else
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot set engine, it's a bug.");
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
2020-07-13 21:21:01 +00:00
|
|
|
void InterpreterCreateQuery::assertOrSetUUID(ASTCreateQuery & create, const DatabasePtr & database) const
|
|
|
|
{
|
2020-07-15 10:18:30 +00:00
|
|
|
const auto * kind = create.is_dictionary ? "Dictionary" : "Table";
|
|
|
|
const auto * kind_upper = create.is_dictionary ? "DICTIONARY" : "TABLE";
|
2020-07-13 21:21:01 +00:00
|
|
|
|
2021-02-08 19:36:17 +00:00
|
|
|
if (database->getEngineName() == "Replicated" && context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && !internal)
|
2020-10-20 16:14:54 +00:00
|
|
|
{
|
|
|
|
if (create.uuid == UUIDHelpers::Nil)
|
|
|
|
throw Exception("Table UUID is not specified in DDL log", ErrorCodes::LOGICAL_ERROR);
|
|
|
|
}
|
|
|
|
|
2020-12-08 14:13:35 +00:00
|
|
|
bool from_path = create.attach_from_path.has_value();
|
|
|
|
|
2020-09-14 19:25:02 +00:00
|
|
|
if (database->getUUID() != UUIDHelpers::Nil)
|
2020-07-13 21:21:01 +00:00
|
|
|
{
|
2020-12-08 14:13:35 +00:00
|
|
|
if (create.attach && !from_path && create.uuid == UUIDHelpers::Nil)
|
|
|
|
{
|
2020-07-13 21:21:01 +00:00
|
|
|
throw Exception(ErrorCodes::INCORRECT_QUERY,
|
2020-12-08 14:13:35 +00:00
|
|
|
"Incorrect ATTACH {} query for Atomic database engine. "
|
|
|
|
"Use one of the following queries instead:\n"
|
|
|
|
"1. ATTACH {} {};\n"
|
|
|
|
"2. CREATE {} {} <table definition>;\n"
|
|
|
|
"3. ATTACH {} {} FROM '/path/to/data/' <table definition>;\n"
|
|
|
|
"4. ATTACH {} {} UUID '<uuid>' <table definition>;",
|
|
|
|
kind_upper,
|
|
|
|
kind_upper, create.table,
|
|
|
|
kind_upper, create.table,
|
|
|
|
kind_upper, create.table,
|
|
|
|
kind_upper, create.table);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (create.uuid == UUIDHelpers::Nil)
|
2020-07-13 21:21:01 +00:00
|
|
|
create.uuid = UUIDHelpers::generateV4();
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
bool is_on_cluster = context.getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY;
|
|
|
|
if (create.uuid != UUIDHelpers::Nil && !is_on_cluster)
|
|
|
|
throw Exception(ErrorCodes::INCORRECT_QUERY,
|
2020-07-17 13:11:44 +00:00
|
|
|
"{} UUID specified, but engine of database {} is not Atomic", kind, create.database);
|
2020-07-13 21:21:01 +00:00
|
|
|
|
|
|
|
/// Ignore UUID if it's ON CLUSTER query
|
|
|
|
create.uuid = UUIDHelpers::Nil;
|
|
|
|
}
|
2020-12-25 16:28:10 +00:00
|
|
|
|
|
|
|
if (create.replace_table)
|
|
|
|
{
|
|
|
|
if (database->getUUID() == UUIDHelpers::Nil)
|
|
|
|
throw Exception(ErrorCodes::INCORRECT_QUERY,
|
|
|
|
"{} query is supported only for Atomic databases",
|
|
|
|
create.create_or_replace ? "CREATE OR REPLACE TABLE" : "REPLACE TABLE");
|
|
|
|
|
|
|
|
UUID uuid_of_table_to_replace;
|
|
|
|
if (create.create_or_replace)
|
|
|
|
{
|
|
|
|
uuid_of_table_to_replace = context.tryResolveStorageID(StorageID(create.database, create.table)).uuid;
|
|
|
|
if (uuid_of_table_to_replace == UUIDHelpers::Nil)
|
|
|
|
{
|
|
|
|
/// Convert to usual CREATE
|
|
|
|
create.replace_table = false;
|
|
|
|
assert(!database->isTableExist(create.table, context));
|
|
|
|
}
|
|
|
|
else
|
|
|
|
create.table = "_tmp_replace_" + toString(uuid_of_table_to_replace);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
uuid_of_table_to_replace = context.resolveStorageID(StorageID(create.database, create.table)).uuid;
|
|
|
|
if (uuid_of_table_to_replace == UUIDHelpers::Nil)
|
|
|
|
throw Exception(ErrorCodes::UNKNOWN_TABLE, "Table {}.{} doesn't exist",
|
|
|
|
backQuoteIfNeed(create.database), backQuoteIfNeed(create.table));
|
|
|
|
create.table = "_tmp_replace_" + toString(uuid_of_table_to_replace);
|
|
|
|
}
|
|
|
|
}
|
2020-07-13 21:21:01 +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
|
|
|
{
|
2019-02-11 14:46:09 +00:00
|
|
|
/// Temporary tables are created out of databases.
|
2020-02-14 18:20:20 +00:00
|
|
|
if (create.temporary && !create.database.empty())
|
2019-02-11 14:46:09 +00:00
|
|
|
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);
|
|
|
|
|
2020-02-10 13:10:17 +00:00
|
|
|
String current_database = context.getCurrentDatabase();
|
2020-07-04 16:32:23 +00:00
|
|
|
auto database_name = create.database.empty() ? current_database : create.database;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
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
|
|
|
{
|
2021-01-28 19:02:39 +00:00
|
|
|
auto database = DatabaseCatalog::instance().getDatabase(database_name);
|
2020-02-03 10:38:09 +00:00
|
|
|
bool if_not_exists = create.if_not_exists;
|
2020-03-30 14:53:05 +00:00
|
|
|
|
2020-11-30 17:52:32 +00:00
|
|
|
// Table SQL definition is available even if the table is detached (even permanently)
|
2020-05-28 20:10:45 +00:00
|
|
|
auto query = database->getCreateTableQuery(create.table, context);
|
2019-03-15 16:14:13 +00:00
|
|
|
create = query->as<ASTCreateQuery &>(); // Copy the saved create query, but use ATTACH instead of CREATE
|
2020-11-11 14:34:58 +00:00
|
|
|
if (create.is_dictionary)
|
2020-12-25 16:28:10 +00:00
|
|
|
throw Exception(ErrorCodes::INCORRECT_QUERY,
|
|
|
|
"Cannot ATTACH TABLE {}.{}, it is a Dictionary",
|
|
|
|
backQuoteIfNeed(database_name), backQuoteIfNeed(create.table));
|
2017-10-21 20:38:39 +00:00
|
|
|
create.attach = true;
|
2020-03-23 00:12:13 +00:00
|
|
|
create.attach_short_syntax = true;
|
2020-02-03 10:38:09 +00:00
|
|
|
create.if_not_exists = if_not_exists;
|
2017-10-21 20:38:39 +00:00
|
|
|
}
|
2020-12-08 14:13:35 +00:00
|
|
|
|
|
|
|
/// TODO throw exception if !create.attach_short_syntax && !create.attach_from_path && !internal
|
|
|
|
|
|
|
|
if (create.attach_from_path)
|
|
|
|
{
|
|
|
|
fs::path user_files = fs::path(context.getUserFilesPath()).lexically_normal();
|
|
|
|
fs::path root_path = fs::path(context.getPath()).lexically_normal();
|
2021-02-08 09:14:17 +00:00
|
|
|
|
|
|
|
if (context.getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY)
|
|
|
|
{
|
|
|
|
fs::path data_path = fs::path(*create.attach_from_path).lexically_normal();
|
|
|
|
if (data_path.is_relative())
|
|
|
|
data_path = (user_files / data_path).lexically_normal();
|
|
|
|
if (!startsWith(data_path, user_files))
|
|
|
|
throw Exception(ErrorCodes::PATH_ACCESS_DENIED,
|
|
|
|
"Data directory {} must be inside {} to attach it", String(data_path), String(user_files));
|
|
|
|
|
|
|
|
/// Data path must be relative to root_path
|
|
|
|
create.attach_from_path = fs::relative(data_path, root_path) / "";
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
fs::path data_path = (root_path / *create.attach_from_path).lexically_normal();
|
|
|
|
if (!startsWith(data_path, user_files))
|
|
|
|
throw Exception(ErrorCodes::PATH_ACCESS_DENIED,
|
|
|
|
"Data directory {} must be inside {} to attach it", String(data_path), String(user_files));
|
|
|
|
}
|
2020-12-08 14:13:35 +00:00
|
|
|
}
|
2021-02-08 19:36:17 +00:00
|
|
|
else if (create.attach && !create.attach_short_syntax && context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY)
|
2020-12-08 14:13:35 +00:00
|
|
|
{
|
|
|
|
auto * log = &Poco::Logger::get("InterpreterCreateQuery");
|
|
|
|
LOG_WARNING(log, "ATTACH TABLE query with full table definition is not recommended: "
|
|
|
|
"use either ATTACH TABLE {}; to attach existing table "
|
|
|
|
"or CREATE TABLE {} <table definition>; to create new table "
|
|
|
|
"or ATTACH TABLE {} FROM '/path/to/data/' <table definition>; to create new table and attach data.",
|
|
|
|
create.table, create.table, create.table);
|
|
|
|
}
|
2017-10-21 20:38:39 +00:00
|
|
|
|
2019-12-12 12:30:31 +00:00
|
|
|
if (!create.temporary && create.database.empty())
|
2019-12-05 11:42:13 +00:00
|
|
|
create.database = current_database;
|
2020-03-17 23:51:35 +00:00
|
|
|
if (create.to_table_id && create.to_table_id.database_name.empty())
|
2019-12-10 19:48:16 +00:00
|
|
|
create.to_table_id.database_name = current_database;
|
2017-10-30 17:53:01 +00:00
|
|
|
|
2020-12-16 03:19:38 +00:00
|
|
|
if (create.select && create.isView())
|
2018-11-01 14:49:37 +00:00
|
|
|
{
|
|
|
|
AddDefaultDatabaseVisitor visitor(current_database);
|
|
|
|
visitor.visit(*create.select);
|
|
|
|
}
|
2017-11-21 13:30:45 +00:00
|
|
|
|
2019-10-23 18:39:07 +00:00
|
|
|
/// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way.
|
|
|
|
TableProperties properties = setProperties(create);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2021-01-28 19:02:39 +00:00
|
|
|
DatabasePtr database;
|
2020-11-05 09:52:23 +00:00
|
|
|
bool need_add_to_database = !create.temporary;
|
2021-01-28 19:02:39 +00:00
|
|
|
if (need_add_to_database)
|
|
|
|
database = DatabaseCatalog::instance().getDatabase(database_name);
|
|
|
|
|
2020-11-29 11:45:32 +00:00
|
|
|
if (need_add_to_database && database->getEngineName() == "Replicated")
|
2020-10-22 15:08:00 +00:00
|
|
|
{
|
2020-11-05 09:52:23 +00:00
|
|
|
auto guard = DatabaseCatalog::instance().getDDLGuard(create.database, create.table);
|
|
|
|
database = DatabaseCatalog::instance().getDatabase(create.database);
|
2021-02-08 19:36:17 +00:00
|
|
|
if (typeid_cast<DatabaseReplicated *>(database.get()) && context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY)
|
2020-11-05 09:52:23 +00:00
|
|
|
{
|
|
|
|
assertOrSetUUID(create, database);
|
2021-02-02 19:39:04 +00:00
|
|
|
guard->releaseTableLock();
|
2021-02-04 19:41:44 +00:00
|
|
|
return typeid_cast<DatabaseReplicated *>(database.get())->propose(query_ptr, context);
|
2020-11-05 09:52:23 +00:00
|
|
|
}
|
2020-07-04 16:32:23 +00:00
|
|
|
}
|
|
|
|
|
2020-12-25 16:28:10 +00:00
|
|
|
if (create.replace_table)
|
|
|
|
return doCreateOrReplaceTable(create, properties);
|
|
|
|
|
2020-11-05 09:52:23 +00:00
|
|
|
/// Actually creates table
|
|
|
|
bool created = doCreateTable(create, properties);
|
|
|
|
|
2019-12-16 18:29:18 +00:00
|
|
|
if (!created) /// Table already exists
|
2019-10-23 18:39:07 +00:00
|
|
|
return {};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2019-12-16 18:29:18 +00:00
|
|
|
return fillTableIfNeeded(create);
|
2019-10-23 18:39:07 +00:00
|
|
|
}
|
2019-07-18 18:29:49 +00:00
|
|
|
|
2020-04-06 23:22:44 +00:00
|
|
|
bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
|
2019-12-05 11:42:13 +00:00
|
|
|
const InterpreterCreateQuery::TableProperties & properties)
|
2019-10-23 18:39:07 +00:00
|
|
|
{
|
|
|
|
std::unique_ptr<DDLGuard> guard;
|
|
|
|
|
|
|
|
String data_path;
|
|
|
|
DatabasePtr database;
|
|
|
|
|
2020-02-14 18:20:20 +00:00
|
|
|
bool need_add_to_database = !create.temporary;
|
2019-10-23 18:39:07 +00:00
|
|
|
if (need_add_to_database)
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-10-23 18:39:07 +00:00
|
|
|
/** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing).
|
2020-08-08 01:21:04 +00:00
|
|
|
* If table doesn't exist, one thread is creating table, while others wait in DDLGuard.
|
2019-10-23 18:39:07 +00:00
|
|
|
*/
|
2020-12-25 16:28:10 +00:00
|
|
|
guard = DatabaseCatalog::instance().getDDLGuard(create.database, create.table);
|
2019-10-23 18:39:07 +00:00
|
|
|
|
2020-08-18 15:15:27 +00:00
|
|
|
database = DatabaseCatalog::instance().getDatabase(create.database);
|
|
|
|
assertOrSetUUID(create, database);
|
|
|
|
|
2019-10-23 18:39:07 +00:00
|
|
|
/// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard.
|
2020-12-25 16:28:10 +00:00
|
|
|
if (database->isTableExist(create.table, context))
|
2019-10-23 18:39:07 +00:00
|
|
|
{
|
2019-10-29 14:11:08 +00:00
|
|
|
/// TODO Check structure of table
|
2019-10-23 18:39:07 +00:00
|
|
|
if (create.if_not_exists)
|
|
|
|
return false;
|
|
|
|
else if (create.replace_view)
|
|
|
|
{
|
|
|
|
/// when executing CREATE OR REPLACE VIEW, drop current existing view
|
|
|
|
auto drop_ast = std::make_shared<ASTDropQuery>();
|
2019-12-16 18:29:18 +00:00
|
|
|
drop_ast->database = create.database;
|
2020-12-25 16:28:10 +00:00
|
|
|
drop_ast->table = create.table;
|
2019-10-23 18:39:07 +00:00
|
|
|
drop_ast->no_ddl_lock = true;
|
|
|
|
|
2021-01-28 13:48:17 +00:00
|
|
|
Context drop_context = context;
|
|
|
|
InterpreterDropQuery interpreter(drop_ast, drop_context);
|
2019-10-23 18:39:07 +00:00
|
|
|
interpreter.execute();
|
|
|
|
}
|
|
|
|
else
|
2020-12-25 16:28:10 +00:00
|
|
|
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Table {}.{} already exists.", backQuoteIfNeed(create.database), backQuoteIfNeed(create.table));
|
2019-10-23 18:39:07 +00:00
|
|
|
}
|
2020-03-18 17:38:52 +00:00
|
|
|
|
|
|
|
data_path = database->getTableDataPath(create);
|
2020-07-02 20:39:31 +00:00
|
|
|
if (!create.attach && !data_path.empty() && fs::exists(fs::path{context.getPath()} / data_path))
|
2020-12-25 16:28:10 +00:00
|
|
|
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Directory for table data {} already exists", String(data_path));
|
2017-04-01 07:20:54 +00:00
|
|
|
}
|
2020-03-10 19:36:17 +00:00
|
|
|
else
|
|
|
|
{
|
2020-12-25 16:28:10 +00:00
|
|
|
if (create.if_not_exists && context.tryResolveStorageID({"", create.table}, Context::ResolveExternal))
|
2020-03-10 19:36:17 +00:00
|
|
|
return false;
|
|
|
|
|
2020-12-25 17:43:03 +00:00
|
|
|
String temporary_table_name = create.table;
|
2020-05-29 02:08:48 +00:00
|
|
|
auto temporary_table = TemporaryTableHolder(context, properties.columns, properties.constraints, query_ptr);
|
2020-12-25 17:43:03 +00:00
|
|
|
context.getSessionContext().addExternalTable(temporary_table_name, std::move(temporary_table));
|
2020-03-10 19:36:17 +00:00
|
|
|
return true;
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-12-08 14:13:35 +00:00
|
|
|
bool from_path = create.attach_from_path.has_value();
|
|
|
|
String actual_data_path = data_path;
|
|
|
|
if (from_path)
|
|
|
|
{
|
|
|
|
if (data_path.empty())
|
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
|
|
|
"ATTACH ... FROM ... query is not supported for {} database engine", database->getEngineName());
|
|
|
|
/// We will try to create Storage instance with provided data path
|
|
|
|
data_path = *create.attach_from_path;
|
|
|
|
create.attach_from_path = std::nullopt;
|
|
|
|
}
|
|
|
|
|
2019-07-22 13:49:16 +00:00
|
|
|
StoragePtr res;
|
2020-02-20 20:24:04 +00:00
|
|
|
/// NOTE: CREATE query may be rewritten by Storage creator or table function
|
2019-07-18 18:29:49 +00:00
|
|
|
if (create.as_table_function)
|
2019-05-28 15:59:37 +00:00
|
|
|
{
|
2019-07-18 18:29:49 +00:00
|
|
|
const auto & factory = TableFunctionFactory::instance();
|
2020-10-14 12:19:29 +00:00
|
|
|
res = factory.get(create.as_table_function, context)->execute(create.as_table_function, context, create.table, properties.columns);
|
2020-04-07 14:05:51 +00:00
|
|
|
res->renameInMemory({create.database, create.table, create.uuid});
|
2019-07-18 18:29:49 +00:00
|
|
|
}
|
2019-07-22 12:18:53 +00:00
|
|
|
else
|
2017-04-01 07:20:54 +00:00
|
|
|
{
|
2019-10-23 18:39:07 +00:00
|
|
|
res = StorageFactory::instance().get(create,
|
2020-12-08 14:13:35 +00:00
|
|
|
data_path,
|
2019-10-23 18:39:07 +00:00
|
|
|
context,
|
|
|
|
context.getGlobalContext(),
|
|
|
|
properties.columns,
|
|
|
|
properties.constraints,
|
|
|
|
false);
|
2019-07-22 12:18:53 +00:00
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
2020-12-08 14:13:35 +00:00
|
|
|
if (from_path && !res->storesDataOnDisk())
|
|
|
|
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
|
|
|
|
"ATTACH ... FROM ... query is not supported for {} table engine, "
|
|
|
|
"because such tables do not store any data on disk. Use CREATE instead.", res->getName());
|
|
|
|
|
2020-12-25 16:28:10 +00:00
|
|
|
database->createTable(context, create.table, res, query_ptr);
|
2019-10-23 18:39:07 +00:00
|
|
|
|
2020-12-08 14:13:35 +00:00
|
|
|
/// Move table data to the proper place. Wo do not move data earlier to avoid situations
|
|
|
|
/// when data directory moved, but table has not been created due to some error.
|
|
|
|
if (from_path)
|
|
|
|
res->rename(actual_data_path, {create.database, create.table, create.uuid});
|
2019-10-23 18:39:07 +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.
|
|
|
|
|
2020-06-03 22:11:06 +00:00
|
|
|
/// Also note that "startup" method is exception-safe. If exception is thrown from "startup",
|
|
|
|
/// we can safely destroy the object without a call to "shutdown", because there is guarantee
|
|
|
|
/// that no background threads/similar resources remain after exception from "startup".
|
|
|
|
|
2019-10-23 18:39:07 +00:00
|
|
|
res->startup();
|
|
|
|
return true;
|
|
|
|
}
|
2017-06-06 17:06:14 +00:00
|
|
|
|
2020-12-25 16:28:10 +00:00
|
|
|
|
|
|
|
BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create,
|
|
|
|
const InterpreterCreateQuery::TableProperties & properties)
|
|
|
|
{
|
|
|
|
auto ast_drop = std::make_shared<ASTDropQuery>();
|
|
|
|
String table_to_replace_name = create.table;
|
|
|
|
bool created = false;
|
|
|
|
bool replaced = false;
|
|
|
|
|
|
|
|
try
|
|
|
|
{
|
|
|
|
[[maybe_unused]] bool done = doCreateTable(create, properties);
|
|
|
|
assert(done);
|
|
|
|
ast_drop->table = create.table;
|
|
|
|
ast_drop->database = create.database;
|
|
|
|
ast_drop->kind = ASTDropQuery::Drop;
|
|
|
|
created = true;
|
|
|
|
if (!create.replace_table)
|
|
|
|
return fillTableIfNeeded(create);
|
|
|
|
|
|
|
|
auto ast_rename = std::make_shared<ASTRenameQuery>();
|
|
|
|
ASTRenameQuery::Element elem
|
|
|
|
{
|
|
|
|
ASTRenameQuery::Table{create.database, create.table},
|
|
|
|
ASTRenameQuery::Table{create.database, table_to_replace_name}
|
|
|
|
};
|
|
|
|
ast_rename->elements.push_back(std::move(elem));
|
|
|
|
ast_rename->exchange = true;
|
|
|
|
InterpreterRenameQuery(ast_rename, context).execute();
|
|
|
|
replaced = true;
|
|
|
|
|
|
|
|
InterpreterDropQuery(ast_drop, context).execute();
|
|
|
|
|
|
|
|
create.table = table_to_replace_name;
|
|
|
|
return fillTableIfNeeded(create);
|
|
|
|
}
|
|
|
|
catch (...)
|
|
|
|
{
|
|
|
|
if (created && create.replace_table && !replaced)
|
|
|
|
InterpreterDropQuery(ast_drop, context).execute();
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-12-16 18:29:18 +00:00
|
|
|
BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create)
|
2019-10-23 18:39:07 +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
|
2020-12-16 03:19:38 +00:00
|
|
|
&& !create.is_ordinary_view && !create.is_live_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>();
|
2020-03-03 19:53:18 +00:00
|
|
|
insert->table_id = {create.database, create.table, create.uuid};
|
2018-02-19 00:45:32 +00:00
|
|
|
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())
|
2019-07-08 00:51:43 +00:00
|
|
|
context.getSessionContext().makeQueryContext();
|
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
|
|
|
}
|
|
|
|
|
2019-10-11 13:21:52 +00:00
|
|
|
BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create)
|
|
|
|
{
|
|
|
|
String dictionary_name = create.table;
|
|
|
|
|
2020-02-10 13:10:17 +00:00
|
|
|
create.database = context.resolveDatabase(create.database);
|
2019-12-25 23:12:12 +00:00
|
|
|
const String & database_name = create.database;
|
2019-10-11 13:21:52 +00:00
|
|
|
|
2020-02-10 18:31:52 +00:00
|
|
|
auto guard = DatabaseCatalog::instance().getDDLGuard(database_name, dictionary_name);
|
2020-02-10 13:10:17 +00:00
|
|
|
DatabasePtr database = DatabaseCatalog::instance().getDatabase(database_name);
|
2019-10-11 13:21:52 +00:00
|
|
|
|
2021-02-08 19:36:17 +00:00
|
|
|
if (typeid_cast<DatabaseReplicated *>(database.get()) && context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY)
|
2021-02-08 09:46:30 +00:00
|
|
|
{
|
2021-02-08 19:36:17 +00:00
|
|
|
if (!create.attach)
|
|
|
|
assertOrSetUUID(create, database);
|
2021-02-08 09:46:30 +00:00
|
|
|
guard->releaseTableLock();
|
|
|
|
return typeid_cast<DatabaseReplicated *>(database.get())->propose(query_ptr, context);
|
|
|
|
}
|
|
|
|
|
2020-04-23 16:51:48 +00:00
|
|
|
if (database->isDictionaryExist(dictionary_name))
|
2019-10-11 13:21:52 +00:00
|
|
|
{
|
2019-10-22 11:03:16 +00:00
|
|
|
/// TODO Check structure of dictionary
|
2019-10-11 13:21:52 +00:00
|
|
|
if (create.if_not_exists)
|
|
|
|
return {};
|
|
|
|
else
|
|
|
|
throw Exception(
|
|
|
|
"Dictionary " + database_name + "." + dictionary_name + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
|
|
|
|
}
|
|
|
|
|
|
|
|
if (create.attach)
|
2019-10-15 18:04:17 +00:00
|
|
|
{
|
2020-04-23 16:51:48 +00:00
|
|
|
auto query = DatabaseCatalog::instance().getDatabase(database_name)->getCreateDictionaryQuery(dictionary_name);
|
2019-10-15 18:04:17 +00:00
|
|
|
create = query->as<ASTCreateQuery &>();
|
|
|
|
create.attach = true;
|
2020-07-13 21:21:01 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
assertOrSetUUID(create, database);
|
|
|
|
|
|
|
|
if (create.attach)
|
|
|
|
{
|
2020-11-17 13:24:13 +00:00
|
|
|
auto config = getDictionaryConfigurationFromAST(create, context);
|
2020-04-12 20:50:32 +00:00
|
|
|
auto modification_time = database->getObjectMetadataModificationTime(dictionary_name);
|
|
|
|
database->attachDictionary(dictionary_name, DictionaryAttachInfo{query_ptr, config, modification_time});
|
|
|
|
}
|
2019-10-11 13:21:52 +00:00
|
|
|
else
|
2019-10-16 14:59:52 +00:00
|
|
|
database->createDictionary(context, dictionary_name, query_ptr);
|
2019-10-11 13:21:52 +00:00
|
|
|
|
|
|
|
return {};
|
|
|
|
}
|
2017-04-13 13:42:29 +00:00
|
|
|
|
2020-09-23 18:28:59 +00:00
|
|
|
void InterpreterCreateQuery::prepareOnClusterQuery(ASTCreateQuery & create, const Context & context, const String & cluster_name)
|
|
|
|
{
|
|
|
|
if (create.attach)
|
|
|
|
return;
|
|
|
|
|
|
|
|
/// For CREATE query generate UUID on initiator, so it will be the same on all hosts.
|
|
|
|
/// It will be ignored if database does not support UUIDs.
|
|
|
|
if (create.uuid == UUIDHelpers::Nil)
|
|
|
|
create.uuid = UUIDHelpers::generateV4();
|
|
|
|
|
|
|
|
/// For cross-replication cluster we cannot use UUID in replica path.
|
|
|
|
String cluster_name_expanded = context.getMacros()->expand(cluster_name);
|
|
|
|
ClusterPtr cluster = context.getCluster(cluster_name_expanded);
|
|
|
|
|
|
|
|
if (cluster->maybeCrossReplication())
|
|
|
|
{
|
|
|
|
/// Check that {uuid} macro is not used in zookeeper_path for ReplicatedMergeTree.
|
|
|
|
/// Otherwise replicas will generate different paths.
|
|
|
|
if (!create.storage)
|
|
|
|
return;
|
|
|
|
if (!create.storage->engine)
|
|
|
|
return;
|
|
|
|
if (!startsWith(create.storage->engine->name, "Replicated"))
|
|
|
|
return;
|
|
|
|
|
|
|
|
bool has_explicit_zk_path_arg = create.storage->engine->arguments &&
|
|
|
|
create.storage->engine->arguments->children.size() >= 2 &&
|
|
|
|
create.storage->engine->arguments->children[0]->as<ASTLiteral>() &&
|
|
|
|
create.storage->engine->arguments->children[0]->as<ASTLiteral>()->value.getType() == Field::Types::String;
|
|
|
|
|
|
|
|
if (has_explicit_zk_path_arg)
|
|
|
|
{
|
|
|
|
String zk_path = create.storage->engine->arguments->children[0]->as<ASTLiteral>()->value.get<String>();
|
|
|
|
Macros::MacroExpansionInfo info;
|
2020-09-26 19:18:28 +00:00
|
|
|
info.table_id.uuid = create.uuid;
|
2020-09-23 18:28:59 +00:00
|
|
|
info.ignore_unknown = true;
|
|
|
|
context.getMacros()->expand(zk_path, info);
|
|
|
|
if (!info.expanded_uuid)
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
throw Exception("Seems like cluster is configured for cross-replication, "
|
|
|
|
"but zookeeper_path for ReplicatedMergeTree is not specified or contains {uuid} macro. "
|
|
|
|
"It's not supported for cross replication, because tables must have different UUIDs. "
|
|
|
|
"Please specify unique zookeeper_path explicitly.", ErrorCodes::INCORRECT_QUERY);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-03-19 01:18:49 +00:00
|
|
|
BlockIO InterpreterCreateQuery::execute()
|
|
|
|
{
|
2019-03-15 16:14:13 +00:00
|
|
|
auto & create = query_ptr->as<ASTCreateQuery &>();
|
2020-01-24 16:20:36 +00:00
|
|
|
if (!create.cluster.empty())
|
2020-03-16 11:38:50 +00:00
|
|
|
{
|
2020-09-23 18:28:59 +00:00
|
|
|
prepareOnClusterQuery(create, context, create.cluster);
|
2020-01-24 16:20:36 +00:00
|
|
|
return executeDDLQueryOnCluster(query_ptr, context, getRequiredAccess());
|
2020-03-16 11:38:50 +00:00
|
|
|
}
|
2020-01-24 16:20:36 +00:00
|
|
|
|
|
|
|
context.checkAccess(getRequiredAccess());
|
|
|
|
|
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);
|
2019-10-11 13:21:52 +00:00
|
|
|
else if (!create.is_dictionary)
|
2017-04-21 12:39:28 +00:00
|
|
|
return createTable(create);
|
2019-10-11 13:21:52 +00:00
|
|
|
else
|
|
|
|
return createDictionary(create);
|
2016-03-19 01:18:49 +00:00
|
|
|
}
|
|
|
|
|
2017-12-22 19:20:18 +00:00
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
AccessRightsElements InterpreterCreateQuery::getRequiredAccess() const
|
2017-12-20 07:39:52 +00:00
|
|
|
{
|
2018-01-18 23:40:32 +00:00
|
|
|
/// Internal queries (initiated by the server itself) always have access to everything.
|
|
|
|
if (internal)
|
2020-01-24 16:20:36 +00:00
|
|
|
return {};
|
2017-12-20 07:39:52 +00:00
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
AccessRightsElements required_access;
|
|
|
|
const auto & create = query_ptr->as<const ASTCreateQuery &>();
|
2016-03-19 01:18:49 +00:00
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
if (create.table.empty())
|
2017-12-20 07:39:52 +00:00
|
|
|
{
|
2020-01-24 16:20:36 +00:00
|
|
|
required_access.emplace_back(AccessType::CREATE_DATABASE, create.database);
|
2017-12-20 07:39:52 +00:00
|
|
|
}
|
2020-01-24 16:20:36 +00:00
|
|
|
else if (create.is_dictionary)
|
2019-10-18 15:44:32 +00:00
|
|
|
{
|
2020-01-24 16:20:36 +00:00
|
|
|
required_access.emplace_back(AccessType::CREATE_DICTIONARY, create.database, create.table);
|
|
|
|
}
|
2020-12-16 04:07:50 +00:00
|
|
|
else if (create.isView())
|
2020-01-24 16:20:36 +00:00
|
|
|
{
|
2020-12-25 16:28:10 +00:00
|
|
|
assert(!create.temporary);
|
|
|
|
if (create.replace_view)
|
|
|
|
required_access.emplace_back(AccessType::DROP_VIEW | AccessType::CREATE_VIEW, create.database, create.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
else
|
2020-12-25 16:28:10 +00:00
|
|
|
required_access.emplace_back(AccessType::CREATE_VIEW, create.database, create.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
if (create.temporary)
|
|
|
|
required_access.emplace_back(AccessType::CREATE_TEMPORARY_TABLE);
|
|
|
|
else
|
2020-12-25 16:28:10 +00:00
|
|
|
{
|
|
|
|
if (create.replace_table)
|
|
|
|
required_access.emplace_back(AccessType::DROP_TABLE, create.database, create.table);
|
2020-01-24 16:20:36 +00:00
|
|
|
required_access.emplace_back(AccessType::CREATE_TABLE, create.database, create.table);
|
2020-12-25 16:28:10 +00:00
|
|
|
}
|
2019-10-18 15:44:32 +00:00
|
|
|
}
|
2017-12-20 07:39:52 +00:00
|
|
|
|
2020-03-16 11:38:50 +00:00
|
|
|
if (create.to_table_id)
|
2020-04-08 19:00:46 +00:00
|
|
|
required_access.emplace_back(AccessType::SELECT | AccessType::INSERT, create.to_table_id.database_name, create.to_table_id.table_name);
|
2020-04-06 05:19:40 +00:00
|
|
|
|
|
|
|
if (create.storage && create.storage->engine)
|
|
|
|
{
|
|
|
|
auto source_access_type = StorageFactory::instance().getSourceAccessType(create.storage->engine->name);
|
|
|
|
if (source_access_type != AccessType::NONE)
|
|
|
|
required_access.emplace_back(source_access_type);
|
|
|
|
}
|
2018-09-11 18:37:19 +00:00
|
|
|
|
2020-01-24 16:20:36 +00:00
|
|
|
return required_access;
|
2017-12-20 07:39:52 +00:00
|
|
|
}
|
2019-10-18 15:44:32 +00:00
|
|
|
|
2021-02-08 19:36:17 +00:00
|
|
|
void InterpreterCreateQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, const Context &) const
|
2020-12-14 03:30:39 +00:00
|
|
|
{
|
|
|
|
elem.query_kind = "Create";
|
2021-02-08 19:36:17 +00:00
|
|
|
if (!as_table_saved.empty())
|
2020-12-14 03:30:39 +00:00
|
|
|
{
|
2021-02-08 19:36:17 +00:00
|
|
|
String database = backQuoteIfNeed(as_database_saved.empty() ? context.getCurrentDatabase() : as_database_saved);
|
2020-12-14 03:30:39 +00:00
|
|
|
elem.query_databases.insert(database);
|
2021-02-08 19:36:17 +00:00
|
|
|
elem.query_tables.insert(database + "." + backQuoteIfNeed(as_table_saved));
|
2020-12-14 03:30:39 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2011-08-18 20:33:20 +00:00
|
|
|
}
|