2016-03-19 01:18:49 +00:00
# include <memory>
2011-11-05 23:31:19 +00:00
# include <Poco/File.h>
2011-10-24 12:10:59 +00:00
2017-04-01 09:19:00 +00:00
# include <Common/escapeForFileName.h>
2018-11-08 12:03:42 +00:00
# include <Common/typeid_cast.h>
2017-04-01 09:19:00 +00:00
# include <IO/WriteBufferFromFile.h>
# include <IO/WriteHelpers.h>
# include <Parsers/ASTColumnDeclaration.h>
2017-12-20 07:39:52 +00:00
# include <Parsers/ASTCreateQuery.h>
2017-04-01 09:19:00 +00:00
# include <Parsers/ASTIdentifier.h>
# include <Parsers/ASTLiteral.h>
2017-12-20 07:39:52 +00:00
# include <Parsers/ASTNameTypePair.h>
2018-02-19 00:45:32 +00:00
# include <Parsers/ASTInsertQuery.h>
2017-04-01 09:19:00 +00:00
# include <Parsers/ParserCreateQuery.h>
2017-12-20 07:39:52 +00:00
# include <Parsers/formatAST.h>
2017-04-01 09:19:00 +00:00
# include <Parsers/parseQuery.h>
# include <Storages/StorageFactory.h>
# include <Storages/StorageLog.h>
2017-05-23 18:33:48 +00:00
# include <Interpreters/Context.h>
2017-04-13 16:12:56 +00:00
# include <Interpreters/DDLWorker.h>
2017-12-20 07:39:52 +00:00
# include <Interpreters/ExpressionAnalyzer.h>
2018-11-08 15:43:14 +00:00
# include <Interpreters/SyntaxAnalyzer.h>
2017-12-20 07:39:52 +00:00
# include <Interpreters/InterpreterCreateQuery.h>
2018-02-27 19:38:59 +00:00
# include <Interpreters/InterpreterSelectWithUnionQuery.h>
2018-02-19 00:45:32 +00:00
# include <Interpreters/InterpreterInsertQuery.h>
2018-02-27 19:38:59 +00:00
# include <Interpreters/ExpressionActions.h>
2018-11-01 14:49:37 +00:00
# include <Interpreters/AddDefaultDatabaseVisitor.h>
2017-04-01 09:19:00 +00:00
# include <DataTypes/DataTypeFactory.h>
2017-12-25 18:58:39 +00:00
# include <DataTypes/NestedUtils.h>
2017-12-20 07:39:52 +00:00
# include <DataTypes/DataTypesNumber.h>
2018-08-14 12:51:45 +00:00
# include <DataTypes/DataTypesDecimal.h>
2019-05-28 15:59:37 +00:00
# include <DataTypes/DataTypeLowCardinality.h>
# include <DataTypes/DataTypeNullable.h>
2017-04-01 09:19:00 +00:00
# include <Databases/DatabaseFactory.h>
# include <Databases/IDatabase.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>
2019-06-16 16:47:47 +00:00
# include <Interpreters/addTypeConversionToAST.h>
2018-10-11 02:57:48 +00:00
2019-07-18 18:29:49 +00:00
# include <TableFunctions/TableFunctionFactory.h>
2018-02-19 00:45:32 +00:00
2011-08-18 20:33:20 +00:00
namespace DB
{
2018-02-19 00:45:32 +00:00
2016-01-11 21:46:36 +00:00
namespace ErrorCodes
{
2017-04-01 07:20:54 +00:00
extern const int TABLE_ALREADY_EXISTS ;
extern const int EMPTY_LIST_OF_COLUMNS_PASSED ;
extern const int INCORRECT_QUERY ;
extern const int ENGINE_REQUIRED ;
extern const int UNKNOWN_DATABASE_ENGINE ;
extern const int DUPLICATE_COLUMN ;
2017-12-20 07:39:52 +00:00
extern const int READONLY ;
2018-08-13 16:53:23 +00:00
extern const int ILLEGAL_COLUMN ;
2018-09-11 14:33:06 +00:00
extern const int DATABASE_ALREADY_EXISTS ;
2018-09-11 18:37:19 +00:00
extern const int QUERY_IS_PROHIBITED ;
2018-10-10 13:15:17 +00:00
extern const int THERE_IS_NO_DEFAULT_VALUE ;
2019-02-11 14:46:09 +00:00
extern const int BAD_DATABASE_FOR_TEMPORARY_TABLE ;
2019-05-28 15:59:37 +00:00
extern const int SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY ;
2016-01-11 21:46:36 +00:00
}
2011-08-18 20:33:20 +00:00
2017-12-22 19:20:18 +00:00
InterpreterCreateQuery : : InterpreterCreateQuery ( const ASTPtr & query_ptr_ , Context & context_ )
: query_ptr ( query_ptr_ ) , context ( context_ )
{
}
2014-06-26 00:58:14 +00:00
2011-11-01 15:16:04 +00:00
2017-04-25 15:21:03 +00:00
BlockIO InterpreterCreateQuery : : createDatabase ( ASTCreateQuery & create )
2011-08-18 20:33:20 +00:00
{
2017-04-25 15:21:03 +00:00
if ( ! create . cluster . empty ( ) )
2018-04-17 19:33:58 +00:00
return executeDDLQueryOnCluster ( query_ptr , context , { create . database } ) ;
2017-04-25 15:21:03 +00:00
2017-04-01 07:20:54 +00:00
String database_name = create . database ;
2018-09-18 18:33:15 +00:00
auto guard = context . getDDLGuard ( database_name , " " ) ;
/// Database can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard
if ( context . isDatabaseExist ( database_name ) )
2018-09-11 14:33:06 +00:00
{
if ( create . if_not_exists )
return { } ;
else
throw Exception ( " Database " + database_name + " already exists. " , ErrorCodes : : DATABASE_ALREADY_EXISTS ) ;
}
2017-04-01 07:20:54 +00:00
String database_engine_name ;
if ( ! create . storage )
{
2017-12-20 07:39:52 +00:00
database_engine_name = " Ordinary " ; /// Default database engine.
2017-09-17 18:49:43 +00:00
auto engine = std : : make_shared < ASTFunction > ( ) ;
engine - > name = database_engine_name ;
auto storage = std : : make_shared < ASTStorage > ( ) ;
storage - > set ( storage - > engine , engine ) ;
create . set ( create . storage , storage ) ;
2017-04-01 07:20:54 +00:00
}
else
{
2017-09-18 14:18:29 +00:00
const ASTStorage & storage = * create . storage ;
const ASTFunction & engine = * storage . engine ;
2017-04-01 07:20:54 +00:00
/// Currently, there are no database engines, that support any arguments.
2019-06-10 09:40:33 +00:00
if ( ( create . columns_list & & create . columns_list - > indices & & ! create . columns_list - > indices - > children . empty ( ) ) )
2017-04-01 07:20:54 +00:00
{
std : : stringstream ostr ;
2017-12-01 18:36:55 +00:00
formatAST ( storage , ostr , false , false ) ;
2017-04-01 07:20:54 +00:00
throw Exception ( " Unknown database engine: " + ostr . str ( ) , ErrorCodes : : UNKNOWN_DATABASE_ENGINE ) ;
}
2017-09-17 18:49:43 +00:00
database_engine_name = engine . name ;
2017-04-01 07:20:54 +00:00
}
String database_name_escaped = escapeForFileName ( database_name ) ;
2017-11-03 19:53:10 +00:00
/// Create directories for tables metadata.
2017-04-01 07:20:54 +00:00
String path = context . getPath ( ) ;
String metadata_path = path + " metadata/ " + database_name_escaped + " / " ;
Poco : : File ( metadata_path ) . createDirectory ( ) ;
2019-06-10 09:40:33 +00:00
DatabasePtr database = DatabaseFactory : : get ( database_name , metadata_path , create . storage , context ) ;
2017-04-01 07:20:54 +00:00
/// Will write file with database metadata, if needed.
String metadata_file_tmp_path = path + " metadata/ " + database_name_escaped + " .sql.tmp " ;
String metadata_file_path = path + " metadata/ " + database_name_escaped + " .sql " ;
bool need_write_metadata = ! create . attach ;
if ( need_write_metadata )
{
create . attach = true ;
create . if_not_exists = false ;
std : : ostringstream statement_stream ;
2017-12-01 18:36:55 +00:00
formatAST ( create , statement_stream , false ) ;
2017-04-01 07:20:54 +00:00
statement_stream < < ' \n ' ;
String statement = statement_stream . str ( ) ;
/// Exclusive flag guarantees, that database is not created right now in another thread.
WriteBufferFromFile out ( metadata_file_tmp_path , statement . size ( ) , O_WRONLY | O_CREAT | O_EXCL ) ;
writeString ( statement , out ) ;
out . next ( ) ;
if ( context . getSettingsRef ( ) . fsync_metadata )
out . sync ( ) ;
out . close ( ) ;
}
try
{
context . addDatabase ( database_name , database ) ;
if ( need_write_metadata )
Poco : : File ( metadata_file_tmp_path ) . renameTo ( metadata_file_path ) ;
2019-07-19 14:22:57 +00:00
database - > loadTables ( context , has_force_restore_data_flag ) ;
2017-04-01 07:20:54 +00:00
}
catch ( . . . )
{
if ( need_write_metadata )
Poco : : File ( metadata_file_tmp_path ) . remove ( ) ;
throw ;
}
2017-04-25 15:21:03 +00:00
return { } ;
2011-08-18 20:33:20 +00:00
}
2016-03-19 01:18:49 +00:00
2017-12-25 21:57:29 +00:00
ASTPtr InterpreterCreateQuery : : formatColumns ( const NamesAndTypesList & columns )
2014-07-10 11:13:45 +00:00
{
2017-04-01 07:20:54 +00:00
auto columns_list = std : : make_shared < ASTExpressionList > ( ) ;
2014-07-10 11:13:45 +00:00
2017-04-01 07:20:54 +00:00
for ( const auto & column : columns )
{
const auto column_declaration = std : : make_shared < ASTColumnDeclaration > ( ) ;
column_declaration - > name = column . name ;
2014-07-10 11:13:45 +00:00
2017-04-01 07:20:54 +00:00
ParserIdentifierWithOptionalParameters storage_p ;
2019-01-18 16:30:35 +00:00
String type_name = column . type - > getName ( ) ;
auto pos = type_name . data ( ) ;
const auto end = pos + type_name . size ( ) ;
2018-04-16 15:11:13 +00:00
column_declaration - > type = parseQuery ( storage_p , pos , end , " data type " , 0 ) ;
2017-04-01 07:20:54 +00:00
columns_list - > children . emplace_back ( column_declaration ) ;
}
2014-07-10 11:13:45 +00:00
2017-04-01 07:20:54 +00:00
return columns_list ;
2014-07-10 11:13:45 +00:00
}
2018-03-06 20:18:34 +00:00
ASTPtr InterpreterCreateQuery : : formatColumns ( const ColumnsDescription & columns )
2014-09-25 15:01:09 +00:00
{
2017-04-01 07:20:54 +00:00
auto columns_list = std : : make_shared < ASTExpressionList > ( ) ;
2014-09-30 03:08:47 +00:00
2019-03-14 15:20:51 +00:00
for ( const auto & column : columns )
2017-04-01 07:20:54 +00:00
{
const auto column_declaration = std : : make_shared < ASTColumnDeclaration > ( ) ;
ASTPtr column_declaration_ptr { column_declaration } ;
2014-09-30 03:08:47 +00:00
2017-04-01 07:20:54 +00:00
column_declaration - > name = column . name ;
2014-09-30 03:08:47 +00:00
2017-04-01 07:20:54 +00:00
ParserIdentifierWithOptionalParameters storage_p ;
2019-01-18 16:30:35 +00:00
String type_name = column . type - > getName ( ) ;
auto type_name_pos = type_name . data ( ) ;
const auto type_name_end = type_name_pos + type_name . size ( ) ;
2019-01-04 12:10:00 +00:00
column_declaration - > type = parseQuery ( storage_p , type_name_pos , type_name_end , " data type " , 0 ) ;
2014-09-30 03:08:47 +00:00
2019-03-14 15:20:51 +00:00
if ( column . default_desc . expression )
2017-04-01 07:20:54 +00:00
{
2019-03-14 15:20:51 +00:00
column_declaration - > default_specifier = toString ( column . default_desc . kind ) ;
column_declaration - > default_expression = column . default_desc . expression - > clone ( ) ;
2018-10-14 15:30:06 +00:00
}
2019-03-14 15:20:51 +00:00
if ( ! column . comment . empty ( ) )
2017-04-01 07:20:54 +00:00
{
2019-03-14 15:20:51 +00:00
column_declaration - > comment = std : : make_shared < ASTLiteral > ( Field ( column . comment ) ) ;
2017-04-01 07:20:54 +00:00
}
2014-09-25 15:01:09 +00:00
2019-03-14 15:20:51 +00:00
if ( column . codec )
2018-10-11 02:57:48 +00:00
{
2019-03-14 15:20:51 +00:00
String codec_desc = column . codec - > getCodecDesc ( ) ;
2018-10-11 02:57:48 +00:00
codec_desc = " CODEC( " + codec_desc + " ) " ;
2019-01-04 12:10:00 +00:00
auto codec_desc_pos = codec_desc . data ( ) ;
const auto codec_desc_end = codec_desc_pos + codec_desc . size ( ) ;
2018-10-11 02:57:48 +00:00
ParserIdentifierWithParameters codec_p ;
2019-01-04 12:10:00 +00:00
column_declaration - > codec = parseQuery ( codec_p , codec_desc_pos , codec_desc_end , " column codec " , 0 ) ;
2018-10-11 02:57:48 +00:00
}
2019-04-15 09:30:45 +00:00
if ( column . ttl )
column_declaration - > ttl = column . ttl ;
2017-04-01 07:20:54 +00:00
columns_list - > children . push_back ( column_declaration_ptr ) ;
}
2014-07-10 11:13:45 +00:00
2017-04-01 07:20:54 +00:00
return columns_list ;
2014-07-10 11:13:45 +00:00
}
2019-01-21 09:47:35 +00:00
ASTPtr InterpreterCreateQuery : : formatIndices ( const IndicesDescription & indices )
{
auto res = std : : make_shared < ASTExpressionList > ( ) ;
for ( const auto & index : indices . indices )
2019-01-22 15:58:11 +00:00
res - > children . push_back ( index - > clone ( ) ) ;
2019-01-21 09:47:35 +00:00
return res ;
}
2019-05-17 04:08:03 +00:00
ASTPtr InterpreterCreateQuery : : formatConstraints ( const ConstraintsDescription & constraints )
{
auto res = std : : make_shared < ASTExpressionList > ( ) ;
for ( const auto & constraint : constraints . constraints )
res - > children . push_back ( constraint - > clone ( ) ) ;
return res ;
}
2019-03-14 15:20:51 +00:00
ColumnsDescription InterpreterCreateQuery : : getColumnsDescription ( const ASTExpressionList & columns_ast , const Context & context )
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 */
ASTPtr default_expr_list = std : : make_shared < ASTExpressionList > ( ) ;
NamesAndTypesList column_names_and_types ;
for ( const auto & ast : columns_ast . children )
{
2019-03-16 21:46:53 +00:00
const auto & col_decl = ast - > as < ASTColumnDeclaration & > ( ) ;
2019-03-14 15:20:51 +00:00
DataTypePtr column_type = nullptr ;
if ( col_decl . type )
{
column_type = DataTypeFactory : : instance ( ) . get ( col_decl . type ) ;
column_names_and_types . emplace_back ( col_decl . name , column_type ) ;
}
else
{
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions
column_names_and_types . emplace_back ( col_decl . name , std : : make_shared < DataTypeUInt8 > ( ) ) ;
}
/// add column to postprocessing if there is a default_expression specified
if ( col_decl . default_expression )
{
2019-06-16 16:47:47 +00:00
/** For columns with explicitly-specified type create two expressions:
* 1. default_expression aliased as column name with _tmp suffix
* 2. conversion of expression ( 1 ) to explicitly - specified type alias as column name
*/
2019-03-14 15:20:51 +00:00
if ( col_decl . type )
{
const auto & final_column_name = col_decl . name ;
const auto tmp_column_name = final_column_name + " _tmp " ;
const auto data_type_ptr = column_names_and_types . back ( ) . type . get ( ) ;
2019-06-16 16:47:47 +00:00
default_expr_list - > children . emplace_back (
setAlias ( addTypeConversionToAST ( std : : make_shared < ASTIdentifier > ( tmp_column_name ) , data_type_ptr - > getName ( ) ) ,
final_column_name ) ) ;
default_expr_list - > children . emplace_back (
setAlias (
col_decl . default_expression - > clone ( ) ,
tmp_column_name ) ) ;
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 ( ) )
{
auto syntax_analyzer_result = SyntaxAnalyzer ( context ) . analyze ( default_expr_list , column_names_and_types ) ;
const auto actions = ExpressionAnalyzer ( default_expr_list , syntax_analyzer_result , context ) . getActions ( true ) ;
for ( auto action : actions - > getActions ( ) )
if ( action . type = = ExpressionAction : : Type : : JOIN | | action . type = = ExpressionAction : : Type : : ARRAY_JOIN )
throw Exception ( " Cannot CREATE table. Unsupported default value that requires ARRAY JOIN or JOIN action " , ErrorCodes : : THERE_IS_NO_DEFAULT_VALUE ) ;
defaults_sample_block = actions - > getSampleBlock ( ) ;
}
2018-03-06 20:18:34 +00:00
ColumnsDescription res ;
2019-03-14 15:20:51 +00:00
auto name_type_it = column_names_and_types . begin ( ) ;
for ( auto ast_it = columns_ast . children . begin ( ) ; ast_it ! = columns_ast . children . end ( ) ; + + ast_it , + + name_type_it )
{
ColumnDescription column ;
2019-03-16 21:46:53 +00:00
auto & col_decl = ( * ast_it ) - > as < ASTColumnDeclaration & > ( ) ;
2019-03-14 15:20:51 +00:00
column . name = col_decl . name ;
2016-03-19 01:18:49 +00:00
2019-03-14 15:20:51 +00:00
if ( col_decl . default_expression )
{
ASTPtr default_expr = col_decl . default_expression - > clone ( ) ;
if ( col_decl . type )
{
const auto & deduced_type = defaults_sample_block . getByName ( column . name + " _tmp " ) . type ;
column . type = name_type_it - > type ;
2016-03-19 01:18:49 +00:00
2019-03-14 15:20:51 +00:00
if ( ! column . type - > equals ( * deduced_type ) )
2019-06-16 16:47:47 +00:00
default_expr = addTypeConversionToAST ( std : : move ( default_expr ) , column . type - > getName ( ) ) ;
2019-03-14 15:20:51 +00:00
}
else
column . type = defaults_sample_block . getByName ( column . name ) . type ;
column . default_desc . kind = columnDefaultKindFromString ( col_decl . default_specifier ) ;
column . default_desc . expression = default_expr ;
}
else if ( col_decl . type )
column . type = name_type_it - > type ;
else
throw Exception ( ) ;
if ( col_decl . comment )
2019-03-16 21:46:53 +00:00
column . comment = col_decl . comment - > as < ASTLiteral & > ( ) . value . get < String > ( ) ;
2019-03-14 15:20:51 +00:00
if ( col_decl . codec )
column . codec = CompressionCodecFactory : : instance ( ) . get ( col_decl . codec , column . type ) ;
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 ) ) ;
}
res . flattenNested ( ) ;
if ( res . getAllPhysical ( ) . empty ( ) )
2017-04-01 07:20:54 +00:00
throw Exception { " Cannot CREATE table without physical columns " , ErrorCodes : : EMPTY_LIST_OF_COLUMNS_PASSED } ;
2016-03-19 01:18:49 +00:00
2017-04-01 07:20:54 +00:00
return res ;
2016-03-19 01:18:49 +00:00
}
2019-08-24 21:20:20 +00:00
ConstraintsDescription InterpreterCreateQuery : : getConstraintsDescription ( const ASTExpressionList * constraints )
{
ConstraintsDescription res ;
if ( constraints )
for ( const auto & constraint : constraints - > children )
res . constraints . push_back ( std : : dynamic_pointer_cast < ASTConstraintDeclaration > ( constraint - > clone ( ) ) ) ;
return res ;
}
ColumnsDescription InterpreterCreateQuery : : setProperties (
2017-04-01 07:20:54 +00:00
ASTCreateQuery & create , const Block & as_select_sample , const StoragePtr & as_storage ) const
2016-03-19 01:18:49 +00:00
{
2019-03-14 15:20:51 +00:00
ColumnsDescription columns ;
2019-02-05 13:52:45 +00:00
IndicesDescription indices ;
2019-05-17 04:08:03 +00:00
ConstraintsDescription constraints ;
2019-02-05 13:52:45 +00:00
if ( create . columns_list )
2017-04-01 07:20:54 +00:00
{
2019-02-05 13:52:45 +00:00
if ( create . columns_list - > columns )
2019-03-14 15:20:51 +00:00
columns = getColumnsDescription ( * create . columns_list - > columns , context ) ;
2019-08-24 21:20:20 +00:00
2019-02-05 13:52:45 +00:00
if ( create . columns_list - > indices )
for ( const auto & index : create . columns_list - > indices - > children )
indices . indices . push_back (
2019-03-14 15:20:51 +00:00
std : : dynamic_pointer_cast < ASTIndexDeclaration > ( index - > clone ( ) ) ) ;
2019-08-24 21:20:20 +00:00
2019-05-17 04:08:03 +00:00
if ( create . columns_list - > constraints )
for ( const auto & constraint : create . columns_list - > constraints - > children )
constraints . constraints . push_back (
2019-08-24 21:20:20 +00:00
std : : dynamic_pointer_cast < ASTConstraintDeclaration > ( constraint - > clone ( ) ) ) ;
2017-04-01 07:20:54 +00:00
}
else if ( ! create . as_table . empty ( ) )
{
2019-03-14 15:20:51 +00:00
columns = as_storage - > getColumns ( ) ;
2019-05-02 16:07:23 +00:00
indices = as_storage - > getIndices ( ) ;
2019-05-17 05:05:19 +00:00
constraints = as_storage - > getConstraints ( ) ;
2017-04-01 07:20:54 +00:00
}
else if ( create . select )
{
2019-03-14 15:20:51 +00:00
columns = ColumnsDescription ( as_select_sample . getNamesAndTypesList ( ) ) ;
2017-04-01 07:20:54 +00:00
}
else
throw Exception ( " Incorrect CREATE query: required list of column descriptions or AS section or SELECT. " , ErrorCodes : : INCORRECT_QUERY ) ;
/// Even if query has list of columns, canonicalize it (unfold Nested columns).
2019-03-14 15:20:51 +00:00
ASTPtr new_columns = formatColumns ( columns ) ;
2019-02-05 13:52:45 +00:00
ASTPtr new_indices = formatIndices ( indices ) ;
2019-05-17 04:08:03 +00:00
ASTPtr new_constraints = formatConstraints ( constraints ) ;
2019-02-05 13:52:45 +00:00
2019-01-20 15:02:19 +00:00
if ( ! create . columns_list )
{
auto new_columns_list = std : : make_shared < ASTColumns > ( ) ;
create . set ( create . columns_list , new_columns_list ) ;
}
2019-01-20 11:03:21 +00:00
2019-01-20 15:02:19 +00:00
if ( create . columns_list - > columns )
create . columns_list - > replace ( create . columns_list - > columns , new_columns ) ;
2017-04-01 07:20:54 +00:00
else
2019-01-20 15:02:19 +00:00
create . columns_list - > set ( create . columns_list - > columns , new_columns ) ;
2017-04-01 07:20:54 +00:00
2019-02-05 13:52:45 +00:00
if ( new_indices & & create . columns_list - > indices )
create . columns_list - > replace ( create . columns_list - > indices , new_indices ) ;
else if ( new_indices )
create . columns_list - > set ( create . columns_list - > indices , new_indices ) ;
2019-05-17 04:08:03 +00:00
if ( new_constraints & & create . columns_list - > constraints )
create . columns_list - > replace ( create . columns_list - > constraints , new_constraints ) ;
else if ( new_constraints )
create . columns_list - > set ( create . columns_list - > constraints , new_constraints ) ;
2017-04-01 07:20:54 +00:00
/// Check for duplicates
std : : set < String > all_columns ;
2019-03-14 15:20:51 +00:00
for ( const auto & column : columns )
2017-04-01 07:20:54 +00:00
{
2019-03-14 15:20:51 +00:00
if ( ! all_columns . emplace ( column . name ) . second )
throw Exception ( " Column " + backQuoteIfNeed ( column . name ) + " already exists " , ErrorCodes : : DUPLICATE_COLUMN ) ;
}
2017-04-01 07:20:54 +00:00
2019-03-14 15:20:51 +00:00
return columns ;
2016-03-19 01:18:49 +00:00
}
2017-10-25 19:52:32 +00:00
void InterpreterCreateQuery : : setEngine ( ASTCreateQuery & create ) const
2016-03-19 01:18:49 +00:00
{
2017-10-25 19:52:32 +00:00
if ( create . storage )
2017-10-26 12:15:33 +00:00
{
2018-10-18 15:03:14 +00:00
if ( create . temporary & & create . storage - > engine - > name ! = " Memory " )
2017-10-26 12:15:33 +00:00
throw Exception (
" Temporary tables can only be created with ENGINE = Memory, not " + create . storage - > engine - > name ,
ErrorCodes : : INCORRECT_QUERY ) ;
2017-10-25 19:52:32 +00:00
return ;
2017-10-26 12:15:33 +00:00
}
2017-04-01 07:20:54 +00:00
2019-05-30 21:29:30 +00:00
if ( create . temporary & & ! create . is_live_view )
2017-04-01 07:20:54 +00:00
{
2017-09-17 18:49:43 +00:00
auto engine_ast = std : : make_shared < ASTFunction > ( ) ;
2017-10-25 19:52:32 +00:00
engine_ast - > name = " Memory " ;
2017-09-17 18:49:43 +00:00
auto storage_ast = std : : make_shared < ASTStorage > ( ) ;
storage_ast - > set ( storage_ast - > engine , engine_ast ) ;
create . set ( create . storage , storage_ast ) ;
2017-04-01 07:20:54 +00:00
}
else if ( ! create . as_table . empty ( ) )
{
2017-04-02 17:37:49 +00:00
/// NOTE Getting the structure from the table specified in the AS is done not atomically with the creation of the table.
2017-04-01 07:20:54 +00:00
String as_database_name = create . as_database . empty ( ) ? context . getCurrentDatabase ( ) : create . as_database ;
String as_table_name = create . as_table ;
2018-03-13 13:28:32 +00:00
ASTPtr as_create_ptr = context . getCreateTableQuery ( as_database_name , as_table_name ) ;
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
2017-10-25 19:52:32 +00:00
if ( as_create . is_view )
throw Exception (
2017-12-22 19:20:18 +00:00
" Cannot CREATE a table AS " + as_database_name + " . " + as_table_name + " , it is a View " ,
2017-10-25 19:52:32 +00:00
ErrorCodes : : INCORRECT_QUERY ) ;
2017-08-18 20:56:19 +00:00
2019-05-29 21:52:13 +00:00
if ( as_create . is_live_view )
throw Exception (
" Cannot CREATE a table AS " + as_database_name + " . " + as_table_name + " , it is a Live View " ,
ErrorCodes : : INCORRECT_QUERY ) ;
2017-10-25 19:52:32 +00:00
create . set ( create . storage , as_create . storage - > ptr ( ) ) ;
2017-04-01 07:20:54 +00:00
}
2016-03-19 01:18:49 +00:00
}
2017-04-21 12:39:28 +00:00
BlockIO InterpreterCreateQuery : : createTable ( ASTCreateQuery & create )
2016-03-19 01:18:49 +00:00
{
2017-04-21 12:39:28 +00:00
if ( ! create . cluster . empty ( ) )
2018-04-17 19:33:58 +00:00
{
NameSet databases { create . database } ;
if ( ! create . to_table . empty ( ) )
databases . emplace ( create . to_database ) ;
2018-10-25 09:40:30 +00:00
return executeDDLQueryOnCluster ( query_ptr , context , std : : move ( databases ) ) ;
2018-04-17 19:33:58 +00:00
}
2017-04-21 12:39:28 +00:00
2019-02-11 14:46:09 +00:00
/// Temporary tables are created out of databases.
2019-06-03 19:53:11 +00:00
if ( create . temporary & & ! create . database . empty ( ) & & ! create . is_live_view )
2019-02-11 14:46:09 +00:00
throw Exception ( " Temporary tables cannot be inside a database. You should not specify a database for a temporary table. " ,
ErrorCodes : : BAD_DATABASE_FOR_TEMPORARY_TABLE ) ;
2017-04-01 07:20:54 +00:00
String path = context . getPath ( ) ;
String current_database = context . getCurrentDatabase ( ) ;
String database_name = create . database . empty ( ) ? current_database : create . database ;
String table_name = create . table ;
String table_name_escaped = escapeForFileName ( table_name ) ;
2017-10-21 20:38:39 +00:00
// If this is a stub ATTACH query, read the query definition from the database
2019-01-20 11:03:21 +00:00
if ( create . attach & & ! create . storage & & ! create . columns_list )
2017-10-21 20:38:39 +00:00
{
2017-10-25 00:55:31 +00:00
// Table SQL definition is available even if the table is detached
2018-03-13 13:28:32 +00:00
auto query = context . getCreateTableQuery ( database_name , table_name ) ;
2019-03-15 16:14:13 +00:00
create = query - > as < ASTCreateQuery & > ( ) ; // Copy the saved create query, but use ATTACH instead of CREATE
2017-10-21 20:38:39 +00:00
create . attach = true ;
}
2017-10-30 17:53:01 +00:00
if ( create . to_database . empty ( ) )
create . to_database = current_database ;
2019-05-29 21:52:13 +00:00
if ( create . select & & ( create . is_view | | create . is_materialized_view | | create . is_live_view ) )
2018-11-01 14:49:37 +00:00
{
AddDefaultDatabaseVisitor visitor ( current_database ) ;
visitor . visit ( * create . select ) ;
}
2017-11-21 13:30:45 +00:00
2017-04-01 07:20:54 +00:00
Block as_select_sample ;
2019-01-20 11:03:21 +00:00
if ( create . select & & ( ! create . attach | | ! create . columns_list ) )
2018-02-27 19:38:59 +00:00
as_select_sample = InterpreterSelectWithUnionQuery : : getSampleBlock ( create . select - > clone ( ) , context ) ;
2017-04-01 07:20:54 +00:00
String as_database_name = create . as_database . empty ( ) ? current_database : create . as_database ;
String as_table_name = create . as_table ;
StoragePtr as_storage ;
2019-03-07 18:04:47 +00:00
TableStructureReadLockHolder as_storage_lock ;
2019-07-18 18:29:49 +00:00
2017-04-01 07:20:54 +00:00
if ( ! as_table_name . empty ( ) )
{
as_storage = context . getTable ( as_database_name , as_table_name ) ;
2019-03-07 18:04:47 +00:00
as_storage_lock = as_storage - > lockStructureForShare ( false , context . getCurrentQueryId ( ) ) ;
2017-04-01 07:20:54 +00:00
}
2019-07-22 13:49:16 +00:00
ColumnsDescription columns ;
2019-08-26 02:53:42 +00:00
ConstraintsDescription constraints ;
2019-07-22 13:49:16 +00:00
StoragePtr res ;
2017-04-01 07:20:54 +00:00
2019-07-18 18:29:49 +00:00
if ( create . as_table_function )
2019-05-28 15:59:37 +00:00
{
2019-07-22 12:50:10 +00:00
const auto & table_function = create . as_table_function - > as < ASTFunction & > ( ) ;
2019-07-18 18:29:49 +00:00
const auto & factory = TableFunctionFactory : : instance ( ) ;
2019-07-22 12:50:10 +00:00
res = factory . get ( table_function . name , context ) - > execute ( create . as_table_function , context , create . table ) ;
2019-07-18 18:29:49 +00:00
}
2019-07-22 12:18:53 +00:00
else
2017-04-01 07:20:54 +00:00
{
2019-07-22 12:18:53 +00:00
/// Set and retrieve list of columns.
2019-08-24 21:20:20 +00:00
columns = setProperties ( create , as_select_sample , as_storage ) ;
2019-08-26 02:53:42 +00:00
constraints = getConstraintsDescription ( create . columns_list - > constraints ) ;
2017-04-01 07:20:54 +00:00
2019-07-22 12:18:53 +00:00
/// Check low cardinality types in creating table if it was not allowed in setting
2019-08-09 19:17:19 +00:00
if ( ! create . attach & & ! context . getSettingsRef ( ) . allow_suspicious_low_cardinality_types & & ! create . is_materialized_view )
2019-05-28 15:59:37 +00:00
{
2019-07-22 12:18:53 +00:00
for ( const auto & name_and_type_pair : columns . getAllPhysical ( ) )
2019-05-28 15:59:37 +00:00
{
2019-07-22 12:18:53 +00:00
if ( const auto * current_type_ptr = typeid_cast < const DataTypeLowCardinality * > ( name_and_type_pair . type . get ( ) ) )
{
if ( ! isStringOrFixedString ( * removeNullable ( current_type_ptr - > getDictionaryType ( ) ) ) )
throw Exception ( " Creating columns of type " + current_type_ptr - > getName ( ) + " is prohibited by default due to expected negative impact on performance. It can be enabled with the \" allow_suspicious_low_cardinality_types \" setting. " ,
ErrorCodes : : SUSPICIOUS_TYPE_FOR_LOW_CARDINALITY ) ;
}
2019-05-28 15:59:37 +00:00
}
}
2017-04-01 07:20:54 +00:00
2019-07-22 12:18:53 +00:00
/// Set the table engine if it was not specified explicitly.
setEngine ( create ) ;
}
2017-04-01 07:20:54 +00:00
{
std : : unique_ptr < DDLGuard > guard ;
2017-11-03 19:53:10 +00:00
String data_path ;
DatabasePtr database ;
2019-06-03 19:53:11 +00:00
if ( ! create . temporary | | create . is_live_view )
2017-04-01 07:20:54 +00:00
{
2017-11-03 19:53:10 +00:00
database = context . getDatabase ( database_name ) ;
2018-02-21 19:26:59 +00:00
data_path = database - > getDataPath ( ) ;
2017-04-01 07:20:54 +00:00
2018-09-18 18:33:15 +00:00
/** If the request specifies IF NOT EXISTS, we allow concurrent CREATE queries (which do nothing).
* If table doesnt exist , one thread is creating table , while others wait in DDLGuard .
2017-04-01 07:20:54 +00:00
*/
2018-09-18 18:33:15 +00:00
guard = context . getDDLGuard ( database_name , table_name ) ;
2017-04-01 07:20:54 +00:00
2018-09-18 18:33:15 +00:00
/// Table can be created before or it can be created concurrently in another thread, while we were waiting in DDLGuard.
if ( database - > isTableExist ( context , table_name ) )
2017-04-01 07:20:54 +00:00
{
if ( create . if_not_exists )
return { } ;
2019-03-11 16:50:31 +00:00
else if ( create . replace_view )
{
/// when executing CREATE OR REPLACE VIEW, drop current existing view
auto drop_ast = std : : make_shared < ASTDropQuery > ( ) ;
drop_ast - > database = database_name ;
drop_ast - > table = table_name ;
drop_ast - > no_ddl_lock = true ;
InterpreterDropQuery interpreter ( drop_ast , context ) ;
interpreter . execute ( ) ;
}
2017-04-01 07:20:54 +00:00
else
throw Exception ( " Table " + database_name + " . " + table_name + " already exists. " , ErrorCodes : : TABLE_ALREADY_EXISTS ) ;
}
}
2018-03-12 15:36:33 +00:00
else if ( context . tryGetExternalTable ( table_name ) & & create . if_not_exists )
return { } ;
2018-05-07 02:01:11 +00:00
2019-07-22 12:18:53 +00:00
if ( ! create . as_table_function )
2019-07-18 18:29:49 +00:00
{
res = StorageFactory : : instance ( ) . get ( create ,
data_path ,
table_name ,
database_name ,
context ,
context . getGlobalContext ( ) ,
columns ,
2019-08-24 21:20:20 +00:00
constraints ,
2019-07-18 18:29:49 +00:00
create . attach ,
false ) ;
}
2017-04-01 07:20:54 +00:00
2019-06-03 19:53:11 +00:00
if ( create . temporary & & ! create . is_live_view )
2018-02-14 04:00:37 +00:00
context . getSessionContext ( ) . addExternalTable ( table_name , res , query_ptr ) ;
2017-04-01 07:20:54 +00:00
else
2017-11-03 19:53:10 +00:00
database - > createTable ( context , table_name , res , query_ptr ) ;
2017-04-01 07:20:54 +00:00
2018-09-07 01:39:48 +00:00
/// We must call "startup" and "shutdown" while holding DDLGuard.
/// Because otherwise method "shutdown" (from InterpreterDropQuery) can be called before startup
/// (in case when table was created and instantly dropped before started up)
///
/// Method "startup" may create background tasks and method "shutdown" will wait for them.
/// But if "shutdown" is called before "startup", it will exit early, because there are no background tasks to wait.
/// Then background task is created by "startup" method. And when destructor of a table object is called, background task is still active,
/// and the task will use references to freed data.
res - > startup ( ) ;
}
2017-06-06 17:06:14 +00:00
2017-11-21 13:30:45 +00:00
/// If the query is a CREATE SELECT, insert the data into the table.
if ( create . select & & ! create . attach
2019-05-29 21:52:13 +00:00
& & ! create . is_view & & ! create . is_live_view & & ( ! create . is_materialized_view | | create . is_populate ) )
2017-04-01 07:20:54 +00:00
{
2018-02-19 00:45:32 +00:00
auto insert = std : : make_shared < ASTInsertQuery > ( ) ;
2017-04-01 07:20:54 +00:00
2018-10-18 15:03:14 +00:00
if ( ! create . temporary )
2018-02-19 03:00:16 +00:00
insert - > database = database_name ;
2018-02-19 00:45:32 +00:00
insert - > table = table_name ;
insert - > select = create . select - > clone ( ) ;
2017-04-01 07:20:54 +00:00
2018-10-18 15:03:14 +00:00
if ( create . temporary & & ! context . getSessionContext ( ) . hasQueryContext ( ) )
2019-07-08 00:51:43 +00:00
context . getSessionContext ( ) . makeQueryContext ( ) ;
2018-09-13 03:34:58 +00:00
2018-03-07 18:53:21 +00:00
return InterpreterInsertQuery ( insert ,
2018-10-18 15:03:14 +00:00
create . temporary ? context . getSessionContext ( ) : context ,
2018-03-07 18:53:21 +00:00
context . getSettingsRef ( ) . insert_allow_materialized_columns ) . execute ( ) ;
2017-04-01 07:20:54 +00:00
}
return { } ;
2016-03-19 01:18:49 +00:00
}
2017-04-13 13:42:29 +00:00
2016-03-19 01:18:49 +00:00
BlockIO InterpreterCreateQuery : : execute ( )
{
2019-03-15 16:14:13 +00:00
auto & create = query_ptr - > as < ASTCreateQuery & > ( ) ;
2017-12-22 19:20:18 +00:00
checkAccess ( create ) ;
2018-02-12 18:41:53 +00:00
ASTQueryWithOutput : : resetOutputASTIfExist ( create ) ;
2017-04-01 07:20:54 +00:00
/// CREATE|ATTACH DATABASE
if ( ! create . database . empty ( ) & & create . table . empty ( ) )
{
2017-04-25 15:21:03 +00:00
return createDatabase ( create ) ;
2017-04-01 07:20:54 +00:00
}
else
2017-04-21 12:39:28 +00:00
return createTable ( create ) ;
2016-03-19 01:18:49 +00:00
}
2017-12-22 19:20:18 +00:00
2017-12-20 07:39:52 +00:00
void InterpreterCreateQuery : : checkAccess ( const ASTCreateQuery & create )
{
2018-01-18 23:40:32 +00:00
/// Internal queries (initiated by the server itself) always have access to everything.
if ( internal )
return ;
2017-12-20 07:39:52 +00:00
const Settings & settings = context . getSettingsRef ( ) ;
2018-03-11 00:15:26 +00:00
auto readonly = settings . readonly ;
2018-09-11 18:37:19 +00:00
auto allow_ddl = settings . allow_ddl ;
2017-12-20 07:39:52 +00:00
2018-09-11 18:37:19 +00:00
if ( ! readonly & & allow_ddl )
2017-12-20 07:39:52 +00:00
return ;
2016-03-19 01:18:49 +00:00
2017-12-20 07:39:52 +00:00
/// CREATE|ATTACH DATABASE
if ( ! create . database . empty ( ) & & create . table . empty ( ) )
{
2018-09-11 18:37:19 +00:00
if ( readonly )
throw Exception ( " Cannot create database in readonly mode " , ErrorCodes : : READONLY ) ;
2018-09-13 03:34:58 +00:00
throw Exception ( " Cannot create database. DDL queries are prohibited for the user " , ErrorCodes : : QUERY_IS_PROHIBITED ) ;
2017-12-20 07:39:52 +00:00
}
2018-10-18 15:03:14 +00:00
if ( create . temporary & & readonly > = 2 )
2017-12-20 07:39:52 +00:00
return ;
2018-09-11 18:37:19 +00:00
if ( readonly )
throw Exception ( " Cannot create table in readonly mode " , ErrorCodes : : READONLY ) ;
2018-09-13 03:34:58 +00:00
throw Exception ( " Cannot create table. DDL queries are prohibited for the user " , ErrorCodes : : QUERY_IS_PROHIBITED ) ;
2017-12-20 07:39:52 +00:00
}
2011-08-18 20:33:20 +00:00
}