2017-04-01 09:19:00 +00:00
# include <Interpreters/InterpreterAlterQuery.h>
2017-04-25 15:21:03 +00:00
# include <Interpreters/DDLWorker.h>
2017-04-01 09:19:00 +00:00
# include <Parsers/ASTAlterQuery.h>
# include <Parsers/ASTCreateQuery.h>
# include <Parsers/ASTExpressionList.h>
# include <Parsers/ASTNameTypePair.h>
# include <Parsers/ASTIdentifier.h>
# include <Parsers/ASTLiteral.h>
# include <Parsers/ParserCreateQuery.h>
# include <IO/copyData.h>
# include <IO/ReadBufferFromFile.h>
# include <Common/escapeForFileName.h>
# include <DataTypes/DataTypeFactory.h>
# include <Parsers/formatAST.h>
# include <Parsers/parseQuery.h>
2013-08-07 13:07:42 +00:00
2014-06-03 23:09:57 +00:00
# include <Poco/FileStream.h>
2013-08-07 13:07:42 +00:00
# include <algorithm>
2013-08-09 00:12:59 +00:00
2016-01-11 21:46:36 +00:00
namespace DB
{
namespace ErrorCodes
{
2017-04-01 07:20:54 +00:00
extern const int LOGICAL_ERROR ;
extern const int ARGUMENT_OUT_OF_BOUND ;
2017-06-22 11:01:30 +00:00
extern const int BAD_ARGUMENTS ;
2017-07-14 18:09:28 +00:00
extern const int ILLEGAL_COLUMN ;
2016-01-11 21:46:36 +00:00
}
2013-09-23 12:01:19 +00:00
2017-05-23 18:01:50 +00:00
InterpreterAlterQuery : : InterpreterAlterQuery ( const ASTPtr & query_ptr_ , const Context & context_ )
2017-04-01 07:20:54 +00:00
: query_ptr ( query_ptr_ ) , context ( context_ )
2013-08-07 13:07:42 +00:00
{
}
2015-06-18 02:11:05 +00:00
BlockIO InterpreterAlterQuery : : execute ( )
2013-12-18 11:19:37 +00:00
{
2017-04-01 07:20:54 +00:00
auto & alter = typeid_cast < ASTAlterQuery & > ( * query_ptr ) ;
2017-04-25 15:21:03 +00:00
if ( ! alter . cluster . empty ( ) )
return executeDDLQueryOnCluster ( query_ptr , context ) ;
2017-04-01 07:20:54 +00:00
const String & table_name = alter . table ;
String database_name = alter . database . empty ( ) ? context . getCurrentDatabase ( ) : alter . database ;
StoragePtr table = context . getTable ( database_name , table_name ) ;
AlterCommands alter_commands ;
PartitionCommands partition_commands ;
parseAlter ( alter . parameters , alter_commands , partition_commands ) ;
2017-07-14 18:09:28 +00:00
partition_commands . validate ( table . get ( ) ) ;
2017-04-01 07:20:54 +00:00
for ( const PartitionCommand & command : partition_commands )
{
switch ( command . type )
{
case PartitionCommand : : DROP_PARTITION :
2017-09-06 20:34:26 +00:00
table - > dropPartition ( query_ptr , command . partition , command . detach , context ) ;
2017-04-01 07:20:54 +00:00
break ;
case PartitionCommand : : ATTACH_PARTITION :
2017-09-06 20:34:26 +00:00
table - > attachPartition ( command . partition , command . part , context ) ;
2017-04-01 07:20:54 +00:00
break ;
case PartitionCommand : : FETCH_PARTITION :
2017-09-06 20:34:26 +00:00
table - > fetchPartition ( command . partition , command . from , context ) ;
2017-04-01 07:20:54 +00:00
break ;
case PartitionCommand : : FREEZE_PARTITION :
2017-09-06 20:34:26 +00:00
table - > freezePartition ( command . partition , command . with_name , context ) ;
2017-04-01 07:20:54 +00:00
break ;
2017-06-22 11:01:30 +00:00
case PartitionCommand : : CLEAR_COLUMN :
2017-09-06 20:34:26 +00:00
table - > clearColumnInPartition ( command . partition , command . column_name , context ) ;
2017-04-14 12:40:48 +00:00
break ;
2017-04-01 07:20:54 +00:00
}
}
if ( alter_commands . empty ( ) )
return { } ;
alter_commands . validate ( table . get ( ) , context ) ;
table - > alter ( alter_commands , database_name , table_name , context ) ;
return { } ;
2013-12-18 11:19:37 +00:00
}
2014-08-06 10:26:35 +00:00
void InterpreterAlterQuery : : parseAlter (
2017-04-01 07:20:54 +00:00
const ASTAlterQuery : : ParameterContainer & params_container ,
AlterCommands & out_alter_commands , PartitionCommands & out_partition_commands )
2014-04-02 18:53:30 +00:00
{
2017-04-01 07:20:54 +00:00
const DataTypeFactory & data_type_factory = DataTypeFactory : : instance ( ) ;
for ( const auto & params : params_container )
{
if ( params . type = = ASTAlterQuery : : ADD_COLUMN )
{
AlterCommand command ;
command . type = AlterCommand : : ADD_COLUMN ;
const auto & ast_col_decl = typeid_cast < const ASTColumnDeclaration & > ( * params . col_decl ) ;
command . column_name = ast_col_decl . name ;
if ( ast_col_decl . type )
{
2018-02-26 03:37:08 +00:00
command . data_type = data_type_factory . get ( ast_col_decl . type ) ;
2017-04-01 07:20:54 +00:00
}
if ( ast_col_decl . default_expression )
{
2018-03-12 13:47:01 +00:00
command . default_kind = columnDefaultKindFromString ( ast_col_decl . default_specifier ) ;
2017-04-01 07:20:54 +00:00
command . default_expression = ast_col_decl . default_expression ;
}
if ( params . column )
command . after_column = typeid_cast < const ASTIdentifier & > ( * params . column ) . name ;
out_alter_commands . emplace_back ( std : : move ( command ) ) ;
}
else if ( params . type = = ASTAlterQuery : : DROP_COLUMN )
{
2017-04-14 12:40:48 +00:00
if ( params . partition )
{
2017-06-22 11:01:30 +00:00
if ( ! params . clear_column )
throw Exception ( " Can't DROP COLUMN from partition. It is possible only CLEAR COLUMN in partition " , ErrorCodes : : BAD_ARGUMENTS ) ;
2017-04-14 12:40:48 +00:00
const Field & column_name = typeid_cast < const ASTIdentifier & > ( * ( params . column ) ) . name ;
2017-04-01 07:20:54 +00:00
2017-09-06 20:34:26 +00:00
out_partition_commands . emplace_back ( PartitionCommand : : clearColumn ( params . partition , column_name ) ) ;
2017-04-14 12:40:48 +00:00
}
else
{
2017-06-22 11:01:30 +00:00
if ( params . clear_column )
throw Exception ( " \" ALTER TABLE table CLEAR COLUMN column \" queries are not supported yet. Use \" CLEAR COLUMN column IN PARTITION \" . " , ErrorCodes : : NOT_IMPLEMENTED ) ;
2017-04-14 12:40:48 +00:00
AlterCommand command ;
command . type = AlterCommand : : DROP_COLUMN ;
command . column_name = typeid_cast < const ASTIdentifier & > ( * ( params . column ) ) . name ;
out_alter_commands . emplace_back ( std : : move ( command ) ) ;
}
2017-04-01 07:20:54 +00:00
}
else if ( params . type = = ASTAlterQuery : : MODIFY_COLUMN )
{
AlterCommand command ;
command . type = AlterCommand : : MODIFY_COLUMN ;
const auto & ast_col_decl = typeid_cast < const ASTColumnDeclaration & > ( * params . col_decl ) ;
command . column_name = ast_col_decl . name ;
if ( ast_col_decl . type )
{
2018-02-26 03:37:08 +00:00
command . data_type = data_type_factory . get ( ast_col_decl . type ) ;
2017-04-01 07:20:54 +00:00
}
if ( ast_col_decl . default_expression )
{
2018-03-12 13:47:01 +00:00
command . default_kind = columnDefaultKindFromString ( ast_col_decl . default_specifier ) ;
2017-04-01 07:20:54 +00:00
command . default_expression = ast_col_decl . default_expression ;
}
out_alter_commands . emplace_back ( std : : move ( command ) ) ;
}
else if ( params . type = = ASTAlterQuery : : MODIFY_PRIMARY_KEY )
{
AlterCommand command ;
command . type = AlterCommand : : MODIFY_PRIMARY_KEY ;
command . primary_key = params . primary_key ;
out_alter_commands . emplace_back ( std : : move ( command ) ) ;
}
else if ( params . type = = ASTAlterQuery : : DROP_PARTITION )
{
2017-09-06 20:34:26 +00:00
out_partition_commands . emplace_back ( PartitionCommand : : dropPartition ( params . partition , params . detach ) ) ;
2017-04-01 07:20:54 +00:00
}
else if ( params . type = = ASTAlterQuery : : ATTACH_PARTITION )
{
2017-09-06 20:34:26 +00:00
out_partition_commands . emplace_back ( PartitionCommand : : attachPartition ( params . partition , params . part ) ) ;
2017-04-01 07:20:54 +00:00
}
else if ( params . type = = ASTAlterQuery : : FETCH_PARTITION )
{
2017-09-06 20:34:26 +00:00
out_partition_commands . emplace_back ( PartitionCommand : : fetchPartition ( params . partition , params . from ) ) ;
2017-04-01 07:20:54 +00:00
}
else if ( params . type = = ASTAlterQuery : : FREEZE_PARTITION )
{
2017-09-06 20:34:26 +00:00
out_partition_commands . emplace_back ( PartitionCommand : : freezePartition ( params . partition , params . with_name ) ) ;
2017-04-01 07:20:54 +00:00
}
else
throw Exception ( " Wrong parameter type in ALTER query " , ErrorCodes : : LOGICAL_ERROR ) ;
}
2014-05-20 15:00:13 +00:00
}
2017-07-14 18:09:28 +00:00
void InterpreterAlterQuery : : PartitionCommands : : validate ( const IStorage * table )
{
for ( const PartitionCommand & command : * this )
{
if ( command . type = = PartitionCommand : : CLEAR_COLUMN )
{
String column_name = command . column_name . safeGet < String > ( ) ;
2018-03-06 20:18:34 +00:00
if ( ! table - > columns . has ( column_name ) )
2017-07-14 18:09:28 +00:00
{
throw Exception ( " Wrong column name. Cannot find column " + column_name + " to clear it from partition " ,
DB : : ErrorCodes : : ILLEGAL_COLUMN ) ;
}
}
}
}
2016-01-11 21:46:36 +00:00
}