2017-04-01 09:19:00 +00:00
# include <Interpreters/InterpreterAlterQuery.h>
# include <Interpreters/InterpreterCreateQuery.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/ASTWeightedZooKeeperPath.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-05-24 21:38:56 +00:00
table - > dropPartition ( query_ptr , command . partition , command . detach , context . getSettingsRef ( ) ) ;
2017-04-01 07:20:54 +00:00
break ;
case PartitionCommand : : ATTACH_PARTITION :
2017-05-24 21:38:56 +00:00
table - > attachPartition ( query_ptr , command . partition , command . part , context . getSettingsRef ( ) ) ;
2017-04-01 07:20:54 +00:00
break ;
case PartitionCommand : : FETCH_PARTITION :
table - > fetchPartition ( command . partition , command . from , context . getSettingsRef ( ) ) ;
break ;
case PartitionCommand : : FREEZE_PARTITION :
table - > freezePartition ( command . partition , command . with_name , context . getSettingsRef ( ) ) ;
break ;
case PartitionCommand : : RESHARD_PARTITION :
2017-07-10 03:28:12 +00:00
table - > reshardPartitions ( query_ptr , database_name , command . partition ,
2017-04-01 07:20:54 +00:00
command . weighted_zookeeper_paths , command . sharding_key_expr , command . do_copy ,
2017-05-21 22:25:25 +00:00
command . coordinator , context ) ;
2017-04-01 07:20:54 +00:00
break ;
2017-06-22 11:01:30 +00:00
case PartitionCommand : : CLEAR_COLUMN :
2017-06-22 15:17:27 +00:00
table - > clearColumnInPartition ( query_ptr , command . partition , command . column_name , context . getSettingsRef ( ) ) ;
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 )
{
StringRange type_range = ast_col_decl . type - > range ;
String type_string ( type_range . first , type_range . second - type_range . first ) ;
command . data_type = data_type_factory . get ( type_string ) ;
}
if ( ast_col_decl . default_expression )
{
command . default_type = columnDefaultTypeFromString ( ast_col_decl . default_specifier ) ;
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 & partition = typeid_cast < const ASTLiteral & > ( * ( params . partition ) ) . value ;
const Field & column_name = typeid_cast < const ASTIdentifier & > ( * ( params . column ) ) . name ;
2017-04-01 07:20:54 +00:00
2017-06-22 11:01:30 +00:00
out_partition_commands . emplace_back ( PartitionCommand : : clearColumn ( 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 )
{
StringRange type_range = ast_col_decl . type - > range ;
String type_string ( type_range . first , type_range . second - type_range . first ) ;
command . data_type = data_type_factory . get ( type_string ) ;
}
if ( ast_col_decl . default_expression )
{
command . default_type = columnDefaultTypeFromString ( ast_col_decl . default_specifier ) ;
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 )
{
const Field & partition = dynamic_cast < const ASTLiteral & > ( * params . partition ) . value ;
2017-05-24 21:38:56 +00:00
out_partition_commands . emplace_back ( PartitionCommand : : dropPartition ( partition , params . detach ) ) ;
2017-04-01 07:20:54 +00:00
}
else if ( params . type = = ASTAlterQuery : : ATTACH_PARTITION )
{
const Field & partition = dynamic_cast < const ASTLiteral & > ( * params . partition ) . value ;
2017-05-24 21:38:56 +00:00
out_partition_commands . emplace_back ( PartitionCommand : : attachPartition ( partition , params . part ) ) ;
2017-04-01 07:20:54 +00:00
}
else if ( params . type = = ASTAlterQuery : : FETCH_PARTITION )
{
const Field & partition = dynamic_cast < const ASTLiteral & > ( * params . partition ) . value ;
out_partition_commands . emplace_back ( PartitionCommand : : fetchPartition ( partition , params . from ) ) ;
}
else if ( params . type = = ASTAlterQuery : : FREEZE_PARTITION )
{
const Field & partition = dynamic_cast < const ASTLiteral & > ( * params . partition ) . value ;
out_partition_commands . emplace_back ( PartitionCommand : : freezePartition ( partition , params . with_name ) ) ;
}
else if ( params . type = = ASTAlterQuery : : RESHARD_PARTITION )
{
2017-07-10 03:28:12 +00:00
Field partition ;
2017-04-01 07:20:54 +00:00
if ( params . partition )
2017-07-10 03:28:12 +00:00
partition = dynamic_cast < const ASTLiteral & > ( * params . partition ) . value ;
2017-04-01 07:20:54 +00:00
WeightedZooKeeperPaths weighted_zookeeper_paths ;
const ASTs & ast_weighted_zookeeper_paths = typeid_cast < const ASTExpressionList & > ( * params . weighted_zookeeper_paths ) . children ;
for ( size_t i = 0 ; i < ast_weighted_zookeeper_paths . size ( ) ; + + i )
{
const auto & weighted_zookeeper_path = typeid_cast < const ASTWeightedZooKeeperPath & > ( * ast_weighted_zookeeper_paths [ i ] ) ;
weighted_zookeeper_paths . emplace_back ( weighted_zookeeper_path . path , weighted_zookeeper_path . weight ) ;
}
Field coordinator ;
if ( params . coordinator )
coordinator = dynamic_cast < const ASTLiteral & > ( * params . coordinator ) . value ;
out_partition_commands . emplace_back ( PartitionCommand : : reshardPartitions (
2017-07-10 03:28:12 +00:00
partition , weighted_zookeeper_paths , params . sharding_key_expr ,
2017-04-01 07:20:54 +00:00
params . do_copy , coordinator ) ) ;
}
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 > ( ) ;
2017-07-14 21:51:52 +00:00
if ( ! table - > hasRealColumn ( 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
}