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
2022-02-10 08:53:50 +00:00
# include "Common/Exception.h"
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>
2022-04-11 12:32:55 +00:00
# include <Common/atomicRename.h>
2023-11-06 13:08:27 +00:00
# include <Common/PoolId.h>
2023-08-14 17:06:32 +00:00
# include <Common/logger_useful.h>
2023-03-06 14:53:58 +00:00
# include <base/hex.h>
2017-04-01 09:19:00 +00:00
2020-04-15 20:28:05 +00:00
# include <Core/Defines.h>
2022-02-08 12:21:16 +00:00
# include <Core/SettingsEnums.h>
2023-09-13 19:34:18 +00:00
# include <Core/ServerSettings.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>
# include <Parsers/ASTColumnDeclaration.h>
2017-12-20 07:39:52 +00:00
# include <Parsers/ASTCreateQuery.h>
2017-04-01 09:19:00 +00:00
# include <Parsers/ASTIdentifier.h>
# include <Parsers/ASTLiteral.h>
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>
2022-05-05 15:31:15 +00:00
# include <Storages/WindowView/StorageWindowView.h>
2022-03-11 16:03:05 +00:00
# include <Storages/StorageReplicatedMergeTree.h>
2023-09-20 09:31:12 +00:00
# include <Storages/BlockNumberColumn.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>
2021-07-01 13:21:38 +00:00
# include <Interpreters/executeQuery.h>
2021-07-02 16:39:55 +00:00
# include <Interpreters/DDLTask.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>
2023-02-19 20:20:49 +00:00
# include <Interpreters/InterpreterSelectQueryAnalyzer.h>
2018-02-19 00:45:32 +00:00
# include <Interpreters/InterpreterInsertQuery.h>
2022-01-10 19:01:41 +00:00
# include <Interpreters/InterpreterRenameQuery.h>
2018-11-01 14:49:37 +00:00
# include <Interpreters/AddDefaultDatabaseVisitor.h>
2022-09-28 14:28:28 +00:00
# include <Interpreters/GinFilter.h>
2017-04-01 09:19:00 +00:00
2021-10-31 08:51:20 +00:00
# include <Access/Common/AccessRightsElement.h>
2020-01-26 09:49:53 +00:00
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>
2021-05-31 14:44:57 +00:00
# include <DataTypes/DataTypeAggregateFunction.h>
2022-03-16 16:51:23 +00:00
# include <DataTypes/ObjectUtils.h>
2022-04-09 07:00:41 +00:00
# include <DataTypes/hasNullable.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>
2020-07-06 08:30:11 +00:00
# include <Databases/DatabaseOnDisk.h>
2021-08-26 13:19:52 +00:00
# include <Databases/TablesLoader.h>
2021-11-01 18:53:07 +00:00
# include <Databases/DDLDependencyVisitor.h>
2022-12-02 14:05:46 +00:00
# include <Databases/NormalizeAndEvaluateConstantsVisitor.h>
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>
2020-12-14 03:30:39 +00:00
# include <Interpreters/QueryLog.h>
2019-06-16 16:47:47 +00:00
# include <Interpreters/addTypeConversionToAST.h>
2021-02-14 12:53:50 +00:00
# include <Interpreters/FunctionNameNormalizer.h>
2021-04-05 06:36:33 +00:00
# include <Interpreters/ApplyWithSubqueryVisitor.h>
2018-10-11 02:57:48 +00:00
2019-07-18 18:29:49 +00:00
# include <TableFunctions/TableFunctionFactory.h>
2022-09-19 07:00:04 +00:00
# include <DataTypes/DataTypeFixedString.h>
2019-07-18 18:29:49 +00:00
2022-11-23 09:28:08 +00:00
# include <Functions/UserDefined/UserDefinedSQLFunctionFactory.h>
# include <Functions/UserDefined/UserDefinedSQLFunctionVisitor.h>
2018-02-19 00:45:32 +00:00
2011-08-18 20:33:20 +00:00
namespace DB
{
2018-02-19 00:45:32 +00:00
2016-01-11 21:46:36 +00:00
namespace ErrorCodes
{
extern const int TABLE_ALREADY_EXISTS ;
2021-04-22 09:19:59 +00:00
extern const int DICTIONARY_ALREADY_EXISTS ;
2016-01-11 21:46:36 +00:00
extern const int EMPTY_LIST_OF_COLUMNS_PASSED ;
extern const int INCORRECT_QUERY ;
2016-03-19 01:18:49 +00:00
extern const int UNKNOWN_DATABASE_ENGINE ;
2016-11-21 12:58:42 +00:00
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 ;
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 ;
2023-10-10 13:44:15 +00:00
extern const int ILLEGAL_INDEX ;
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 ;
2022-02-10 20:35:03 +00:00
extern const int ENGINE_REQUIRED ;
2022-03-11 16:03:05 +00:00
extern const int UNKNOWN_STORAGE ;
2022-05-13 11:27:17 +00:00
extern const int SYNTAX_ERROR ;
2022-09-28 14:28:28 +00:00
extern const int SUPPORT_IS_DISABLED ;
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
2021-05-31 14:49:02 +00:00
InterpreterCreateQuery : : InterpreterCreateQuery ( const ASTPtr & query_ptr_ , ContextMutablePtr context_ )
: WithMutableContext ( context_ ) , query_ptr ( query_ptr_ )
2017-12-22 19:20:18 +00:00
{
}
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
{
2021-09-06 22:13:54 +00:00
String database_name = create . getDatabase ( ) ;
2017-04-01 07:20:54 +00:00
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
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : DATABASE_ALREADY_EXISTS , " Database {} already exists. " , database_name ) ;
2018-09-11 14:33:06 +00:00
}
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 ) ;
2021-04-10 23:33:54 +00:00
fs : : path metadata_path = fs : : canonical ( getContext ( ) - > getPath ( ) ) ;
2020-07-06 08:30:11 +00:00
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 ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : UNKNOWN_DATABASE_ENGINE , " Database engine must be specified for ATTACH DATABASE query " ) ;
2020-07-06 08:30:11 +00:00
/// Short syntax: try read database definition from file
2021-04-10 23:33:54 +00:00
auto ast = DatabaseOnDisk : : parseQueryFromMetadata ( nullptr , getContext ( ) , metadata_file_path ) ;
2020-07-06 08:30:11 +00:00
create = ast - > as < ASTCreateQuery & > ( ) ;
2021-11-11 13:28:18 +00:00
if ( create . table | | ! 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 ;
2021-09-06 22:13:54 +00:00
create . setDatabase ( database_name ) ;
2020-07-06 08:30:11 +00:00
}
else if ( ! create . storage )
2011-08-19 18:31:14 +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 )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : UNKNOWN_DATABASE_ENGINE , " Database engine must be specified for ATTACH DATABASE query " ) ;
2017-09-17 18:49:43 +00:00
auto engine = std : : make_shared < ASTFunction > ( ) ;
auto storage = std : : make_shared < ASTStorage > ( ) ;
2022-06-23 07:59:13 +00:00
engine - > name = " 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 ) ;
2016-03-19 01:18:49 +00:00
}
2021-02-10 14:12:49 +00:00
else if ( ( create . columns_list
& & ( ( create . columns_list - > indices & & ! create . columns_list - > indices - > children . empty ( ) )
| | ( create . columns_list - > projections & & ! create . columns_list - > projections - > children . empty ( ) ) ) ) )
2016-03-19 01:18:49 +00:00
{
2017-01-23 16:00:26 +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 ) ) ;
2011-11-05 23:31:19 +00:00
}
2017-04-01 07:20:54 +00:00
2022-02-10 20:35:03 +00:00
if ( create . storage & & ! create . storage - > engine )
throw Exception ( ErrorCodes : : INCORRECT_QUERY , " Database engine must be specified " ) ;
2021-09-24 23:10:10 +00:00
if ( create . storage - > engine - > name = = " Atomic "
| | create . storage - > engine - > name = = " Replicated "
| | create . storage - > engine - > name = = " MaterializedPostgreSQL " )
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. "
2021-09-06 22:13:54 +00:00
" If you want to attach existing database, use just ATTACH DATABASE {}; " , create . getDatabase ( ) ) ;
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
}
2021-09-24 23:10:10 +00:00
else if ( create . storage - > engine - > name = = " MaterializeMySQL "
| | create . storage - > engine - > name = = " MaterializedMySQL " )
2020-09-14 19:25:02 +00:00
{
/// 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 )
{
2022-06-23 07:59:13 +00:00
if ( create . uuid = = UUIDHelpers : : Nil )
2020-09-14 19:25:02 +00:00
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.
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INCORRECT_QUERY ,
" Use short attach syntax ('ATTACH DATABASE name;' without engine) "
" to attach existing database or specify UUID to attach new database with Atomic engine " ) ;
2020-09-14 19:25:02 +00:00
}
/// 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
{
2021-04-10 23:33:54 +00:00
bool is_on_cluster = getContext ( ) - > getClientInfo ( ) . query_kind = = ClientInfo : : QueryKind : : SECONDARY_QUERY ;
2023-08-23 20:55:46 +00:00
if ( create . uuid ! = UUIDHelpers : : Nil & & ! is_on_cluster & & ! internal )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INCORRECT_QUERY , " Ordinary database engine does not support UUID " ) ;
2017-04-01 07:20:54 +00:00
2023-08-23 20:55:46 +00:00
/// The database doesn't support UUID so we'll ignore it. The UUID could be set here because of either
/// a) the initiator of `ON CLUSTER` query generated it to ensure the same UUIDs are used on different hosts; or
/// b) `RESTORE from backup` query generated it to ensure the same UUIDs are used on different hosts.
2020-07-10 09:19:32 +00:00
create . uuid = UUIDHelpers : : Nil ;
2020-07-02 20:39:31 +00:00
metadata_path = metadata_path / " metadata " / database_name_escaped ;
}
2023-05-21 21:07:03 +00:00
if ( create . storage - > engine - > name = = " Replicated " & & ! internal & & ! create . attach & & create . storage - > engine - > arguments )
2023-04-22 01:52:13 +00:00
{
/// Fill in default parameters
if ( create . storage - > engine - > arguments - > children . size ( ) = = 1 )
create . storage - > engine - > arguments - > children . push_back ( std : : make_shared < ASTLiteral > ( " {shard} " ) ) ;
if ( create . storage - > engine - > arguments - > children . size ( ) = = 2 )
create . storage - > engine - > arguments - > children . push_back ( std : : make_shared < ASTLiteral > ( " {replica} " ) ) ;
}
2021-07-26 18:24:32 +00:00
if ( ( create . storage - > engine - > name = = " MaterializeMySQL " | | create . storage - > engine - > name = = " MaterializedMySQL " )
& & ! getContext ( ) - > getSettingsRef ( ) . allow_experimental_database_materialized_mysql
2022-12-29 14:34:11 +00:00
& & ! internal & & ! create . attach )
2020-08-11 21:59:46 +00:00
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : UNKNOWN_DATABASE_ENGINE ,
" MaterializedMySQL is an experimental database engine. "
2023-04-19 20:03:02 +00:00
" Enable allow_experimental_database_materialized_mysql to use it " ) ;
2020-08-11 21:59:46 +00:00
}
2021-09-24 23:12:26 +00:00
if ( create . storage - > engine - > name = = " Replicated "
& & ! getContext ( ) - > getSettingsRef ( ) . allow_experimental_database_replicated
2022-12-29 14:34:11 +00:00
& & ! internal & & ! create . attach )
2021-02-15 10:26:34 +00:00
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : UNKNOWN_DATABASE_ENGINE ,
" Replicated is an experimental database engine. "
2023-04-19 20:03:02 +00:00
" Enable allow_experimental_database_replicated to use it " ) ;
2021-02-15 10:26:34 +00:00
}
2021-09-24 23:12:26 +00:00
if ( create . storage - > engine - > name = = " MaterializedPostgreSQL "
& & ! getContext ( ) - > getSettingsRef ( ) . allow_experimental_database_materialized_postgresql
2022-12-29 14:34:11 +00:00
& & ! internal & & ! create . attach )
2021-02-22 12:35:53 +00:00
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : UNKNOWN_DATABASE_ENGINE ,
" MaterializedPostgreSQL is an experimental database engine. "
2023-04-19 20:03:02 +00:00
" Enable allow_experimental_database_materialized_postgresql to use it " ) ;
2021-02-22 12:35:53 +00:00
}
2022-06-22 16:31:42 +00:00
bool need_write_metadata = ! create . attach | | ! fs : : exists ( metadata_file_path ) ;
bool need_lock_uuid = internal | | need_write_metadata ;
2022-07-20 20:54:43 +00:00
auto mode = getLoadingStrictnessLevel ( create . attach , force_attach , has_force_restore_data_flag ) ;
2022-06-22 16:31:42 +00:00
/// Lock uuid, so we will known it's already in use.
/// We do it when attaching databases on server startup (internal) and on CREATE query (!create.attach);
TemporaryLockForUUIDDirectory uuid_lock ;
if ( need_lock_uuid )
uuid_lock = TemporaryLockForUUIDDirectory { create . uuid } ;
else if ( create . uuid ! = UUIDHelpers : : Nil & & ! DatabaseCatalog : : instance ( ) . hasUUIDMapping ( create . uuid ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Cannot find UUID mapping for {}, it's a bug " , create . uuid ) ;
2021-04-10 23:33:54 +00:00
DatabasePtr database = DatabaseFactory : : get ( create , metadata_path / " " , getContext ( ) ) ;
2020-07-17 13:11:44 +00:00
if ( create . uuid ! = UUIDHelpers : : Nil )
2021-09-06 22:13:54 +00:00
create . setDatabase ( TABLE_WITH_UUID_NAME_PLACEHOLDER ) ;
2017-04-01 07:20:54 +00:00
2016-03-19 01:18:49 +00:00
if ( need_write_metadata )
2016-03-03 04:12:47 +00:00
{
2016-03-19 01:18:49 +00:00
create . attach = true ;
create . if_not_exists = false ;
2017-04-01 07:20:54 +00:00
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
2017-01-23 16:00:26 +00:00
/// Exclusive flag guarantees, that database is not created right now in another thread.
2016-03-19 01:18:49 +00:00
WriteBufferFromFile out ( metadata_file_tmp_path , statement . size ( ) , O_WRONLY | O_CREAT | O_EXCL ) ;
writeString ( statement , out ) ;
2017-04-01 07:20:54 +00:00
2016-03-19 01:18:49 +00:00
out . next ( ) ;
2021-04-10 23:33:54 +00:00
if ( getContext ( ) - > getSettingsRef ( ) . fsync_metadata )
2017-01-23 18:05:07 +00:00
out . sync ( ) ;
2016-03-19 01:18:49 +00:00
out . close ( ) ;
2016-03-03 04:12:47 +00:00
}
2017-04-01 07:20:54 +00:00
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 ;
2016-03-19 01:18:49 +00:00
try
2011-11-05 23:31:19 +00:00
{
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
2016-03-19 01:18:49 +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
2021-08-26 13:19:52 +00:00
if ( ! load_database_without_tables )
{
/// We use global context here, because storages lifetime is bigger than query context lifetime
2023-02-19 22:15:09 +00:00
TablesLoader loader { getContext ( ) - > getGlobalContext ( ) , { { database_name , database } } , mode } ;
2023-09-13 19:34:18 +00:00
auto load_tasks = loader . loadTablesAsync ( ) ;
auto startup_tasks = loader . startupTablesAsync ( ) ;
if ( getContext ( ) - > getGlobalContext ( ) - > getServerSettings ( ) . async_load_databases )
{
scheduleLoad ( load_tasks ) ;
scheduleLoad ( startup_tasks ) ;
}
else
{
/// First prioritize, schedule and wait all the load table tasks
2023-11-06 13:08:27 +00:00
waitLoad ( currentPoolOr ( TablesLoaderForegroundPoolId ) , load_tasks ) ;
2023-09-13 19:34:18 +00:00
/// Only then prioritize, schedule and wait all the startup tasks
2023-11-06 13:08:27 +00:00
waitLoad ( currentPoolOr ( TablesLoaderForegroundPoolId ) , startup_tasks ) ;
2023-09-13 19:34:18 +00:00
}
2021-08-26 13:19:52 +00:00
}
2012-08-02 17:33:31 +00:00
}
2016-03-19 01:18:49 +00:00
catch ( . . . )
2011-11-06 06:22:52 +00:00
{
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 )
2021-10-13 10:34:18 +00:00
DatabaseCatalog : : instance ( ) . detachDatabase ( getContext ( ) , database_name , false , false ) ;
2017-04-01 07:20:54 +00:00
2016-03-19 01:18:49 +00:00
throw ;
2011-11-06 06:22:52 +00:00
}
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
{
2016-05-28 16:30:15 +00:00
auto columns_list = std : : make_shared < ASTExpressionList > ( ) ;
2014-07-10 11:13:45 +00:00
2014-09-25 13:40:26 +00:00
for ( const auto & column : columns )
2014-07-10 11:13:45 +00:00
{
2016-05-28 15:42:22 +00:00
const auto column_declaration = std : : make_shared < ASTColumnDeclaration > ( ) ;
2014-09-25 13:40:26 +00:00
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 ) ;
2016-05-28 16:19:27 +00:00
columns_list - > children . emplace_back ( column_declaration ) ;
2014-07-10 11:13:45 +00:00
}
2016-05-28 15:42:22 +00:00
return columns_list ;
2014-07-10 11:13:45 +00:00
}
2021-06-28 11:42:21 +00:00
ASTPtr InterpreterCreateQuery : : formatColumns ( const NamesAndTypesList & columns , const NamesAndAliases & alias_columns )
{
std : : shared_ptr < ASTExpressionList > columns_list = std : : static_pointer_cast < ASTExpressionList > ( formatColumns ( columns ) ) ;
for ( const auto & alias_column : alias_columns )
{
const auto column_declaration = std : : make_shared < ASTColumnDeclaration > ( ) ;
column_declaration - > name = alias_column . name ;
ParserDataType type_parser ;
String type_name = alias_column . type - > getName ( ) ;
const char * type_pos = type_name . data ( ) ;
const char * type_end = type_pos + type_name . size ( ) ;
column_declaration - > type = parseQuery ( type_parser , type_pos , type_end , " data type " , 0 , DBMS_DEFAULT_MAX_PARSER_DEPTH ) ;
column_declaration - > default_specifier = " ALIAS " ;
const auto & alias = alias_column . expression ;
const char * alias_pos = alias . data ( ) ;
const char * alias_end = alias_pos + alias . size ( ) ;
ParserExpression expression_parser ;
column_declaration - > default_expression = parseQuery ( expression_parser , alias_pos , alias_end , " expression " , 0 , DBMS_DEFAULT_MAX_PARSER_DEPTH ) ;
2022-11-24 10:12:06 +00:00
column_declaration - > children . push_back ( column_declaration - > default_expression ) ;
2021-06-28 11:42:21 +00:00
columns_list - > children . emplace_back ( column_declaration ) ;
}
return columns_list ;
}
2018-03-06 20:18:34 +00:00
ASTPtr InterpreterCreateQuery : : formatColumns ( const ColumnsDescription & columns )
2014-09-25 15:01:09 +00:00
{
2016-05-28 15:42:22 +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 )
2014-09-30 03:08:47 +00:00
{
2016-05-28 15:42:22 +00:00
const auto column_declaration = std : : make_shared < ASTColumnDeclaration > ( ) ;
2014-09-30 03:08:47 +00:00
ASTPtr column_declaration_ptr { column_declaration } ;
column_declaration - > name = column . name ;
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 )
2014-09-30 03:08:47 +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 ( ) ;
2021-11-01 18:53:07 +00:00
column_declaration - > children . push_back ( column_declaration - > default_expression ) ;
2018-10-14 15:30:06 +00:00
}
2022-12-11 00:40:35 +00:00
column_declaration - > ephemeral_default = column . default_desc . ephemeral_default ;
2019-03-14 15:20:51 +00:00
if ( ! column . comment . empty ( ) )
2014-09-30 03:08:47 +00:00
{
2019-03-14 15:20:51 +00:00
column_declaration - > comment = std : : make_shared < ASTLiteral > ( Field ( column . comment ) ) ;
2021-11-01 18:53:07 +00:00
column_declaration - > children . push_back ( column_declaration - > comment ) ;
2014-09-30 03:08:47 +00:00
}
2014-09-25 15:01:09 +00:00
2019-03-14 15:20:51 +00:00
if ( column . codec )
2021-11-01 18:53:07 +00:00
{
2020-08-28 17:40:45 +00:00
column_declaration - > codec = column . codec ;
2021-11-01 18:53:07 +00:00
column_declaration - > children . push_back ( column_declaration - > codec ) ;
}
2018-10-11 02:57:48 +00:00
2023-09-26 17:16:01 +00:00
if ( column . stat )
{
column_declaration - > stat_type = column . stat - > ast ;
column_declaration - > children . push_back ( column_declaration - > stat_type ) ;
}
2019-04-15 09:30:45 +00:00
if ( column . ttl )
2021-11-01 18:53:07 +00:00
{
2019-04-15 09:30:45 +00:00
column_declaration - > ttl = column . ttl ;
2021-11-01 18:53:07 +00:00
column_declaration - > children . push_back ( column_declaration - > ttl ) ;
}
2019-04-15 09:30:45 +00:00
2016-05-28 15:42:22 +00:00
columns_list - > children . push_back ( column_declaration_ptr ) ;
2014-07-10 11:13:45 +00:00
}
2016-05-28 15:42:22 +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 > ( ) ;
2021-04-26 14:19:18 +00:00
for ( const auto & constraint : constraints . getConstraints ( ) )
2019-05-17 04:08:03 +00:00
res - > children . push_back ( constraint - > clone ( ) ) ;
return res ;
}
2021-02-10 14:12:49 +00:00
ASTPtr InterpreterCreateQuery : : formatProjections ( const ProjectionsDescription & projections )
{
auto res = std : : make_shared < ASTExpressionList > ( ) ;
for ( const auto & projection : projections )
res - > children . push_back ( projection . definition_ast - > clone ( ) ) ;
return res ;
}
2020-05-04 00:11:49 +00:00
ColumnsDescription InterpreterCreateQuery : : getColumnsDescription (
2023-11-03 15:46:07 +00:00
const ASTExpressionList & columns_ast , ContextPtr context_ , bool attach , bool is_restore_from_backup )
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 ;
2023-11-09 16:21:12 +00:00
bool make_columns_nullable = ! attach & & ! is_restore_from_backup & & context_ - > getSettingsRef ( ) . data_type_default_nullable ;
2019-03-14 15:20:51 +00:00
for ( const auto & ast : columns_ast . children )
{
2022-11-23 10:27:38 +00:00
const auto & col_decl = ast - > as < ASTColumnDeclaration & > ( ) ;
2019-03-14 15:20:51 +00:00
2022-04-14 16:12:31 +00:00
if ( col_decl . collation & & ! context_ - > getSettingsRef ( ) . compatibility_ignore_collation_in_create_table )
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Cannot support collation, please set compatibility_ignore_collation_in_create_table=true " ) ;
2022-04-14 16:12:31 +00:00
}
2020-05-20 01:22:32 +00:00
2022-04-14 16:12:31 +00:00
DataTypePtr column_type = nullptr ;
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
2022-10-14 18:07:02 +00:00
if ( attach )
setVersionToAggregateFunctions ( column_type , true ) ;
2021-05-31 14:44:57 +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 ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : ILLEGAL_SYNTAX_FOR_DATA_TYPE , " Can't use [NOT] NULL modifier with Nullable 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 ) ;
}
2021-04-16 10:15:35 +00:00
else if ( make_columns_nullable )
2020-06-14 16:17:22 +00:00
{
2020-05-20 01:22:32 +00:00
column_type = makeNullable ( column_type ) ;
2020-06-14 16:17:22 +00:00
}
2022-04-09 07:00:41 +00:00
else if ( ! hasNullable ( column_type ) & &
col_decl . default_specifier = = " DEFAULT " & &
col_decl . default_expression & &
col_decl . default_expression - > as < ASTLiteral > ( ) & &
col_decl . default_expression - > as < ASTLiteral > ( ) - > value . isNull ( ) )
{
if ( column_type - > lowCardinality ( ) )
{
const auto * low_cardinality_type = typeid_cast < const DataTypeLowCardinality * > ( column_type . get ( ) ) ;
assert ( low_cardinality_type ) ;
column_type = std : : make_shared < DataTypeLowCardinality > ( makeNullable ( low_cardinality_type - > getDictionaryType ( ) ) ) ;
}
else
column_type = makeNullable ( column_type ) ;
}
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 (
2022-12-11 03:29:45 +00:00
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 ( ) )
2021-04-10 23:33:54 +00:00
defaults_sample_block = validateColumnsDefaultsAndGetSampleBlock ( default_expr_list , column_names_and_types , context_ ) ;
2019-03-14 15:20:51 +00:00
2021-04-16 10:15:35 +00:00
bool sanity_check_compression_codecs = ! attach & & ! context_ - > getSettingsRef ( ) . allow_suspicious_codecs ;
2021-05-24 03:47:00 +00:00
bool allow_experimental_codecs = attach | | context_ - > getSettingsRef ( ) . allow_experimental_codecs ;
2023-06-09 12:38:38 +00:00
bool enable_deflate_qpl_codec = attach | | context_ - > getSettingsRef ( ) . enable_deflate_qpl_codec ;
2021-05-24 03:47:00 +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 ( ) ;
2022-10-18 09:40:12 +00:00
for ( const auto * ast_it = columns_ast . children . begin ( ) ; ast_it ! = columns_ast . children . end ( ) ; + + ast_it , + + name_type_it )
2019-03-14 15:20:51 +00:00
{
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
2022-05-13 08:28:24 +00:00
/// ignore or not other database extensions depending on compatibility settings
if ( col_decl . default_specifier = = " AUTO_INCREMENT "
& & ! context_ - > getSettingsRef ( ) . compatibility_ignore_auto_increment_in_create_table )
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : SYNTAX_ERROR ,
" AUTO_INCREMENT is not supported. To ignore the keyword "
" in column declaration, set `compatibility_ignore_auto_increment_in_create_table` to true " ) ;
2022-05-13 08:28:24 +00:00
}
2019-03-14 15:20:51 +00:00
if ( col_decl . default_expression )
{
2022-10-05 18:58:27 +00:00
if ( context_ - > hasQueryContext ( ) & & context_ - > getQueryContext ( ) . get ( ) = = context_ . get ( ) )
{
/// Normalize query only for original CREATE query, not on metadata loading.
/// And for CREATE query we can pass local context, because result will not change after restart.
NormalizeAndEvaluateConstantsVisitor : : Data visitor_data { context_ } ;
NormalizeAndEvaluateConstantsVisitor visitor ( visitor_data ) ;
visitor . visit ( col_decl . default_expression ) ;
}
2022-12-11 00:40:35 +00:00
ASTPtr default_expr = col_decl . default_expression - > clone ( ) ;
2022-03-31 03:36:49 +00:00
2019-03-14 15:20:51 +00:00
if ( col_decl . type )
column . type = name_type_it - > type ;
else
2022-05-16 22:09:23 +00:00
{
2019-03-14 15:20:51 +00:00
column . type = defaults_sample_block . getByName ( column . name ) . type ;
2022-05-16 22:09:23 +00:00
/// set nullability for case of column declaration w/o type but with default expression
if ( ( col_decl . null_modifier & & * col_decl . null_modifier ) | | make_columns_nullable )
column . type = makeNullable ( column . type ) ;
}
2019-03-14 15:20:51 +00:00
column . default_desc . kind = columnDefaultKindFromString ( col_decl . default_specifier ) ;
column . default_desc . expression = default_expr ;
2022-12-11 00:40:35 +00:00
column . default_desc . ephemeral_default = col_decl . ephemeral_default ;
2019-03-14 15:20:51 +00:00
}
else if ( col_decl . type )
column . type = name_type_it - > type ;
else
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Neither default value expression nor type is provided for a column " ) ;
2019-03-14 15:20:51 +00:00
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 " )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : BAD_ARGUMENTS , " Cannot specify codec for column type ALIAS " ) ;
2020-08-31 07:39:58 +00:00
column . codec = CompressionCodecFactory : : instance ( ) . validateCodecAndGetPreprocessedAST (
2023-06-09 12:38:38 +00:00
col_decl . codec , column . type , sanity_check_compression_codecs , allow_experimental_codecs , enable_deflate_qpl_codec ) ;
2020-08-31 06:24:19 +00:00
}
2019-03-14 15:20:51 +00:00
2023-09-26 17:16:01 +00:00
if ( col_decl . stat_type )
{
2023-10-17 14:25:01 +00:00
if ( ! attach & & ! context_ - > getSettingsRef ( ) . allow_experimental_statistic )
2023-09-26 17:16:01 +00:00
throw Exception ( ErrorCodes : : INCORRECT_QUERY , " Create table with statistic is now disabled. Turn on allow_experimental_statistic " ) ;
column . stat = StatisticDescription : : getStatisticFromColumnDeclaration ( col_decl ) ;
}
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 ) ) ;
}
2023-11-03 15:46:07 +00:00
if ( ! attach & & ! is_restore_from_backup & & context_ - > getSettingsRef ( ) . flatten_nested )
2020-11-10 17:32:00 +00:00
res . flattenNested ( ) ;
2019-03-14 15:20:51 +00:00
if ( res . getAllPhysical ( ) . empty ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : EMPTY_LIST_OF_COLUMNS_PASSED , " Cannot CREATE table without physical columns " ) ;
2016-03-19 01:18:49 +00:00
return res ;
}
2019-08-24 21:20:20 +00:00
ConstraintsDescription InterpreterCreateQuery : : getConstraintsDescription ( const ASTExpressionList * constraints )
{
2021-11-10 17:57:59 +00:00
ASTs constraints_data ;
2019-08-24 21:20:20 +00:00
if ( constraints )
for ( const auto & constraint : constraints - > children )
2021-11-10 17:57:59 +00:00
constraints_data . push_back ( constraint - > clone ( ) ) ;
return ConstraintsDescription { constraints_data } ;
2019-08-24 21:20:20 +00:00
}
2021-08-26 11:01:15 +00:00
InterpreterCreateQuery : : TableProperties InterpreterCreateQuery : : getTablePropertiesAndNormalizeCreateQuery ( ASTCreateQuery & create ) const
2016-03-19 01:18:49 +00:00
{
2022-02-10 20:35:03 +00:00
/// Set the table engine if it was not specified explicitly.
setEngine ( create ) ;
/// We have to check access rights again (in case engine was changed).
if ( create . storage )
{
auto source_access_type = StorageFactory : : instance ( ) . getSourceAccessType ( create . storage - > engine - > name ) ;
if ( source_access_type ! = AccessType : : NONE )
getContext ( ) - > checkAccess ( source_access_type ) ;
}
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 )
2016-03-19 01:18:49 +00:00
{
2020-10-14 12:19:29 +00:00
if ( create . as_table_function & & ( create . columns_list - > indices | | create . columns_list - > constraints ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INCORRECT_QUERY , " Indexes and constraints are not supported for table functions " ) ;
2020-10-14 12:19:29 +00:00
2022-10-05 18:58:27 +00:00
/// Dictionaries have dictionary_attributes_list instead of columns_list
assert ( ! create . is_dictionary ) ;
2019-02-05 13:52:45 +00:00
if ( create . columns_list - > columns )
2020-05-04 00:11:49 +00:00
{
2023-11-03 15:46:07 +00:00
properties . columns = getColumnsDescription ( * create . columns_list - > columns , getContext ( ) , create . attach , is_restore_from_backup ) ;
2020-05-04 00:11:49 +00:00
}
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 )
2022-09-05 12:41:46 +00:00
{
2022-09-28 14:28:28 +00:00
IndexDescription index_desc = IndexDescription : : getIndexFromAST ( index - > clone ( ) , properties . columns , getContext ( ) ) ;
2023-10-10 13:44:15 +00:00
if ( properties . indices . has ( index_desc . name ) )
2023-10-10 13:46:52 +00:00
throw Exception ( ErrorCodes : : ILLEGAL_INDEX , " Duplicated index name {} is not allowed. Please use different index names. " , backQuoteIfNeed ( index_desc . name ) ) ;
2023-01-20 09:32:36 +00:00
const auto & settings = getContext ( ) - > getSettingsRef ( ) ;
if ( index_desc . type = = INVERTED_INDEX_NAME & & ! settings . allow_experimental_inverted_index )
2022-09-28 14:28:28 +00:00
{
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : SUPPORT_IS_DISABLED ,
" Experimental Inverted Index feature is not enabled (the setting 'allow_experimental_inverted_index') " ) ;
2022-09-28 14:28:28 +00:00
}
2023-01-20 09:32:36 +00:00
if ( index_desc . type = = " annoy " & & ! settings . allow_experimental_annoy_index )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INCORRECT_QUERY , " Annoy index is disabled. Turn on allow_experimental_annoy_index " ) ;
2019-08-24 21:20:20 +00:00
2023-08-16 13:29:50 +00:00
if ( index_desc . type = = " usearch " & & ! settings . allow_experimental_usearch_index )
throw Exception ( ErrorCodes : : INCORRECT_QUERY , " USearch index is disabled. Turn on allow_experimental_usearch_index " ) ;
2022-09-28 14:28:28 +00:00
properties . indices . push_back ( index_desc ) ;
}
2023-10-10 13:44:15 +00:00
2021-02-10 14:12:49 +00:00
if ( create . columns_list - > projections )
for ( const auto & projection_ast : create . columns_list - > projections - > children )
{
auto projection = ProjectionDescription : : getProjectionFromAST ( projection_ast , properties . columns , getContext ( ) ) ;
properties . projections . add ( std : : move ( projection ) ) ;
}
2019-10-23 18:39:07 +00:00
properties . constraints = getConstraintsDescription ( create . columns_list - > constraints ) ;
2016-03-19 01:18:49 +00:00
}
else if ( ! create . as_table . empty ( ) )
{
2021-04-10 23:33:54 +00:00
String as_database_name = getContext ( ) - > resolveDatabase ( create . as_database ) ;
StoragePtr as_storage = DatabaseCatalog : : instance ( ) . getTable ( { as_database_name , create . as_table } , getContext ( ) ) ;
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.
2021-04-10 23:33:54 +00:00
as_storage_lock = as_storage - > lockForShare ( getContext ( ) - > getCurrentQueryId ( ) , getContext ( ) - > 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
2021-08-26 11:01:15 +00:00
/// Secondary indices and projections make sense only for MergeTree family of storage engines.
2019-09-20 20:53:14 +00:00
/// 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 " ) )
2021-08-26 11:01:15 +00:00
{
2020-06-16 16:55:04 +00:00
properties . indices = as_storage_metadata - > getSecondaryIndices ( ) ;
2021-08-26 11:01:15 +00:00
properties . projections = as_storage_metadata - > getProjections ( ) . clone ( ) ;
}
2022-02-28 06:12:17 +00:00
else
{
/// Only MergeTree support TTL
properties . columns . resetColumnTTLs ( ) ;
}
2019-09-20 20:53:14 +00:00
2020-06-16 16:55:04 +00:00
properties . constraints = as_storage_metadata - > getConstraints ( ) ;
2016-03-19 01:18:49 +00:00
}
else if ( create . select )
{
2023-02-19 20:20:49 +00:00
Block as_select_sample ;
if ( getContext ( ) - > getSettingsRef ( ) . allow_experimental_analyzer )
{
as_select_sample = InterpreterSelectQueryAnalyzer : : getSampleBlock ( create . select - > clone ( ) , getContext ( ) ) ;
}
else
{
2023-12-14 09:45:01 +00:00
/** To get valid sample block we need to prepare query without only_analyze, because we need to execute scalar
* subqueries . Otherwise functions that expect only constant arguments will throw error during query analysis ,
* because the result of scalar subquery is not a constant .
*
* Example :
* CREATE MATERIALIZED VIEW test_mv ENGINE = MergeTree ORDER BY arr
* AS
* WITH ( SELECT ' \ d [ a - z ] ' ) AS constant_value
* SELECT extractAll ( concat ( toString ( number ) , ' a ' ) , assumeNotNull ( constant_value ) ) AS arr
* FROM test_table ;
*
* For new analyzer this issue does not exists because we always execute scalar subqueries .
* We can improve this in new analyzer , and execute scalar subqueries only in contexts when we expect constant
* for example : LIMIT , OFFSET , functions parameters , functions constant only arguments .
*/
2023-12-15 15:54:15 +00:00
SelectQueryOptions options ;
if ( create . isParameterizedView ( ) )
options = options . createParameterizedView ( ) ;
InterpreterSelectWithUnionQuery interpreter ( create . select - > clone ( ) , getContext ( ) , options ) ;
2023-12-14 09:45:01 +00:00
as_select_sample = interpreter . getSampleBlock ( ) ;
2023-02-19 20:20:49 +00:00
}
2019-10-23 18:39:07 +00:00
properties . columns = ColumnsDescription ( as_select_sample . getNamesAndTypesList ( ) ) ;
2016-03-19 01:18:49 +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.
2022-10-19 11:36:50 +00:00
auto table_function_ast = create . as_table_function - > ptr ( ) ;
auto table_function = TableFunctionFactory : : instance ( ) . get ( table_function_ast , getContext ( ) ) ;
2023-07-06 08:56:07 +00:00
properties . columns = table_function - > getActualTableStructure ( getContext ( ) , /*is_insert_query*/ true ) ;
2020-10-14 12:19:29 +00:00
}
2021-04-21 13:45:13 +00:00
else if ( create . is_dictionary )
{
2022-10-05 18:58:27 +00:00
if ( ! create . dictionary | | ! create . dictionary - > source )
return { } ;
/// Evaluate expressions (like currentDatabase() or tcpPort()) in dictionary source definition.
NormalizeAndEvaluateConstantsVisitor : : Data visitor_data { getContext ( ) } ;
NormalizeAndEvaluateConstantsVisitor visitor ( visitor_data ) ;
visitor . visit ( create . dictionary - > source - > ptr ( ) ) ;
2021-04-21 13:45:13 +00:00
return { } ;
}
2022-10-11 12:04:47 +00:00
else if ( ! create . storage | | ! create . storage - > engine )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Unexpected application state. CREATE query is missing either its storage or engine. " ) ;
2021-12-15 11:30:57 +00:00
/// We can have queries like "CREATE TABLE <table> ENGINE=<engine>" if <engine>
/// supports schema inference (will determine table structure in it's constructor).
2022-10-11 12:04:47 +00:00
else if ( ! StorageFactory : : instance ( ) . checkIfStorageSupportsSchemaInterface ( create . storage - > engine - > name ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INCORRECT_QUERY , " Incorrect CREATE query: required list of column descriptions or AS section or SELECT. " ) ;
2017-04-01 07:20:54 +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 ) ;
2021-02-10 14:12:49 +00:00
ASTPtr new_projections = formatProjections ( properties . projections ) ;
2019-10-23 18:39:07 +00:00
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 ) ;
2021-02-10 14:12:49 +00:00
create . columns_list - > setOrReplace ( create . columns_list - > projections , new_projections ) ;
2017-04-01 07:20:54 +00:00
2019-10-23 18:39:07 +00:00
validateTableStructure ( create , properties ) ;
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
{
2016-11-21 12:58:42 +00:00
/// Check for duplicates
std : : set < String > all_columns ;
2019-10-23 18:39:07 +00:00
for ( const auto & column : properties . columns )
2016-11-21 12:58:42 +00:00
{
2019-03-14 15:20:51 +00:00
if ( ! all_columns . emplace ( column . name ) . second )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : DUPLICATE_COLUMN , " Column {} already exists " , backQuoteIfNeed ( column . name ) ) ;
2019-03-14 15:20:51 +00:00
}
2017-04-01 07:20:54 +00:00
2022-09-26 07:14:58 +00:00
/// Check if _row_exists for lightweight delete column in column_lists for merge tree family.
if ( create . storage & & create . storage - > engine & & endsWith ( create . storage - > engine - > name , " MergeTree " ) )
{
auto search = all_columns . find ( LightweightDeleteDescription : : FILTER_COLUMN . name ) ;
if ( search ! = all_columns . end ( ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : ILLEGAL_COLUMN ,
" Cannot create table with column '{}' for *MergeTree engines because it "
" is reserved for lightweight delete feature " ,
LightweightDeleteDescription : : FILTER_COLUMN . name ) ;
2023-09-20 09:31:12 +00:00
auto search_block_number = all_columns . find ( BlockNumberColumn : : name ) ;
if ( search_block_number ! = all_columns . end ( ) )
throw Exception ( ErrorCodes : : ILLEGAL_COLUMN ,
" Cannot create table with column '{}' for *MergeTree engines because it "
" is reserved for storing block number " ,
BlockNumberColumn : : name ) ;
2022-09-26 07:14:58 +00:00
}
2021-04-10 23:33:54 +00:00
const auto & settings = getContext ( ) - > getSettingsRef ( ) ;
2020-08-19 11:52:17 +00:00
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 ( ) ) ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY ,
" Creating columns of type {} is prohibited by default "
2019-10-23 18:39:07 +00:00
" due to expected negative impact on performance. "
" It can be enabled with the \" allow_suspicious_low_cardinality_types \" setting. " ,
2023-01-23 21:13:58 +00:00
current_type_ptr - > getName ( ) ) ;
2019-10-23 18:39:07 +00:00
}
}
}
2020-05-30 15:26:19 +00:00
2022-03-16 16:51:23 +00:00
if ( ! create . attach & & ! settings . allow_experimental_object_type )
{
for ( const auto & [ name , type ] : properties . columns . getAllPhysical ( ) )
{
2022-05-06 14:44:00 +00:00
if ( type - > hasDynamicSubcolumns ( ) )
2022-03-16 16:51:23 +00:00
{
throw Exception ( ErrorCodes : : ILLEGAL_COLUMN ,
" Cannot create table with column '{}' which type is '{}' "
" because experimental Object type is not allowed. "
" Set setting allow_experimental_object_type = 1 in order to allow it " ,
name , type - > getName ( ) ) ;
}
}
}
2022-09-19 07:00:04 +00:00
if ( ! create . attach & & ! settings . allow_suspicious_fixed_string_types )
{
for ( const auto & [ name , type ] : properties . columns . getAllPhysical ( ) )
{
2023-10-10 17:51:09 +00:00
auto basic_type = removeLowCardinalityAndNullable ( type ) ;
2022-09-19 07:00:04 +00:00
if ( const auto * fixed_string = typeid_cast < const DataTypeFixedString * > ( basic_type . get ( ) ) )
{
if ( fixed_string - > getN ( ) > MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS )
throw Exception ( ErrorCodes : : ILLEGAL_COLUMN ,
" Cannot create table with column '{}' which type is '{}' "
" because fixed string with size > {} is suspicious. "
" Set setting allow_suspicious_fixed_string_types = 1 in order to allow it " ,
name , type - > getName ( ) , MAX_FIXEDSTRING_SIZE_WITHOUT_SUSPICIOUS ) ;
}
}
}
2016-03-19 01:18:49 +00:00
}
2023-07-03 15:28:56 +00:00
namespace
2022-02-08 12:21:16 +00:00
{
2023-07-03 15:28:56 +00:00
void checkTemporaryTableEngineName ( const String & name )
2022-02-08 12:21:16 +00:00
{
2023-10-25 17:51:51 +00:00
if ( name . starts_with ( " Replicated " ) | | name . starts_with ( " Shared " ) | | name = = " KeeperMap " )
throw Exception ( ErrorCodes : : INCORRECT_QUERY , " Temporary tables cannot be created with Replicated, Shared or KeeperMap table engines " ) ;
2022-02-08 12:21:16 +00:00
}
2023-07-03 15:28:56 +00:00
void setDefaultTableEngine ( ASTStorage & storage , DefaultTableEngine engine )
{
2023-07-03 13:14:19 +00:00
if ( engine = = DefaultTableEngine : : None )
throw Exception ( ErrorCodes : : ENGINE_REQUIRED , " Table engine is not specified in CREATE query " ) ;
2022-02-08 12:21:16 +00:00
2023-07-03 13:14:19 +00:00
auto engine_ast = std : : make_shared < ASTFunction > ( ) ;
engine_ast - > name = SettingFieldDefaultTableEngine ( engine ) . toString ( ) ;
engine_ast - > no_empty_args = true ;
storage . set ( storage . engine , engine_ast ) ;
2022-02-08 12:21:16 +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 ;
2022-02-10 22:17:20 +00:00
if ( create . is_dictionary | | create . is_ordinary_view | | create . is_live_view | | create . is_window_view )
return ;
2017-10-26 12:15:33 +00:00
2022-02-10 22:17:20 +00:00
if ( create . is_materialized_view & & create . to_table_id )
2017-10-25 19:52:32 +00:00
return ;
2017-04-01 07:20:54 +00:00
2020-02-14 18:20:20 +00:00
if ( create . temporary )
2016-03-19 01:18:49 +00:00
{
2023-07-03 16:01:35 +00:00
/// Some part of storage definition is specified, but ENGINE is not: just set the one from default_temporary_table_engine setting.
2023-01-30 08:08:20 +00:00
if ( ! create . cluster . empty ( ) )
throw Exception ( ErrorCodes : : INCORRECT_QUERY , " Temporary tables cannot be created with ON CLUSTER clause " ) ;
2023-07-03 13:14:19 +00:00
if ( ! create . storage )
2023-02-07 14:24:47 +00:00
{
2023-01-24 07:23:47 +00:00
auto storage_ast = std : : make_shared < ASTStorage > ( ) ;
create . set ( create . storage , storage_ast ) ;
}
2023-07-03 13:14:19 +00:00
if ( ! create . storage - > engine )
2023-01-24 07:23:47 +00:00
{
2023-07-02 12:20:59 +00:00
setDefaultTableEngine ( * create . storage , getContext ( ) - > getSettingsRef ( ) . default_temporary_table_engine . value ) ;
2023-01-24 07:23:47 +00:00
}
2023-07-03 13:14:19 +00:00
checkTemporaryTableEngineName ( create . storage - > engine - > name ) ;
2022-01-31 17:10:59 +00:00
return ;
2016-03-19 01:18:49 +00:00
}
2022-02-10 20:35:03 +00:00
if ( create . storage )
{
/// Some part of storage definition (such as PARTITION BY) is specified, but ENGINE is not: just set default one.
if ( ! create . storage - > engine )
2023-07-02 12:20:59 +00:00
setDefaultTableEngine ( * create . storage , getContext ( ) - > getSettingsRef ( ) . default_table_engine . value ) ;
2022-02-10 20:35:03 +00:00
return ;
}
2022-02-10 08:53:50 +00:00
if ( ! create . as_table . empty ( ) )
2016-03-19 01:18:49 +00:00
{
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
2021-04-10 23:33:54 +00:00
String as_database_name = getContext ( ) - > resolveDatabase ( create . as_database ) ;
2016-03-19 01:18:49 +00:00
String as_table_name = create . as_table ;
2017-04-01 07:20:54 +00:00
2021-04-10 23:33:54 +00:00
ASTPtr as_create_ptr = DatabaseCatalog : : instance ( ) . getDatabase ( as_database_name ) - > getCreateTableQuery ( as_table_name , getContext ( ) ) ;
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 )
2022-02-10 20:35:03 +00:00
throw Exception ( ErrorCodes : : INCORRECT_QUERY , " Cannot CREATE a table AS {}, it is a View " , qualified_name ) ;
2017-08-18 20:56:19 +00:00
2019-05-29 21:52:13 +00:00
if ( as_create . is_live_view )
2022-02-10 20:35:03 +00:00
throw Exception ( ErrorCodes : : INCORRECT_QUERY , " Cannot CREATE a table AS {}, it is a Live View " , qualified_name ) ;
2019-05-29 21:52:13 +00:00
2020-01-14 03:07:31 +00:00
if ( as_create . is_window_view )
2022-02-10 20:35:03 +00:00
throw Exception ( ErrorCodes : : INCORRECT_QUERY , " Cannot CREATE a table AS {}, it is a Window View " , qualified_name ) ;
2020-01-14 03:07:31 +00:00
2020-01-02 22:45:39 +00:00
if ( as_create . is_dictionary )
2022-02-10 20:35:03 +00:00
throw Exception ( ErrorCodes : : INCORRECT_QUERY , " Cannot CREATE a table AS {}, it is a Dictionary " , qualified_name ) ;
2019-05-29 21:52:13 +00:00
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 )
2022-10-19 11:36:50 +00:00
create . set ( create . as_table_function , as_create . as_table_function - > ptr ( ) ) ;
2020-11-16 10:14:12 +00:00
else
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Cannot set engine, it's a bug. " ) ;
2022-01-31 17:10:59 +00:00
2022-02-01 16:59:41 +00:00
return ;
2016-03-19 01:18:49 +00:00
}
2022-02-10 20:35:03 +00:00
create . set ( create . storage , std : : make_shared < ASTStorage > ( ) ) ;
2023-07-02 12:20:59 +00:00
setDefaultTableEngine ( * create . storage , getContext ( ) - > getSettingsRef ( ) . default_table_engine . value ) ;
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-08-10 09:42:01 +00:00
if ( database - > getEngineName ( ) = = " Replicated " & & getContext ( ) - > getClientInfo ( ) . is_replicated_database_internal
2021-04-10 23:33:54 +00:00
& & ! internal )
2020-10-20 16:14:54 +00:00
{
if ( create . uuid = = UUIDHelpers : : Nil )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Table UUID is not specified in DDL log " ) ;
2020-10-20 16:14:54 +00:00
}
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 ) ;
}
2023-08-23 20:55:46 +00:00
create . generateRandomUUID ( ) ;
2020-07-13 21:21:01 +00:00
}
else
{
2021-04-10 23:33:54 +00:00
bool is_on_cluster = getContext ( ) - > getClientInfo ( ) . query_kind = = ClientInfo : : QueryKind : : SECONDARY_QUERY ;
2021-03-08 17:26:38 +00:00
bool has_uuid = create . uuid ! = UUIDHelpers : : Nil | | create . to_inner_uuid ! = UUIDHelpers : : Nil ;
2023-08-23 20:55:46 +00:00
if ( has_uuid & & ! is_on_cluster & & ! internal )
2023-09-05 18:46:13 +00:00
{
/// We don't show the following error message either
/// 1) if it's a secondary query (an initiator of a CREATE TABLE ON CLUSTER query
/// doesn't know the exact database engines on replicas and generates an UUID, and then the replicas are free to ignore that UUID); or
/// 2) if it's an internal query (for example RESTORE uses internal queries to create tables and it generates an UUID
/// before creating a table to be possibly ignored if the database engine doesn't need it).
2020-07-13 21:21:01 +00:00
throw Exception ( ErrorCodes : : INCORRECT_QUERY ,
2021-09-06 22:13:54 +00:00
" {} UUID specified, but engine of database {} is not Atomic " , kind , create . getDatabase ( ) ) ;
2023-09-05 18:46:13 +00:00
}
2020-07-13 21:21:01 +00:00
2023-08-23 20:55:46 +00:00
/// The database doesn't support UUID so we'll ignore it. The UUID could be set here because of either
/// a) the initiator of `ON CLUSTER` query generated it to ensure the same UUIDs are used on different hosts; or
/// b) `RESTORE from backup` query generated it to ensure the same UUIDs are used on different hosts.
2020-07-13 21:21:01 +00:00
create . uuid = UUIDHelpers : : Nil ;
2021-03-08 17:26:38 +00:00
create . to_inner_uuid = UUIDHelpers : : Nil ;
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.
2021-11-11 13:28:18 +00:00
if ( create . temporary & & create . database )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : BAD_DATABASE_FOR_TEMPORARY_TABLE ,
" Temporary tables cannot be inside a database. "
" You should not specify a database for a temporary table. " ) ;
2019-02-11 14:46:09 +00:00
2021-04-10 23:33:54 +00:00
String current_database = getContext ( ) - > getCurrentDatabase ( ) ;
2021-11-15 10:05:24 +00:00
auto database_name = create . database ? create . getDatabase ( ) : current_database ;
2017-04-01 07:20:54 +00:00
2022-09-14 16:03:36 +00:00
DDLGuardPtr ddl_guard ;
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 ) ;
2022-09-15 19:15:57 +00:00
if ( database - > shouldReplicateQuery ( getContext ( ) , query_ptr ) )
2021-03-08 21:34:52 +00:00
{
2021-09-06 22:13:54 +00:00
auto guard = DatabaseCatalog : : instance ( ) . getDDLGuard ( database_name , create . getTable ( ) ) ;
2022-09-15 19:15:57 +00:00
create . setDatabase ( database_name ) ;
guard - > releaseTableLock ( ) ;
2023-11-09 15:03:51 +00:00
return database - > tryEnqueueReplicatedDDL ( query_ptr , getContext ( ) , QueryFlags { . internal = internal , . distributed_backup_restore = is_restore_from_backup } ) ;
2021-03-08 21:34:52 +00:00
}
2022-07-06 16:50:00 +00:00
if ( ! create . cluster . empty ( ) )
return executeQueryOnCluster ( create ) ;
2022-09-14 16:03:36 +00:00
/// For short syntax of ATTACH query we have to lock table name here, before reading metadata
/// and hold it until table is attached
2023-03-28 21:47:45 +00:00
if ( likely ( need_ddl_guard ) )
2023-03-26 05:15:23 +00:00
ddl_guard = DatabaseCatalog : : instance ( ) . getDDLGuard ( database_name , create . getTable ( ) ) ;
2022-09-14 16:03:36 +00:00
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)
2021-09-06 22:13:54 +00:00
auto query = database - > getCreateTableQuery ( create . getTable ( ) , getContext ( ) ) ;
2022-06-17 08:42:05 +00:00
FunctionNameNormalizer ( ) . visit ( query . get ( ) ) ;
2021-04-22 14:09:30 +00:00
auto create_query = query - > as < ASTCreateQuery & > ( ) ;
2021-04-21 13:45:13 +00:00
2021-04-22 14:09:30 +00:00
if ( ! create . is_dictionary & & create_query . is_dictionary )
throw Exception ( ErrorCodes : : INCORRECT_QUERY ,
" Cannot ATTACH TABLE {}.{}, it is a Dictionary " ,
2021-09-06 22:13:54 +00:00
backQuoteIfNeed ( database_name ) , backQuoteIfNeed ( create . getTable ( ) ) ) ;
2021-04-22 14:09:30 +00:00
if ( create . is_dictionary & & ! create_query . is_dictionary )
throw Exception ( ErrorCodes : : INCORRECT_QUERY ,
" Cannot ATTACH DICTIONARY {}.{}, it is a Table " ,
2021-09-06 22:13:54 +00:00
backQuoteIfNeed ( database_name ) , backQuoteIfNeed ( create . getTable ( ) ) ) ;
2021-04-22 14:09:30 +00:00
create = create_query ; // Copy the saved create query, but use ATTACH instead of CREATE
2021-04-21 13:45:13 +00:00
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 ;
2022-06-29 18:29:50 +00:00
/// Compatibility setting which should be enabled by default on attach
/// Otherwise server will be unable to start for some old-format of IPv6/IPv4 types
getContext ( ) - > setSetting ( " cast_ipv4_ipv6_default_on_conversion_error " , 1 ) ;
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 )
{
2022-09-14 16:03:36 +00:00
chassert ( ! ddl_guard ) ;
2021-04-10 23:33:54 +00:00
fs : : path user_files = fs : : path ( getContext ( ) - > getUserFilesPath ( ) ) . lexically_normal ( ) ;
fs : : path root_path = fs : : path ( getContext ( ) - > getPath ( ) ) . lexically_normal ( ) ;
2021-02-08 09:14:17 +00:00
2021-04-10 23:33:54 +00:00
if ( getContext ( ) - > getClientInfo ( ) . query_kind = = ClientInfo : : QueryKind : : INITIAL_QUERY )
2021-02-08 09:14:17 +00:00
{
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-04-10 23:33:54 +00:00
else if ( create . attach & & ! create . attach_short_syntax & & getContext ( ) - > 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. " ,
2021-09-06 22:13:54 +00:00
create . getTable ( ) , create . getTable ( ) , create . getTable ( ) ) ;
2020-12-08 14:13:35 +00:00
}
2017-10-21 20:38:39 +00:00
2021-11-11 13:28:18 +00:00
if ( ! create . temporary & & ! create . database )
2021-09-06 22:13:54 +00:00
create . setDatabase ( 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
{
2021-04-05 06:36:33 +00:00
// Expand CTE before filling default database
ApplyWithSubqueryVisitor ( ) . visit ( * create . select ) ;
2021-10-01 19:35:17 +00:00
AddDefaultDatabaseVisitor visitor ( getContext ( ) , current_database ) ;
2018-11-01 14:49:37 +00:00
visitor . visit ( * create . select ) ;
}
2017-11-21 13:30:45 +00:00
2021-10-01 19:35:17 +00:00
if ( create . columns_list )
{
AddDefaultDatabaseVisitor visitor ( getContext ( ) , current_database ) ;
visitor . visit ( * create . columns_list ) ;
}
2022-11-23 10:27:38 +00:00
// substitute possible UDFs with their definitions
if ( ! UserDefinedSQLFunctionFactory : : instance ( ) . empty ( ) )
2022-11-23 13:32:36 +00:00
UserDefinedSQLFunctionVisitor : : visit ( query_ptr ) ;
2022-11-23 10:27:38 +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.
2021-08-26 11:01:15 +00:00
TableProperties properties = getTablePropertiesAndNormalizeCreateQuery ( create ) ;
2017-04-01 07:20:54 +00:00
2022-04-12 00:28:11 +00:00
/// Check type compatible for materialized dest table and select columns
2023-11-06 17:14:35 +00:00
if ( create . select & & create . is_materialized_view & & create . to_table_id & & ! create . attach & & ! is_restore_from_backup )
2022-04-12 00:28:11 +00:00
{
2022-04-12 19:10:11 +00:00
if ( StoragePtr to_table = DatabaseCatalog : : instance ( ) . tryGetTable (
2022-04-12 13:29:32 +00:00
{ create . to_table_id . database_name , create . to_table_id . table_name , create . to_table_id . uuid } ,
getContext ( )
2022-04-12 19:10:11 +00:00
) )
{
2023-12-15 15:54:15 +00:00
Block input_block ;
if ( getContext ( ) - > getSettingsRef ( ) . allow_experimental_analyzer )
{
input_block = InterpreterSelectQueryAnalyzer : : getSampleBlock ( create . select - > clone ( ) , getContext ( ) ) ;
}
else
{
input_block = InterpreterSelectWithUnionQuery ( create . select - > clone ( ) ,
getContext ( ) ,
{ } ) . getSampleBlock ( ) ;
}
2022-04-12 19:10:11 +00:00
Block output_block = to_table - > getInMemoryMetadataPtr ( ) - > getSampleBlock ( ) ;
2023-12-15 15:54:15 +00:00
2022-04-12 19:10:11 +00:00
ColumnsWithTypeAndName input_columns ;
ColumnsWithTypeAndName output_columns ;
2023-12-15 15:54:15 +00:00
for ( const auto & input_column : input_block )
2022-04-12 19:10:11 +00:00
{
if ( const auto * output_column = output_block . findByName ( input_column . name ) )
{
2023-12-15 15:54:15 +00:00
input_columns . push_back ( input_column . cloneEmpty ( ) ) ;
2022-04-12 19:10:11 +00:00
output_columns . push_back ( output_column - > cloneEmpty ( ) ) ;
}
}
2022-04-12 00:28:11 +00:00
2022-04-12 19:10:11 +00:00
ActionsDAG : : makeConvertingActions (
input_columns ,
output_columns ,
ActionsDAG : : MatchColumnsMode : : Position
) ;
}
2022-04-12 00:28:11 +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 )
2023-11-08 23:30:35 +00:00
database = DatabaseCatalog : : instance ( ) . tryGetDatabase ( database_name ) ;
2021-01-28 19:02:39 +00:00
2023-11-08 23:30:35 +00:00
if ( need_add_to_database & & database & & database - > shouldReplicateQuery ( getContext ( ) , query_ptr ) )
2020-10-22 15:08:00 +00:00
{
2022-09-15 19:15:57 +00:00
chassert ( ! ddl_guard ) ;
2021-09-06 22:13:54 +00:00
auto guard = DatabaseCatalog : : instance ( ) . getDDLGuard ( create . getDatabase ( ) , create . getTable ( ) ) ;
2022-09-15 19:15:57 +00:00
assertOrSetUUID ( create , database ) ;
guard - > releaseTableLock ( ) ;
2023-11-09 15:03:51 +00:00
return database - > tryEnqueueReplicatedDDL ( query_ptr , getContext ( ) , QueryFlags { . internal = internal , . distributed_backup_restore = is_restore_from_backup } ) ;
2020-07-04 16:32:23 +00:00
}
2022-07-06 16:50:00 +00:00
if ( ! create . cluster . empty ( ) )
2022-09-14 16:03:36 +00:00
{
chassert ( ! ddl_guard ) ;
2022-07-06 16:50:00 +00:00
return executeQueryOnCluster ( create ) ;
2022-09-14 16:03:36 +00:00
}
2022-07-06 16:50:00 +00:00
2023-11-08 23:30:35 +00:00
if ( need_add_to_database & & ! database )
throw Exception ( ErrorCodes : : UNKNOWN_DATABASE , " Database {} does not exist " , backQuoteIfNeed ( database_name ) ) ;
2020-12-25 16:28:10 +00:00
if ( create . replace_table )
2022-09-14 16:03:36 +00:00
{
chassert ( ! ddl_guard ) ;
2020-12-25 16:28:10 +00:00
return doCreateOrReplaceTable ( create , properties ) ;
2022-09-14 16:03:36 +00:00
}
2020-12-25 16:28:10 +00:00
2020-11-05 09:52:23 +00:00
/// Actually creates table
2022-09-14 16:03:36 +00:00
bool created = doCreateTable ( create , properties , ddl_guard ) ;
ddl_guard . reset ( ) ;
2020-11-05 09:52:23 +00:00
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
2021-11-01 18:53:07 +00:00
/// If table has dependencies - add them to the graph
2021-12-06 13:35:34 +00:00
QualifiedTableName qualified_name { database_name , create . getTable ( ) } ;
2023-02-01 23:30:49 +00:00
auto ref_dependencies = getDependenciesFromCreateQuery ( getContext ( ) - > getGlobalContext ( ) , qualified_name , query_ptr ) ;
auto loading_dependencies = getLoadingDependenciesFromCreateQuery ( getContext ( ) - > getGlobalContext ( ) , qualified_name , query_ptr ) ;
DatabaseCatalog : : instance ( ) . addDependencies ( qualified_name , ref_dependencies , loading_dependencies ) ;
2021-11-01 18:53:07 +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 ,
2022-09-14 16:03:36 +00:00
const InterpreterCreateQuery : : TableProperties & properties ,
DDLGuardPtr & ddl_guard )
2019-10-23 18:39:07 +00:00
{
2022-06-22 16:31:42 +00:00
if ( create . temporary )
{
if ( create . if_not_exists & & getContext ( ) - > tryResolveStorageID ( { " " , create . getTable ( ) } , Context : : ResolveExternal ) )
return false ;
2023-02-14 12:10:44 +00:00
DatabasePtr database = DatabaseCatalog : : instance ( ) . getDatabase ( DatabaseCatalog : : TEMPORARY_DATABASE ) ;
2023-01-24 07:39:29 +00:00
2022-06-22 16:31:42 +00:00
String temporary_table_name = create . getTable ( ) ;
2023-01-24 07:39:29 +00:00
auto creator = [ & ] ( const StorageID & table_id )
{
return StorageFactory : : instance ( ) . get ( create ,
database - > getTableDataPath ( table_id . getTableName ( ) ) ,
getContext ( ) ,
getContext ( ) - > getGlobalContext ( ) ,
properties . columns ,
properties . constraints ,
false ) ;
} ;
auto temporary_table = TemporaryTableHolder ( getContext ( ) , creator , query_ptr ) ;
2022-06-22 16:31:42 +00:00
getContext ( ) - > getSessionContext ( ) - > addExternalTable ( temporary_table_name , std : : move ( temporary_table ) ) ;
return true ;
}
2023-03-28 21:47:45 +00:00
if ( ! ddl_guard & & likely ( need_ddl_guard ) )
2022-09-14 16:03:36 +00:00
ddl_guard = DatabaseCatalog : : instance ( ) . getDDLGuard ( create . getDatabase ( ) , create . getTable ( ) ) ;
2019-10-23 18:39:07 +00:00
String data_path ;
DatabasePtr database ;
2022-06-22 16:31:42 +00:00
database = DatabaseCatalog : : instance ( ) . getDatabase ( create . getDatabase ( ) ) ;
assertOrSetUUID ( create , database ) ;
2020-08-18 15:15:27 +00:00
2022-06-22 16:31:42 +00:00
String storage_name = create . is_dictionary ? " Dictionary " : " Table " ;
auto storage_already_exists_error_code = create . is_dictionary ? ErrorCodes : : DICTIONARY_ALREADY_EXISTS : ErrorCodes : : TABLE_ALREADY_EXISTS ;
2021-04-21 14:02:29 +00:00
2022-06-22 16:31:42 +00:00
/// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard.
if ( database - > isTableExist ( create . getTable ( ) , getContext ( ) ) )
{
/// TODO Check structure of table
if ( create . if_not_exists )
return false ;
else if ( create . replace_view )
2022-02-03 12:17:55 +00:00
{
2022-06-22 16:31:42 +00:00
/// when executing CREATE OR REPLACE VIEW, drop current existing view
auto drop_ast = std : : make_shared < ASTDropQuery > ( ) ;
drop_ast - > setDatabase ( create . getDatabase ( ) ) ;
drop_ast - > setTable ( create . getTable ( ) ) ;
drop_ast - > no_ddl_lock = true ;
auto drop_context = Context : : createCopy ( context ) ;
InterpreterDropQuery interpreter ( drop_ast , drop_context ) ;
interpreter . execute ( ) ;
2022-02-03 12:17:55 +00:00
}
2022-06-22 16:31:42 +00:00
else
2021-06-20 08:24:43 +00:00
throw Exception ( storage_already_exists_error_code ,
2022-06-22 16:31:42 +00:00
" {} {}.{} already exists " , storage_name , backQuoteIfNeed ( create . getDatabase ( ) ) , backQuoteIfNeed ( create . getTable ( ) ) ) ;
2016-03-19 01:18:49 +00:00
}
2022-06-22 16:31:42 +00:00
else if ( ! create . attach )
2020-03-10 19:36:17 +00:00
{
2022-06-22 16:31:42 +00:00
/// Checking that table may exists in detached/detached permanently state
try
{
database - > checkMetadataFilenameAvailability ( create . getTable ( ) ) ;
}
catch ( const Exception & )
{
if ( create . if_not_exists )
return false ;
throw ;
}
2020-03-10 19:36:17 +00:00
}
2017-04-01 07:20:54 +00:00
2022-06-22 16:31:42 +00:00
data_path = database - > getTableDataPath ( create ) ;
2023-08-14 17:06:32 +00:00
auto full_data_path = fs : : path { getContext ( ) - > getPath ( ) } / data_path ;
2022-06-22 16:31:42 +00:00
2023-08-14 17:06:32 +00:00
if ( ! create . attach & & ! data_path . empty ( ) & & fs : : exists ( full_data_path ) )
{
if ( getContext ( ) - > getZooKeeperMetadataTransaction ( ) & &
! getContext ( ) - > getZooKeeperMetadataTransaction ( ) - > isInitialQuery ( ) & &
! DatabaseCatalog : : instance ( ) . hasUUIDMapping ( create . uuid ) & &
Context : : getGlobalContextInstance ( ) - > isServerCompletelyStarted ( ) & &
2023-08-15 13:47:11 +00:00
Context : : getGlobalContextInstance ( ) - > getConfigRef ( ) . getBool ( " allow_moving_table_directory_to_trash " , false ) )
2023-08-14 17:06:32 +00:00
{
/// This is a secondary query from a Replicated database. It cannot be retried with another UUID, we must execute it as is.
/// We don't have a table with this UUID (and all metadata is loaded),
/// so the existing directory probably contains some leftovers from previous unsuccessful attempts to create the table
fs : : path trash_path = fs : : path { getContext ( ) - > getPath ( ) } / " trash " / data_path / getHexUIntLowercase ( thread_local_rng ( ) ) ;
LOG_WARNING ( & Poco : : Logger : : get ( " InterpreterCreateQuery " ) , " Directory for {} data {} already exists. Will move it to {} " ,
Poco : : toLower ( storage_name ) , String ( data_path ) , trash_path ) ;
fs : : create_directories ( trash_path . parent_path ( ) ) ;
renameNoReplace ( full_data_path , trash_path ) ;
}
else
{
throw Exception ( storage_already_exists_error_code ,
" Directory for {} data {} already exists " , Poco : : toLower ( storage_name ) , String ( data_path ) ) ;
}
}
2022-06-22 16:31:42 +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 ;
}
2021-04-16 12:18:53 +00:00
if ( create . attach )
{
/// If table was detached it's not possible to attach it back while some threads are using
/// old instance of the storage. For example, AsynchronousMetrics may cause ATTACH to fail,
/// so we allow waiting here. If database_atomic_wait_for_drop_and_detach_synchronously is disabled
/// and old storage instance still exists it will throw exception.
2022-03-24 19:24:05 +00:00
if ( getContext ( ) - > getSettingsRef ( ) . database_atomic_wait_for_drop_and_detach_synchronously )
2021-04-20 11:22:02 +00:00
database - > waitDetachedTableNotInUse ( create . uuid ) ;
2022-03-24 19:24:05 +00:00
else
database - > checkDetachedTableNotInUse ( create . uuid ) ;
2021-04-16 12:18:53 +00:00
}
2022-06-22 16:31:42 +00:00
/// We should lock UUID on CREATE query (because for ATTACH it must be already locked previously).
/// But ATTACH without create.attach_short_syntax flag works like CREATE actually, that's why we check it.
bool need_lock_uuid = ! create . attach_short_syntax ;
TemporaryLockForUUIDDirectory uuid_lock ;
if ( need_lock_uuid )
uuid_lock = TemporaryLockForUUIDDirectory { create . uuid } ;
else if ( create . uuid ! = UUIDHelpers : : Nil & & ! DatabaseCatalog : : instance ( ) . hasUUIDMapping ( create . uuid ) )
2022-06-24 15:15:15 +00:00
{
/// FIXME MaterializedPostgreSQL works with UUIDs incorrectly and breaks invariants
if ( database - > getEngineName ( ) ! = " MaterializedPostgreSQL " )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Cannot find UUID mapping for {}, it's a bug " , create . uuid ) ;
}
2022-06-22 16:31:42 +00:00
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
{
2022-10-19 11:36:50 +00:00
auto table_function_ast = create . as_table_function - > ptr ( ) ;
auto table_function = TableFunctionFactory : : instance ( ) . get ( table_function_ast , getContext ( ) ) ;
2021-12-15 11:30:57 +00:00
/// In case of CREATE AS table_function() query we should use global context
/// in storage creation because there will be no query context on server startup
/// and because storage lifetime is bigger than query context lifetime.
2022-10-19 11:36:50 +00:00
res = table_function - > execute ( table_function_ast , getContext ( ) , create . getTable ( ) , properties . columns , /*use_global_context=*/ true ) ;
2021-09-06 22:13:54 +00:00
res - > renameInMemory ( { create . getDatabase ( ) , create . getTable ( ) , create . uuid } ) ;
2019-07-18 18:29:49 +00:00
}
2019-07-22 12:18:53 +00:00
else
2016-03-19 01:18:49 +00:00
{
2023-01-24 07:39:29 +00:00
res = StorageFactory : : instance ( ) . get ( create ,
data_path ,
getContext ( ) ,
getContext ( ) - > getGlobalContext ( ) ,
properties . columns ,
properties . constraints ,
false ) ;
2022-03-04 18:23:19 +00:00
/// If schema wes inferred while storage creation, add columns description to create query.
addColumnsDescriptionToCreateQueryIfNecessary ( query_ptr - > as < ASTCreateQuery & > ( ) , res ) ;
2019-07-22 12:18:53 +00:00
}
2017-04-01 07:20:54 +00:00
2022-03-11 16:03:05 +00:00
if ( ! create . attach & & getContext ( ) - > getSettingsRef ( ) . database_replicated_allow_only_replicated_engine )
{
bool is_replicated_storage = typeid_cast < const StorageReplicatedMergeTree * > ( res . get ( ) ) ! = nullptr ;
2022-03-14 19:30:52 +00:00
if ( ! is_replicated_storage & & res - > storesDataOnDisk ( ) & & database & & database - > getEngineName ( ) = = " Replicated " )
throw Exception ( ErrorCodes : : UNKNOWN_STORAGE ,
2023-01-23 21:13:58 +00:00
" Only tables with a Replicated engine "
" or tables which do not store data on disk are allowed in a Replicated database " ) ;
2022-03-11 16:03:05 +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 ( ) ) ;
2023-11-15 16:21:02 +00:00
auto * replicated_storage = typeid_cast < StorageReplicatedMergeTree * > ( res . get ( ) ) ;
if ( replicated_storage )
2023-10-26 03:17:30 +00:00
{
2023-11-02 00:42:57 +00:00
const auto probability = getContext ( ) - > getSettingsRef ( ) . create_replicated_merge_tree_fault_injection_probability ;
std : : bernoulli_distribution fault ( probability ) ;
if ( fault ( thread_local_rng ) )
2023-11-15 16:21:02 +00:00
{
/// We emulate the case when the exception was thrown in StorageReplicatedMergeTree constructor
if ( ! create . attach )
replicated_storage - > dropIfEmpty ( ) ;
2023-11-02 00:42:57 +00:00
throw Coordination : : Exception ( Coordination : : Error : : ZCONNECTIONLOSS , " Fault injected (during table creation) " ) ;
2023-11-15 16:21:02 +00:00
}
2023-10-26 03:17:30 +00:00
}
2021-09-06 22:13:54 +00:00
database - > createTable ( getContext ( ) , create . getTable ( ) , 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 )
2021-09-06 22:13:54 +00:00
res - > rename ( actual_data_path , { create . getDatabase ( ) , create . getTable ( ) , 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".
2022-05-06 14:44:00 +00:00
if ( ! res - > supportsDynamicSubcolumns ( ) & & hasDynamicSubcolumns ( res - > getInMemoryMetadataPtr ( ) - > getColumns ( ) ) )
2022-03-16 16:51:23 +00:00
{
throw Exception ( ErrorCodes : : ILLEGAL_COLUMN ,
" Cannot create table with column of type Object, "
" because storage {} doesn't support dynamic subcolumns " ,
res - > getName ( ) ) ;
}
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 )
{
2021-07-02 16:39:55 +00:00
/// Replicated database requires separate contexts for each DDL query
ContextPtr current_context = getContext ( ) ;
2022-08-03 12:13:01 +00:00
if ( auto txn = current_context - > getZooKeeperMetadataTransaction ( ) )
txn - > setIsCreateOrReplaceQuery ( ) ;
2021-07-02 16:39:55 +00:00
ContextMutablePtr create_context = Context : : createCopy ( current_context ) ;
create_context - > setQueryContext ( std : : const_pointer_cast < Context > ( current_context ) ) ;
2022-08-03 12:13:01 +00:00
auto make_drop_context = [ & ] ( ) - > ContextMutablePtr
2021-07-02 16:39:55 +00:00
{
ContextMutablePtr drop_context = Context : : createCopy ( current_context ) ;
2022-08-03 12:13:01 +00:00
drop_context - > setQueryContext ( std : : const_pointer_cast < Context > ( current_context ) ) ;
2021-07-02 16:39:55 +00:00
return drop_context ;
} ;
2020-12-25 16:28:10 +00:00
auto ast_drop = std : : make_shared < ASTDropQuery > ( ) ;
2021-09-06 22:13:54 +00:00
String table_to_replace_name = create . getTable ( ) ;
2020-12-25 16:28:10 +00:00
{
2021-09-06 22:13:54 +00:00
auto database = DatabaseCatalog : : instance ( ) . getDatabase ( create . getDatabase ( ) ) ;
2021-07-01 13:21:38 +00:00
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 " ) ;
2021-09-06 22:13:54 +00:00
UInt64 name_hash = sipHash64 ( create . getDatabase ( ) + create . getTable ( ) ) ;
2021-07-01 13:21:38 +00:00
UInt16 random_suffix = thread_local_rng ( ) ;
2021-07-06 10:26:03 +00:00
if ( auto txn = current_context - > getZooKeeperMetadataTransaction ( ) )
{
/// Avoid different table name on database replicas
random_suffix = sipHash64 ( txn - > getTaskZooKeeperPath ( ) ) ;
}
2021-09-06 22:13:54 +00:00
create . setTable ( fmt : : format ( " _tmp_replace_{}_{} " ,
getHexUIntLowercase ( name_hash ) ,
getHexUIntLowercase ( random_suffix ) ) ) ;
2021-07-02 16:39:55 +00:00
2021-09-06 22:13:54 +00:00
ast_drop - > setTable ( create . getTable ( ) ) ;
2021-06-17 22:29:41 +00:00
ast_drop - > is_dictionary = create . is_dictionary ;
2021-09-06 22:13:54 +00:00
ast_drop - > setDatabase ( create . getDatabase ( ) ) ;
2020-12-25 16:28:10 +00:00
ast_drop - > kind = ASTDropQuery : : Drop ;
2021-07-01 13:21:38 +00:00
}
bool created = false ;
bool renamed = false ;
2020-12-25 16:28:10 +00:00
try
{
2021-07-01 13:21:38 +00:00
/// Create temporary table (random name will be generated)
2022-09-14 16:03:36 +00:00
DDLGuardPtr ddl_guard ;
[[maybe_unused]] bool done = InterpreterCreateQuery ( query_ptr , create_context ) . doCreateTable ( create , properties , ddl_guard ) ;
ddl_guard . reset ( ) ;
2020-12-25 16:28:10 +00:00
assert ( done ) ;
created = true ;
2021-07-01 13:21:38 +00:00
/// Try fill temporary table
BlockIO fill_io = fillTableIfNeeded ( create ) ;
executeTrivialBlockIO ( fill_io , getContext ( ) ) ;
/// Replace target table with created one
2020-12-25 16:28:10 +00:00
auto ast_rename = std : : make_shared < ASTRenameQuery > ( ) ;
ASTRenameQuery : : Element elem
{
2023-02-14 13:01:06 +00:00
ASTRenameQuery : : Table
{
create . getDatabase ( ) . empty ( ) ? nullptr : std : : make_shared < ASTIdentifier > ( create . getDatabase ( ) ) ,
std : : make_shared < ASTIdentifier > ( create . getTable ( ) )
} ,
ASTRenameQuery : : Table
{
create . getDatabase ( ) . empty ( ) ? nullptr : std : : make_shared < ASTIdentifier > ( create . getDatabase ( ) ) ,
std : : make_shared < ASTIdentifier > ( table_to_replace_name )
}
2020-12-25 16:28:10 +00:00
} ;
2021-06-17 22:29:41 +00:00
2020-12-25 16:28:10 +00:00
ast_rename - > elements . push_back ( std : : move ( elem ) ) ;
2021-06-17 22:29:41 +00:00
ast_rename - > dictionary = create . is_dictionary ;
2021-08-05 11:14:50 +00:00
if ( create . create_or_replace )
{
/// CREATE OR REPLACE TABLE
/// Will execute ordinary RENAME instead of EXCHANGE if the target table does not exist
ast_rename - > rename_if_cannot_exchange = true ;
ast_rename - > exchange = false ;
}
else
{
/// REPLACE TABLE
/// Will execute EXCHANGE query and fail if the target table does not exist
ast_rename - > exchange = true ;
}
2021-06-17 22:29:41 +00:00
2021-07-02 16:39:55 +00:00
InterpreterRenameQuery interpreter_rename { ast_rename , current_context } ;
2021-07-01 13:21:38 +00:00
interpreter_rename . execute ( ) ;
renamed = true ;
2020-12-25 16:28:10 +00:00
2021-07-01 13:21:38 +00:00
if ( ! interpreter_rename . renamedInsteadOfExchange ( ) )
{
/// Target table was replaced with new one, drop old table
2022-08-03 12:13:01 +00:00
auto drop_context = make_drop_context ( ) ;
2021-07-02 16:39:55 +00:00
InterpreterDropQuery ( ast_drop , drop_context ) . execute ( ) ;
2021-07-01 13:21:38 +00:00
}
2020-12-25 16:28:10 +00:00
2021-09-06 22:13:54 +00:00
create . setTable ( table_to_replace_name ) ;
2021-06-17 22:29:41 +00:00
2021-07-01 13:21:38 +00:00
return { } ;
2020-12-25 16:28:10 +00:00
}
catch ( . . . )
{
2021-07-01 13:21:38 +00:00
/// Drop temporary table if it was successfully created, but was not renamed to target name
if ( created & & ! renamed )
2021-07-02 16:39:55 +00:00
{
2022-08-03 12:13:01 +00:00
auto drop_context = make_drop_context ( ) ;
2021-07-02 16:39:55 +00:00
InterpreterDropQuery ( ast_drop , drop_context ) . execute ( ) ;
}
2020-12-25 16:28:10 +00:00
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.
2022-06-21 13:03:58 +00:00
if ( create . select & & ! create . attach & & ! create . is_create_empty
2022-05-08 15:43:08 +00:00
& & ! create . is_ordinary_view & & ! create . is_live_view
2022-05-10 07:06:58 +00:00
& & ( ! ( create . is_materialized_view | | create . is_window_view ) | | create . is_populate ) )
2016-03-19 01:18:49 +00:00
{
2018-02-19 00:45:32 +00:00
auto insert = std : : make_shared < ASTInsertQuery > ( ) ;
2021-09-06 22:13:54 +00:00
insert - > table_id = { create . getDatabase ( ) , create . getTable ( ) , create . uuid } ;
2022-05-08 15:43:08 +00:00
if ( create . is_window_view )
{
2022-05-10 07:06:58 +00:00
auto table = DatabaseCatalog : : instance ( ) . getTable ( insert - > table_id , getContext ( ) ) ;
insert - > select = typeid_cast < StorageWindowView * > ( table . get ( ) ) - > getSourceTableSelectQuery ( ) ;
2022-05-08 15:43:08 +00:00
}
2022-05-10 07:06:58 +00:00
else
insert - > select = create . select - > clone ( ) ;
2017-04-01 07:20:54 +00:00
2022-01-16 05:12:55 +00:00
return InterpreterInsertQuery ( insert , getContext ( ) ,
2021-04-10 23:33:54 +00:00
getContext ( ) - > getSettingsRef ( ) . insert_allow_materialized_columns ) . execute ( ) ;
2016-03-19 01:18:49 +00:00
}
2017-04-01 07:20:54 +00:00
2016-03-19 01:18:49 +00:00
return { } ;
}
2021-04-10 23:33:54 +00:00
void InterpreterCreateQuery : : prepareOnClusterQuery ( ASTCreateQuery & create , ContextPtr local_context , const String & cluster_name )
2020-09-23 18:28:59 +00:00
{
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.
2023-08-23 20:55:46 +00:00
create . generateRandomUUID ( ) ;
2020-09-23 18:28:59 +00:00
/// For cross-replication cluster we cannot use UUID in replica path.
2021-04-10 23:33:54 +00:00
String cluster_name_expanded = local_context - > getMacros ( ) - > expand ( cluster_name ) ;
ClusterPtr cluster = local_context - > getCluster ( cluster_name_expanded ) ;
2020-09-23 18:28:59 +00:00
if ( cluster - > maybeCrossReplication ( ) )
{
2022-07-07 21:59:15 +00:00
auto on_cluster_version = local_context - > getSettingsRef ( ) . distributed_ddl_entry_format_version ;
if ( DDLLogEntry : : NORMALIZE_CREATE_ON_INITIATOR_VERSION < = on_cluster_version )
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Value {} of setting distributed_ddl_entry_format_version "
" is incompatible with cross-replication " , on_cluster_version ) ;
2020-09-23 18:28:59 +00:00
/// 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 ;
2021-04-10 23:33:54 +00:00
local_context - > getMacros ( ) - > expand ( zk_path , info ) ;
2020-09-23 18:28:59 +00:00
if ( ! info . expanded_uuid )
return ;
}
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INCORRECT_QUERY ,
" Seems like cluster is configured for cross-replication, "
2020-09-23 18:28:59 +00:00
" 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. "
2023-01-23 21:13:58 +00:00
" Please specify unique zookeeper_path explicitly. " ) ;
2020-09-23 18:28:59 +00:00
}
}
2022-07-06 16:50:00 +00:00
BlockIO InterpreterCreateQuery : : executeQueryOnCluster ( ASTCreateQuery & create )
{
prepareOnClusterQuery ( create , getContext ( ) , create . cluster ) ;
DDLQueryOnClusterParams params ;
params . access_to_check = getRequiredAccess ( ) ;
return executeDDLQueryOnCluster ( query_ptr , getContext ( ) , params ) ;
}
2016-03-19 01:18:49 +00:00
BlockIO InterpreterCreateQuery : : execute ( )
{
2021-02-14 12:53:50 +00:00
FunctionNameNormalizer ( ) . visit ( query_ptr . get ( ) ) ;
2019-03-15 16:14:13 +00:00
auto & create = query_ptr - > as < ASTCreateQuery & > ( ) ;
2022-07-06 16:50:00 +00:00
bool is_create_database = create . database & & ! create . table ;
2022-07-06 17:54:05 +00:00
if ( ! create . cluster . empty ( ) & & ! maybeRemoveOnCluster ( query_ptr , getContext ( ) ) )
2020-03-16 11:38:50 +00:00
{
2022-07-06 16:50:00 +00:00
auto on_cluster_version = getContext ( ) - > getSettingsRef ( ) . distributed_ddl_entry_format_version ;
if ( is_create_database | | on_cluster_version < DDLLogEntry : : NORMALIZE_CREATE_ON_INITIATOR_VERSION )
return executeQueryOnCluster ( create ) ;
2020-03-16 11:38:50 +00:00
}
2020-01-24 16:20:36 +00:00
2021-04-10 23:33:54 +00:00
getContext ( ) - > checkAccess ( getRequiredAccess ( ) ) ;
2020-01-24 16:20:36 +00:00
2018-02-12 18:41:53 +00:00
ASTQueryWithOutput : : resetOutputASTIfExist ( create ) ;
2017-04-01 07:20:54 +00:00
2016-03-19 01:18:49 +00:00
/// CREATE|ATTACH DATABASE
2022-07-06 16:50:00 +00:00
if ( is_create_database )
2017-04-25 15:21:03 +00:00
return createDatabase ( create ) ;
2019-10-11 13:21:52 +00:00
else
2021-04-21 13:45:13 +00:00
return createTable ( 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
2021-11-11 13:28:18 +00:00
if ( ! create . table )
2017-12-20 07:39:52 +00:00
{
2021-09-06 22:13:54 +00:00
required_access . emplace_back ( AccessType : : CREATE_DATABASE , create . getDatabase ( ) ) ;
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
{
2021-09-06 22:13:54 +00:00
required_access . emplace_back ( AccessType : : CREATE_DICTIONARY , create . getDatabase ( ) , create . getTable ( ) ) ;
2020-01-24 16:20:36 +00:00
}
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 )
2021-09-06 22:13:54 +00:00
required_access . emplace_back ( AccessType : : DROP_VIEW | AccessType : : CREATE_VIEW , create . getDatabase ( ) , create . getTable ( ) ) ;
2020-01-24 16:20:36 +00:00
else
2021-09-06 22:13:54 +00:00
required_access . emplace_back ( AccessType : : CREATE_VIEW , create . getDatabase ( ) , create . getTable ( ) ) ;
2020-01-24 16:20:36 +00:00
}
else
{
if ( create . temporary )
2023-03-06 07:20:05 +00:00
{
2023-03-13 16:39:15 +00:00
/// Currently default table engine for temporary tables is Memory. default_table_engine does not affect temporary tables.
2023-03-06 07:20:05 +00:00
if ( create . storage & & create . storage - > engine & & create . storage - > engine - > name ! = " Memory " )
required_access . emplace_back ( AccessType : : CREATE_ARBITRARY_TEMPORARY_TABLE ) ;
2023-03-06 16:37:41 +00:00
else
required_access . emplace_back ( AccessType : : CREATE_TEMPORARY_TABLE ) ;
2023-03-06 07:20:05 +00:00
}
2020-01-24 16:20:36 +00:00
else
2020-12-25 16:28:10 +00:00
{
if ( create . replace_table )
2021-09-06 22:13:54 +00:00
required_access . emplace_back ( AccessType : : DROP_TABLE , create . getDatabase ( ) , create . getTable ( ) ) ;
required_access . emplace_back ( AccessType : : CREATE_TABLE , create . getDatabase ( ) , create . getTable ( ) ) ;
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-04-10 23:33:54 +00:00
void InterpreterCreateQuery : : extendQueryLogElemImpl ( QueryLogElement & elem , const ASTPtr & , ContextPtr ) const
2020-12-14 03:30:39 +00:00
{
2021-02-08 19:36:17 +00:00
if ( ! as_table_saved . empty ( ) )
2020-12-14 03:30:39 +00:00
{
2021-04-10 23:33:54 +00:00
String database = backQuoteIfNeed ( as_database_saved . empty ( ) ? getContext ( ) - > 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
}
}
2022-03-04 18:23:19 +00:00
void InterpreterCreateQuery : : addColumnsDescriptionToCreateQueryIfNecessary ( ASTCreateQuery & create , const StoragePtr & storage )
{
if ( create . is_dictionary | | ( create . columns_list & & create . columns_list - > columns & & ! create . columns_list - > columns - > children . empty ( ) ) )
return ;
auto ast_storage = std : : make_shared < ASTStorage > ( ) ;
2022-10-07 10:46:45 +00:00
unsigned max_parser_depth = static_cast < unsigned > ( getContext ( ) - > getSettingsRef ( ) . max_parser_depth ) ;
auto query_from_storage = DB : : getCreateQueryFromStorage ( storage ,
ast_storage ,
false ,
max_parser_depth ,
true ) ;
2022-03-04 18:23:19 +00:00
auto & create_query_from_storage = query_from_storage - > as < ASTCreateQuery & > ( ) ;
if ( ! create . columns_list )
{
ASTPtr columns_list = std : : make_shared < ASTColumns > ( * create_query_from_storage . columns_list ) ;
create . set ( create . columns_list , columns_list ) ;
}
else
{
ASTPtr columns = std : : make_shared < ASTExpressionList > ( * create_query_from_storage . columns_list - > columns ) ;
create . columns_list - > set ( create . columns_list - > columns , columns ) ;
}
}
2011-08-18 20:33:20 +00:00
}