ClickHouse/dbms/src/Interpreters/InterpreterCreateQuery.cpp

816 lines
31 KiB
C++
Raw Normal View History

Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
#include <memory>
2011-11-05 23:31:19 +00:00
#include <Poco/File.h>
2011-10-24 12:10:59 +00:00
#include <Common/StringUtils/StringUtils.h>
#include <Common/escapeForFileName.h>
2018-11-08 12:03:42 +00:00
#include <Common/typeid_cast.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteHelpers.h>
#include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTIdentifier.h>
2019-10-07 14:18:18 +00:00
#include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTNameTypePair.h>
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/parseQuery.h>
#include <Storages/StorageFactory.h>
2017-05-23 18:33:48 +00:00
#include <Interpreters/Context.h>
#include <Interpreters/DDLWorker.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/SyntaxAnalyzer.h>
#include <Interpreters/InterpreterCreateQuery.h>
2018-02-27 19:38:59 +00:00
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/InterpreterInsertQuery.h>
2018-02-27 19:38:59 +00:00
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/AddDefaultDatabaseVisitor.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>
#include <Databases/DatabaseFactory.h>
#include <Databases/IDatabase.h>
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
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>
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>
2019-11-11 11:34:03 +00:00
#include <boost/uuid/uuid_generators.hpp>
#include <boost/uuid/uuid_io.hpp>
2011-08-18 20:33:20 +00:00
namespace DB
{
namespace ErrorCodes
{
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;
extern const int READONLY;
extern const int ILLEGAL_COLUMN;
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;
extern const int BAD_DATABASE_FOR_TEMPORARY_TABLE;
extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY;
extern const int DICTIONARY_ALREADY_EXISTS;
}
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_)
{
}
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())
return executeDDLQueryOnCluster(query_ptr, context, {create.database});
2017-04-25 15:21:03 +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))
{
if (create.if_not_exists)
return {};
else
throw Exception("Database " + database_name + " already exists.", ErrorCodes::DATABASE_ALREADY_EXISTS);
}
if (!create.storage)
{
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
//FIXME is it possible, that database engine is not specified in metadata file?
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;
auto engine = std::make_shared<ASTFunction>();
auto storage = std::make_shared<ASTStorage>();
2019-11-26 19:06:19 +00:00
engine->name = old_style_database ? "Ordinary" : "Atomic";
storage->set(storage->engine, engine);
create.set(create.storage, storage);
2019-11-26 19:06:19 +00:00
if (database_name == "datasets")
{
//FIXME it's just to run stateful and stress tests without updating docker images
engine->name = "Ordinary";
}
}
2019-11-26 19:06:19 +00:00
else if ((create.columns_list && create.columns_list->indices && !create.columns_list->indices->children.empty()))
{
/// Currently, there are no database engines, that support any arguments.
2019-11-26 19:06:19 +00:00
std::stringstream ostr;
formatAST(*create.storage, ostr, false, false);
throw Exception("Unknown database engine: " + ostr.str(), ErrorCodes::UNKNOWN_DATABASE_ENGINE);
}
2019-11-26 19:06:19 +00:00
String database_name_escaped = escapeForFileName(database_name);
String path = context.getPath();
String metadata_path = path + "metadata/" + database_name_escaped + "/";
DatabasePtr database = DatabaseFactory::get(database_name, metadata_path, create.storage, 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);
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();
}
2019-11-26 19:06:19 +00:00
bool added = false;
bool renamed = false;
try
{
context.addDatabase(database_name, database);
2019-11-26 19:06:19 +00:00
added = true;
if (need_write_metadata)
2019-11-26 19:06:19 +00:00
{
Poco::File(metadata_file_tmp_path).renameTo(metadata_file_path);
2019-11-26 19:06:19 +00:00
renamed = true;
}
2019-10-10 17:33:01 +00:00
database->loadStoredObjects(context, has_force_restore_data_flag);
}
catch (...)
{
2019-11-26 19:06:19 +00:00
if (renamed)
Poco::File(metadata_file_tmp_path).remove();
2019-11-26 19:06:19 +00:00
if (added)
context.detachDatabase(database_name);
throw;
}
2017-04-25 15:21:03 +00:00
return {};
2011-08-18 20:33:20 +00:00
}
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
2014-07-10 11:13:45 +00:00
{
auto columns_list = std::make_shared<ASTExpressionList>();
2014-07-10 11:13:45 +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
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);
columns_list->children.emplace_back(column_declaration);
}
2014-07-10 11:13:45 +00:00
return columns_list;
2014-07-10 11:13:45 +00:00
}
ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
{
auto columns_list = std::make_shared<ASTExpressionList>();
for (const auto & column : columns)
{
/// Do not include virtual columns
if (column.is_virtual)
continue;
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
ASTPtr column_declaration_ptr{column_declaration};
column_declaration->name = column.name;
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();
column_declaration->type = parseQuery(storage_p, type_name_pos, type_name_end, "data type", 0);
if (column.default_desc.expression)
{
column_declaration->default_specifier = toString(column.default_desc.kind);
column_declaration->default_expression = column.default_desc.expression->clone();
}
if (!column.comment.empty())
{
column_declaration->comment = std::make_shared<ASTLiteral>(Field(column.comment));
}
if (column.codec)
2018-10-11 02:57:48 +00:00
{
String codec_desc = column.codec->getCodecDesc();
2018-10-11 02:57:48 +00:00
codec_desc = "CODEC(" + codec_desc + ")";
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;
column_declaration->codec = parseQuery(codec_p, codec_desc_pos, codec_desc_end, "column codec", 0);
2018-10-11 02:57:48 +00:00
}
if (column.ttl)
column_declaration->ttl = column.ttl;
columns_list->children.push_back(column_declaration_ptr);
}
2014-07-10 11:13:45 +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;
}
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;
}
ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpressionList & columns_ast, const Context & context)
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
{
/// First, deduce implicit types.
/** 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>();
NamesAndTypesList column_names_and_types;
for (const auto & ast : columns_ast.children)
{
const auto & col_decl = ast->as<ASTColumnDeclaration &>();
DataTypePtr column_type = nullptr;
if (col_decl.type)
{
column_type = DataTypeFactory::instance().get(col_decl.type);
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
*/
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 = column_names_and_types.back().type.get();
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));
}
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())
{
auto syntax_analyzer_result = SyntaxAnalyzer(context).analyze(default_expr_list, column_names_and_types);
const auto actions = ExpressionAnalyzer(default_expr_list, syntax_analyzer_result, context).getActions(true);
for (auto action : actions->getActions())
if (action.type == ExpressionAction::Type::JOIN || action.type == ExpressionAction::Type::ARRAY_JOIN)
throw Exception("Cannot CREATE table. Unsupported default value that requires ARRAY JOIN or JOIN action", ErrorCodes::THERE_IS_NO_DEFAULT_VALUE);
defaults_sample_block = actions->getSampleBlock();
}
ColumnsDescription res;
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;
auto & col_decl = (*ast_it)->as<ASTColumnDeclaration &>();
column.name = col_decl.name;
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
if (col_decl.default_expression)
{
ASTPtr default_expr = col_decl.default_expression->clone();
if (col_decl.type)
{
const auto & deduced_type = defaults_sample_block.getByName(column.name + "_tmp").type;
column.type = name_type_it->type;
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
if (!column.type->equals(*deduced_type))
2019-06-16 16:47:47 +00:00
default_expr = addTypeConversionToAST(std::move(default_expr), column.type->getName());
}
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)
column.comment = col_decl.comment->as<ASTLiteral &>().value.get<String>();
if (col_decl.codec)
column.codec = CompressionCodecFactory::instance().get(col_decl.codec, column.type);
if (col_decl.ttl)
column.ttl = col_decl.ttl;
res.add(std::move(column));
}
res.flattenNested();
if (res.getAllPhysical().empty())
throw Exception{"Cannot CREATE table without physical columns", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED};
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
return res;
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
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;
}
InterpreterCreateQuery::TableProperties InterpreterCreateQuery::setProperties(ASTCreateQuery & create) const
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
{
TableProperties properties;
TableStructureReadLockHolder as_storage_lock;
2019-02-05 13:52:45 +00:00
if (create.columns_list)
{
2019-02-05 13:52:45 +00:00
if (create.columns_list->columns)
properties.columns = getColumnsDescription(*create.columns_list->columns, context);
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)
properties.indices.indices.push_back(
std::dynamic_pointer_cast<ASTIndexDeclaration>(index->clone()));
2019-08-24 21:20:20 +00:00
properties.constraints = getConstraintsDescription(create.columns_list->constraints);
}
else if (!create.as_table.empty())
{
String as_database_name = create.as_database.empty() ? context.getCurrentDatabase() : create.as_database;
StoragePtr as_storage = context.getTable(as_database_name, create.as_table);
/// as_storage->getColumns() and setEngine(...) must be called under structure lock of other_table for CREATE ... AS other_table.
as_storage_lock = as_storage->lockStructureForShare(false, context.getCurrentQueryId());
properties.columns = as_storage->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.
if (create.storage && endsWith(create.storage->engine->name, "MergeTree"))
properties.indices = as_storage->getIndices();
2019-09-20 20:53:14 +00:00
properties.constraints = as_storage->getConstraints();
}
else if (create.select)
{
Block as_select_sample = InterpreterSelectWithUnionQuery::getSampleBlock(create.select->clone(), context);
properties.columns = ColumnsDescription(as_select_sample.getNamesAndTypesList());
}
else if (create.as_table_function)
return {};
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
/// Even if query has list of columns, canonicalize it (unfold Nested columns).
2019-01-20 15:02:19 +00:00
if (!create.columns_list)
create.set(create.columns_list, std::make_shared<ASTColumns>());
2019-01-20 11:03:21 +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);
validateTableStructure(create, properties);
2019-02-05 13:52:45 +00:00
/// Set the table engine if it was not specified explicitly.
setEngine(create);
return properties;
}
void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & create,
const InterpreterCreateQuery::TableProperties & properties) const
{
/// Check for duplicates
std::set<String> all_columns;
for (const auto & column : properties.columns)
{
if (!all_columns.emplace(column.name).second)
throw Exception("Column " + backQuoteIfNeed(column.name) + " already exists", ErrorCodes::DUPLICATE_COLUMN);
}
/// Check low cardinality types in creating table if it was not allowed in setting
if (!create.attach && !context.getSettingsRef().allow_suspicious_low_cardinality_types && !create.is_materialized_view)
{
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);
}
}
}
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
}
void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
{
if (create.storage)
{
if (create.temporary && create.storage->engine->name != "Memory")
throw Exception(
"Temporary tables can only be created with ENGINE = Memory, not " + create.storage->engine->name,
ErrorCodes::INCORRECT_QUERY);
return;
}
if (create.temporary && !create.is_live_view)
{
auto engine_ast = std::make_shared<ASTFunction>();
engine_ast->name = "Memory";
auto storage_ast = std::make_shared<ASTStorage>();
storage_ast->set(storage_ast->engine, engine_ast);
create.set(create.storage, storage_ast);
}
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.
String as_database_name = create.as_database.empty() ? context.getCurrentDatabase() : create.as_database;
String as_table_name = create.as_table;
2019-12-05 13:44:22 +00:00
ASTPtr as_create_ptr = context.getDatabase(as_database_name)->getCreateTableQuery(context, as_table_name);
const auto & as_create = as_create_ptr->as<ASTCreateQuery &>();
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",
ErrorCodes::INCORRECT_QUERY);
if (as_create.is_live_view)
throw Exception(
"Cannot CREATE a table AS " + as_database_name + "." + as_table_name + ", it is a Live View",
ErrorCodes::INCORRECT_QUERY);
create.set(create.storage, as_create.storage->ptr());
}
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
}
2017-04-21 12:39:28 +00:00
BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
{
2017-04-21 12:39:28 +00:00
if (!create.cluster.empty())
{
NameSet databases{create.database};
2019-12-10 19:48:16 +00:00
if (!create.to_table_id.empty())
databases.emplace(create.to_table_id.database_name);
2019-11-11 11:34:03 +00:00
/// NOTE: if it's CREATE query and create.database is DatabaseAtomic, different UUIDs will be generated on all servers.
/// However, it allows to use UUID as replica name.
return executeDDLQueryOnCluster(query_ptr, context, std::move(databases));
}
2017-04-21 12:39:28 +00:00
/// Temporary tables are created out of databases.
if (create.temporary && !create.database.empty() && !create.is_live_view)
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);
2019-12-05 11:42:13 +00:00
auto & database_name = create.database;
auto & table_name = create.table;
// 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-25 00:55:31 +00:00
// Table SQL definition is available even if the table is detached
2019-12-05 13:44:22 +00:00
auto query = context.getDatabase(database_name)->getCreateTableQuery(context, table_name);
create = query->as<ASTCreateQuery &>(); // Copy the saved create query, but use ATTACH instead of CREATE
create.attach = true;
}
2019-12-05 11:42:13 +00:00
String current_database = context.getCurrentDatabase();
if (create.database.empty())
create.database = current_database;
2019-12-10 19:48:16 +00:00
if (!create.to_table_id.empty() && create.to_table_id.database_name.empty())
create.to_table_id.database_name = current_database;
if (create.select && (create.is_view || create.is_materialized_view || create.is_live_view))
{
AddDefaultDatabaseVisitor visitor(current_database);
visitor.visit(*create.select);
}
/// Set and retrieve list of columns, indices and constraints. Set table engine if needed. Rewrite query in canonical way.
TableProperties properties = setProperties(create);
/// Actually creates table
2019-12-05 11:42:13 +00:00
bool created = doCreateTable(create, properties);
if (!created)
return {};
return fillTableIfNeeded(create, database_name);
}
2019-07-18 18:29:49 +00:00
2019-11-11 11:34:03 +00:00
bool InterpreterCreateQuery::doCreateTable(/*const*/ ASTCreateQuery & create,
2019-12-05 11:42:13 +00:00
const InterpreterCreateQuery::TableProperties & properties)
{
std::unique_ptr<DDLGuard> guard;
String data_path;
DatabasePtr database;
2019-12-05 11:42:13 +00:00
const String & database_name = create.database;
const String & table_name = create.table;
bool need_add_to_database = !create.temporary || create.is_live_view;
if (need_add_to_database)
{
database = context.getDatabase(database_name);
2019-11-11 11:34:03 +00:00
if (database->getEngineName() == "Atomic")
{
//TODO implement ATTACH FROM 'path/to/data': generate UUID and move table data to store/
if (create.attach && create.uuid.empty())
throw Exception("UUID must be specified in ATTACH TABLE query for Atomic database engine", ErrorCodes::INCORRECT_QUERY);
if (!create.attach && create.uuid.empty())
create.uuid = boost::uuids::to_string(boost::uuids::random_generator()());
}
else
{
if (!create.uuid.empty())
throw Exception("Table UUID specified, but engine of database " + database_name + " is not Atomic", ErrorCodes::INCORRECT_QUERY);
}
2019-11-11 11:34:03 +00:00
if (!create.attach && create.uuid.empty() && database->getEngineName() == "Atomic")
create.uuid = boost::uuids::to_string(boost::uuids::random_generator()());
data_path = database->getDataPath(create);
/** 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.
*/
guard = context.getDDLGuard(database_name, table_name);
2019-11-11 11:34:03 +00:00
if (!create.attach && !data_path.empty() && Poco::File(context.getPath() + data_path).exists())
throw Exception("Directory for table data " + data_path + " already exists", ErrorCodes::TABLE_ALREADY_EXISTS);
/// 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))
{
/// TODO Check structure of table
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>();
drop_ast->database = database_name;
drop_ast->table = table_name;
drop_ast->no_ddl_lock = true;
InterpreterDropQuery interpreter(drop_ast, context);
interpreter.execute();
}
else
throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
}
}
else if (context.tryGetExternalTable(table_name) && create.if_not_exists)
return false;
2019-07-22 13:49:16 +00:00
StoragePtr res;
2019-07-18 18:29:49 +00:00
if (create.as_table_function)
{
2019-07-22 12:50:10 +00:00
const auto & table_function = create.as_table_function->as<ASTFunction &>();
2019-07-18 18:29:49 +00:00
const auto & factory = TableFunctionFactory::instance();
2019-07-22 12:50:10 +00:00
res = factory.get(table_function.name, context)->execute(create.as_table_function, context, create.table);
2019-07-18 18:29:49 +00:00
}
2019-07-22 12:18:53 +00:00
else
{
res = StorageFactory::instance().get(create,
2019-11-11 11:34:03 +00:00
data_path,
context,
context.getGlobalContext(),
properties.columns,
properties.constraints,
false);
2019-07-22 12:18:53 +00:00
}
if (need_add_to_database)
database->createTable(context, table_name, res, query_ptr);
else
context.getSessionContext().addExternalTable(table_name, res, query_ptr);
/// 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();
return true;
}
BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create, const String & database_name)
{
/// If the query is a CREATE SELECT, insert the data into the table.
if (create.select && !create.attach
&& !create.is_view && !create.is_live_view && (!create.is_materialized_view || create.is_populate))
{
auto insert = std::make_shared<ASTInsertQuery>();
if (!create.temporary)
insert->database = database_name;
insert->table = create.table;
insert->select = create.select->clone();
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
return InterpreterInsertQuery(insert,
create.temporary ? context.getSessionContext() : context,
context.getSettingsRef().insert_allow_materialized_columns).execute();
}
return {};
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
}
BlockIO InterpreterCreateQuery::createDictionary(ASTCreateQuery & create)
{
String dictionary_name = create.table;
String database_name = !create.database.empty() ? create.database : context.getCurrentDatabase();
auto guard = context.getDDLGuard(database_name, dictionary_name);
DatabasePtr database = context.getDatabase(database_name);
if (database->isDictionaryExist(context, dictionary_name))
{
2019-10-22 11:03:16 +00:00
/// TODO Check structure of dictionary
if (create.if_not_exists)
return {};
else
throw Exception(
"Dictionary " + database_name + "." + dictionary_name + " already exists.", ErrorCodes::DICTIONARY_ALREADY_EXISTS);
}
2019-10-15 18:04:17 +00:00
if (create.attach)
{
2019-12-05 13:44:22 +00:00
auto query = context.getDatabase(database_name)->getCreateDictionaryQuery(context, dictionary_name);
2019-10-15 18:04:17 +00:00
create = query->as<ASTCreateQuery &>();
create.attach = true;
}
if (create.attach)
2019-10-16 17:06:52 +00:00
database->attachDictionary(dictionary_name, context);
else
2019-10-16 14:59:52 +00:00
database->createDictionary(context, dictionary_name, query_ptr);
return {};
}
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
BlockIO InterpreterCreateQuery::execute()
{
auto & create = query_ptr->as<ASTCreateQuery &>();
2017-12-22 19:20:18 +00:00
checkAccess(create);
ASTQueryWithOutput::resetOutputASTIfExist(create);
/// CREATE|ATTACH DATABASE
if (!create.database.empty() && create.table.empty())
2017-04-25 15:21:03 +00:00
return createDatabase(create);
else if (!create.is_dictionary)
2017-04-21 12:39:28 +00:00
return createTable(create);
else
return createDictionary(create);
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +00:00
}
2017-12-22 19:20:18 +00:00
void InterpreterCreateQuery::checkAccess(const ASTCreateQuery & create)
{
/// Internal queries (initiated by the server itself) always have access to everything.
if (internal)
return;
const Settings & settings = context.getSettingsRef();
auto readonly = settings.readonly;
2018-09-11 18:37:19 +00:00
auto allow_ddl = settings.allow_ddl;
2018-09-11 18:37:19 +00:00
if (!readonly && allow_ddl)
return;
Squashed commit of the following: commit f9b478181cd49224154cc350fb57df7121842f1c Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 04:06:36 2016 +0300 Database engines: development [#METR-19997]. commit f7a10a67761ccfd05f3dac32d6444920cd8d4d60 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 03:44:37 2016 +0300 Database engines: development [#METR-19997]. commit bd98a8558e98bad2bed278e5762c4e0fc66e6f38 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:33:59 2016 +0300 Database engines: development [#METR-19997]. commit 19712fd884c22a4e2c2b67474086dea8f44e7c7b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Sat Mar 19 00:03:11 2016 +0300 Database engines: development [#METR-19997]. commit 50274d6df7e91fcc34aab8a8c72347daa2c6512f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 23:24:57 2016 +0300 Database engines: development [#METR-19997]. commit 4a0b99b19b34e90ef8b7be2d199f6232e36ef3f7 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 22:50:36 2016 +0300 Database engines: development [#METR-19997]. commit 44ff3ebba7a3e460a27a89f31ddf199dbea1d182 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 15:09:17 2016 +0300 Database engines: development [#METR-19997]. commit 137c31f3004cfd282473b6acb01cbe1b4ca2aadd Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:26:34 2016 +0300 Database engines: development [#METR-19997]. commit aa4c0496d4afe4a691164254be2bd5600542b38a Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 03:22:59 2016 +0300 Database engines: development [#METR-19997]. commit 5a94d1f0607450a2dac28a4d7df8b1393a864c23 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Fri Mar 18 01:02:40 2016 +0300 Database engines: development [#METR-19997]. commit 50fd5b52ea1141955a5dfba0dcb191f3289ac25b Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 23:23:40 2016 +0300 Database engines: development [#METR-19997]. commit a333d91b058e4f56dd83a6d2878c3c2bd8efc002 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 20:29:07 2016 +0300 Database engines: development [#METR-19997]. commit f81d366e7ac8348436f2698d040f8e341743a024 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Thu Mar 17 01:30:23 2016 +0300 Database engines: development [#METR-19997]. commit d0696860c9060827896214c08d147c759ea79376 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 21:55:31 2016 +0300 Database engines: development [#METR-19997]. commit 46a168c2ada140a0e95cd8d4b9d8ba9bac855d11 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 08:00:58 2016 +0300 Database engines: development [#METR-19997]. commit 20a2bad161454225fc1b5f9b919b842fbebc3231 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:51:10 2016 +0300 Database engines: development [#METR-19997]. commit ca0a77fcc2a8d0b276eb3743c53551ad3fe16314 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 06:02:20 2016 +0300 Reverted erroneous modification [#METR-19997]. commit 1370bdcc4594182f6ef2b146f9afabfe1c295080 Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Wed Mar 16 00:41:34 2016 +0300 Database engines: development [#METR-19997]. commit 16e72c67041cae6471509d3f0f3d4a9aa7b7dc0f Author: Alexey Milovidov <milovidov@yandex-team.ru> Date: Tue Mar 15 00:41:48 2016 +0300 Database engines: development [#METR-19997].
2016-03-19 01:18:49 +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);
}
String object = "table";
if (create.is_dictionary)
{
if (readonly)
throw Exception("Cannot create dictionary in readonly mode", ErrorCodes::READONLY);
object = "dictionary";
}
if (create.temporary && readonly >= 2)
return;
2018-09-11 18:37:19 +00:00
if (readonly)
2019-10-17 17:18:54 +00:00
throw Exception("Cannot create table or dictionary in readonly mode", ErrorCodes::READONLY);
2018-09-11 18:37:19 +00:00
throw Exception("Cannot create " + object + ". DDL queries are prohibited for the user", ErrorCodes::QUERY_IS_PROHIBITED);
}
2011-08-18 20:33:20 +00:00
}