2019-06-24 11:17:15 +00:00
# include "config_core.h"
2019-10-11 17:27:54 +00:00
# include <Interpreters/Set.h>
2017-04-01 09:19:00 +00:00
# include <Common/ProfileEvents.h>
2018-08-30 16:31:20 +00:00
# include <Common/SipHash.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/ExpressionActions.h>
Add a JIT interface for row-wise default-nullable functions.
Not actually implemented, though. It does print out some jit-compiled stuff,
but that's about it. For example, this query:
select number from system.numbers where something(cast(number as Float64)) == 4
results in this on server's stderr:
define double @"something(CAST(number, 'Float64'))"(void**, i8*, void*) {
"something(CAST(number, 'Float64'))":
ret double 1.234500e+04
}
(and an exception, because that's what the non-jitted method does.)
As one may notice, this function neither reads the input (first argument;
tuple of arrays) nor writes the output (third argument; array), instead
returning some general nonsense.
In addition, `#if USE_EMBEDDED_COMPILER` doesn't work for some reason,
including LLVM headers requires -Wno-unused-parameter, this probably only
works on LLVM 5.0 due to rampant API instability, and I'm definitely
no expert on CMake. In short, there's still a long way to go.
2018-04-23 22:29:39 +00:00
# include <Interpreters/ExpressionJIT.h>
2019-09-03 14:36:02 +00:00
# include <Interpreters/AnalyzedJoin.h>
2017-04-01 09:19:00 +00:00
# include <Columns/ColumnsNumber.h>
2017-07-13 20:58:19 +00:00
# include <Common/typeid_cast.h>
2017-04-01 09:19:00 +00:00
# include <DataTypes/DataTypeArray.h>
2019-09-03 14:36:02 +00:00
# include <DataTypes/DataTypesNumber.h>
2017-04-01 09:19:00 +00:00
# include <Functions/FunctionFactory.h>
# include <Functions/IFunction.h>
2017-12-09 06:32:22 +00:00
# include <optional>
2019-10-11 17:27:54 +00:00
# include <Columns/ColumnSet.h>
2019-10-27 18:12:40 +00:00
# include <Functions/FunctionHelpers.h>
2013-05-24 10:49:19 +00:00
2014-01-04 04:53:07 +00:00
2016-10-24 02:02:37 +00:00
namespace ProfileEvents
{
2017-04-01 07:20:54 +00:00
extern const Event FunctionExecute ;
2018-08-29 20:05:31 +00:00
extern const Event CompiledFunctionExecute ;
2016-10-24 02:02:37 +00:00
}
2013-05-24 10:49:19 +00:00
namespace DB
{
2014-06-12 18:41:09 +00:00
2016-01-11 21:46:36 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int LOGICAL_ERROR ;
2017-04-01 07:20:54 +00:00
extern const int DUPLICATE_COLUMN ;
extern const int UNKNOWN_IDENTIFIER ;
extern const int NOT_FOUND_COLUMN_IN_BLOCK ;
2018-03-09 23:23:15 +00:00
extern const int TOO_MANY_TEMPORARY_COLUMNS ;
extern const int TOO_MANY_TEMPORARY_NON_CONST_COLUMNS ;
2016-01-11 21:46:36 +00:00
}
2019-07-31 10:22:56 +00:00
/// Read comment near usage
static constexpr auto DUMMY_COLUMN_NAME = " _dummy " ;
2016-01-07 19:52:21 +00:00
2020-02-05 10:12:19 +00:00
2014-06-12 18:41:09 +00:00
Names ExpressionAction : : getNeededColumns ( ) const
2013-06-10 16:03:23 +00:00
{
2017-04-01 07:20:54 +00:00
Names res = argument_names ;
2014-06-12 18:41:09 +00:00
2020-02-05 10:12:19 +00:00
if ( array_join )
res . insert ( res . end ( ) , array_join - > columns . begin ( ) , array_join - > columns . end ( ) ) ;
2014-06-12 18:41:09 +00:00
2019-09-03 14:36:02 +00:00
if ( table_join )
res . insert ( res . end ( ) , table_join - > keyNamesLeft ( ) . begin ( ) , table_join - > keyNamesLeft ( ) . end ( ) ) ;
2018-07-24 16:43:33 +00:00
2017-04-01 07:20:54 +00:00
for ( const auto & column : projection )
res . push_back ( column . first ) ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
if ( ! source_name . empty ( ) )
res . push_back ( source_name ) ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
return res ;
2013-06-10 16:03:23 +00:00
}
2013-08-01 13:29:32 +00:00
2017-01-14 09:00:19 +00:00
2018-09-10 03:59:48 +00:00
ExpressionAction ExpressionAction : : applyFunction (
2019-12-08 21:06:37 +00:00
const FunctionOverloadResolverPtr & function_ ,
2017-04-01 07:20:54 +00:00
const std : : vector < std : : string > & argument_names_ ,
2018-11-26 01:39:04 +00:00
std : : string result_name_ )
2013-06-04 14:59:05 +00:00
{
2020-03-08 21:53:03 +00:00
if ( result_name_ . empty ( ) )
2017-04-01 07:20:54 +00:00
{
result_name_ = function_ - > getName ( ) + " ( " ;
for ( size_t i = 0 ; i < argument_names_ . size ( ) ; + + i )
{
if ( i )
result_name_ + = " , " ;
result_name_ + = argument_names_ [ i ] ;
}
result_name_ + = " ) " ;
}
ExpressionAction a ;
a . type = APPLY_FUNCTION ;
a . result_name = result_name_ ;
2018-02-02 08:33:36 +00:00
a . function_builder = function_ ;
2017-04-01 07:20:54 +00:00
a . argument_names = argument_names_ ;
return a ;
2013-06-04 14:59:05 +00:00
}
2013-05-24 10:49:19 +00:00
2018-09-10 03:59:48 +00:00
ExpressionAction ExpressionAction : : addColumn (
2018-11-26 01:39:04 +00:00
const ColumnWithTypeAndName & added_column_ )
2017-01-14 09:00:19 +00:00
{
2017-04-01 07:20:54 +00:00
ExpressionAction a ;
a . type = ADD_COLUMN ;
a . result_name = added_column_ . name ;
a . result_type = added_column_ . type ;
a . added_column = added_column_ . column ;
return a ;
2017-01-14 09:00:19 +00:00
}
ExpressionAction ExpressionAction : : removeColumn ( const std : : string & removed_name )
{
2017-04-01 07:20:54 +00:00
ExpressionAction a ;
a . type = REMOVE_COLUMN ;
a . source_name = removed_name ;
return a ;
2017-01-14 09:00:19 +00:00
}
2018-08-09 19:17:55 +00:00
ExpressionAction ExpressionAction : : copyColumn ( const std : : string & from_name , const std : : string & to_name , bool can_replace )
2017-01-14 09:00:19 +00:00
{
2017-04-01 07:20:54 +00:00
ExpressionAction a ;
a . type = COPY_COLUMN ;
a . source_name = from_name ;
a . result_name = to_name ;
2018-08-09 19:17:55 +00:00
a . can_replace = can_replace ;
2017-04-01 07:20:54 +00:00
return a ;
2017-01-14 09:00:19 +00:00
}
ExpressionAction ExpressionAction : : project ( const NamesWithAliases & projected_columns_ )
{
2017-04-01 07:20:54 +00:00
ExpressionAction a ;
a . type = PROJECT ;
a . projection = projected_columns_ ;
return a ;
2017-01-14 09:00:19 +00:00
}
ExpressionAction ExpressionAction : : project ( const Names & projected_columns_ )
{
2017-04-01 07:20:54 +00:00
ExpressionAction a ;
a . type = PROJECT ;
a . projection . resize ( projected_columns_ . size ( ) ) ;
for ( size_t i = 0 ; i < projected_columns_ . size ( ) ; + + i )
a . projection [ i ] = NameWithAlias ( projected_columns_ [ i ] , " " ) ;
return a ;
2017-01-14 09:00:19 +00:00
}
2018-09-03 17:24:46 +00:00
ExpressionAction ExpressionAction : : addAliases ( const NamesWithAliases & aliased_columns_ )
{
ExpressionAction a ;
a . type = ADD_ALIASES ;
a . projection = aliased_columns_ ;
return a ;
}
2017-01-14 09:00:19 +00:00
ExpressionAction ExpressionAction : : arrayJoin ( const NameSet & array_joined_columns , bool array_join_is_left , const Context & context )
{
2017-04-01 07:20:54 +00:00
ExpressionAction a ;
a . type = ARRAY_JOIN ;
2020-02-05 10:12:19 +00:00
a . array_join = std : : make_shared < ArrayJoinAction > ( array_joined_columns , array_join_is_left , context ) ;
2017-04-01 07:20:54 +00:00
return a ;
2017-01-14 09:00:19 +00:00
}
2019-09-16 12:37:46 +00:00
ExpressionAction ExpressionAction : : ordinaryJoin ( std : : shared_ptr < AnalyzedJoin > table_join , JoinPtr join )
2017-01-14 09:00:19 +00:00
{
2017-04-01 07:20:54 +00:00
ExpressionAction a ;
a . type = JOIN ;
2019-09-03 14:36:02 +00:00
a . table_join = table_join ;
2019-09-16 12:37:46 +00:00
a . join = join ;
2017-04-01 07:20:54 +00:00
return a ;
2017-01-14 09:00:19 +00:00
}
2019-08-16 10:29:33 +00:00
void ExpressionAction : : prepare ( Block & sample_block , const Settings & settings , NameSet & names_not_for_constant_folding )
2013-06-04 13:34:46 +00:00
{
2018-07-19 13:36:21 +00:00
// std::cerr << "preparing: " << toString() << std::endl;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/** Constant expressions should be evaluated, and put the result in sample_block.
2017-04-01 07:20:54 +00:00
*/
switch ( type )
{
case APPLY_FUNCTION :
{
if ( sample_block . has ( result_name ) )
throw Exception ( " Column ' " + result_name + " ' already exists " , ErrorCodes : : DUPLICATE_COLUMN ) ;
bool all_const = true ;
2019-08-16 10:29:33 +00:00
bool all_suitable_for_constant_folding = true ;
2017-04-01 07:20:54 +00:00
ColumnNumbers arguments ( argument_names . size ( ) ) ;
for ( size_t i = 0 ; i < argument_names . size ( ) ; + + i )
{
arguments [ i ] = sample_block . getPositionByName ( argument_names [ i ] ) ;
ColumnPtr col = sample_block . safeGetByPosition ( arguments [ i ] ) . column ;
2019-06-27 19:28:52 +00:00
if ( ! col | | ! isColumnConst ( * col ) )
2017-04-01 07:20:54 +00:00
all_const = false ;
2019-08-16 10:29:33 +00:00
if ( names_not_for_constant_folding . count ( argument_names [ i ] ) )
all_suitable_for_constant_folding = false ;
2017-04-01 07:20:54 +00:00
}
2018-09-21 14:15:21 +00:00
size_t result_position = sample_block . columns ( ) ;
2019-08-16 10:41:17 +00:00
sample_block . insert ( { nullptr , result_type , result_name } ) ;
2018-09-21 14:15:21 +00:00
function = function_base - > prepare ( sample_block , arguments , result_position ) ;
2019-12-09 14:41:55 +00:00
function - > createLowCardinalityResultCache ( settings . max_threads ) ;
2017-04-01 07:20:54 +00:00
2018-10-24 11:31:23 +00:00
bool compile_expressions = false ;
# if USE_EMBEDDED_COMPILER
compile_expressions = settings . compile_expressions ;
# endif
2018-09-21 14:15:21 +00:00
/// If all arguments are constants, and function is suitable to be executed in 'prepare' stage - execute function.
2018-10-24 11:31:23 +00:00
/// But if we compile expressions compiled version of this function maybe placed in cache,
/// so we don't want to unfold non deterministic functions
2019-08-16 16:56:34 +00:00
if ( all_const & & function_base - > isSuitableForConstantFolding ( ) & & ( ! compile_expressions | | function_base - > isDeterministic ( ) ) )
2018-09-21 14:15:21 +00:00
{
2018-12-02 11:00:23 +00:00
function - > execute ( sample_block , arguments , result_position , sample_block . rows ( ) , true ) ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// If the result is not a constant, just in case, we will consider the result as unknown.
2017-04-01 07:20:54 +00:00
ColumnWithTypeAndName & col = sample_block . safeGetByPosition ( result_position ) ;
2019-06-27 19:28:52 +00:00
if ( ! isColumnConst ( * col . column ) )
2017-12-17 10:14:12 +00:00
{
2017-04-01 07:20:54 +00:00
col . column = nullptr ;
2017-12-17 10:14:12 +00:00
}
else
{
/// All constant (literal) columns in block are added with size 1.
/// But if there was no columns in block before executing a function, the result has size 0.
/// Change the size to 1.
if ( col . column - > empty ( ) )
col . column = col . column - > cloneResized ( 1 ) ;
2019-08-15 19:31:43 +00:00
2019-08-16 16:56:34 +00:00
if ( ! all_suitable_for_constant_folding )
2019-08-16 10:29:33 +00:00
names_not_for_constant_folding . insert ( result_name ) ;
2017-12-17 10:14:12 +00:00
}
2017-04-01 07:20:54 +00:00
}
2019-08-19 18:10:40 +00:00
/// Some functions like ignore() or getTypeName() always return constant result even if arguments are not constant.
/// We can't do constant folding, but can specify in sample block that function result is constant to avoid
/// unnecessary materialization.
2019-08-16 10:41:17 +00:00
auto & res = sample_block . getByPosition ( result_position ) ;
2019-08-19 17:48:19 +00:00
if ( ! res . column & & function_base - > isSuitableForConstantFolding ( ) )
2019-08-16 10:41:17 +00:00
{
2019-08-19 17:48:19 +00:00
if ( auto col = function_base - > getResultIfAlwaysReturnsConstantAndHasArguments ( sample_block , arguments ) )
{
res . column = std : : move ( col ) ;
names_not_for_constant_folding . insert ( result_name ) ;
}
2019-08-16 10:41:17 +00:00
}
2017-04-01 07:20:54 +00:00
break ;
}
case ARRAY_JOIN :
{
2020-02-05 10:12:19 +00:00
array_join - > prepare ( sample_block ) ;
2017-04-01 07:20:54 +00:00
break ;
}
case JOIN :
{
2019-09-03 14:36:02 +00:00
table_join - > addJoinedColumnsAndCorrectNullability ( sample_block ) ;
2017-04-01 07:20:54 +00:00
break ;
}
case PROJECT :
{
Block new_block ;
2020-03-08 23:48:08 +00:00
for ( const auto & elem : projection )
2017-04-01 07:20:54 +00:00
{
2020-03-08 23:48:08 +00:00
const std : : string & name = elem . first ;
const std : : string & alias = elem . second ;
2017-04-01 07:20:54 +00:00
ColumnWithTypeAndName column = sample_block . getByName ( name ) ;
2020-03-08 23:48:08 +00:00
if ( ! alias . empty ( ) )
2017-04-01 07:20:54 +00:00
column . name = alias ;
new_block . insert ( std : : move ( column ) ) ;
}
sample_block . swap ( new_block ) ;
break ;
}
2018-09-03 17:24:46 +00:00
case ADD_ALIASES :
{
2020-03-08 23:48:08 +00:00
for ( const auto & elem : projection )
2018-09-03 17:24:46 +00:00
{
2020-03-08 23:48:08 +00:00
const std : : string & name = elem . first ;
const std : : string & alias = elem . second ;
2018-09-03 17:24:46 +00:00
const ColumnWithTypeAndName & column = sample_block . getByName ( name ) ;
2020-03-08 23:48:08 +00:00
if ( ! alias . empty ( ) & & ! sample_block . has ( alias ) )
2018-09-03 17:24:46 +00:00
sample_block . insert ( { column . column , column . type , alias } ) ;
}
break ;
}
2017-04-01 07:20:54 +00:00
case REMOVE_COLUMN :
{
sample_block . erase ( source_name ) ;
break ;
}
case ADD_COLUMN :
{
if ( sample_block . has ( result_name ) )
throw Exception ( " Column ' " + result_name + " ' already exists " , ErrorCodes : : DUPLICATE_COLUMN ) ;
sample_block . insert ( ColumnWithTypeAndName ( added_column , result_type , result_name ) ) ;
break ;
}
case COPY_COLUMN :
{
2018-08-09 19:17:55 +00:00
const auto & source = sample_block . getByName ( source_name ) ;
result_type = source . type ;
if ( sample_block . has ( result_name ) )
{
if ( can_replace )
{
auto & result = sample_block . getByName ( result_name ) ;
result . type = result_type ;
result . column = source . column ;
}
else
throw Exception ( " Column ' " + result_name + " ' already exists " , ErrorCodes : : DUPLICATE_COLUMN ) ;
}
else
sample_block . insert ( ColumnWithTypeAndName ( source . column , result_type , result_name ) ) ;
2017-04-01 07:20:54 +00:00
break ;
}
}
2013-05-24 10:49:19 +00:00
}
2018-04-30 19:33:32 +00:00
2020-01-15 20:33:29 +00:00
void ExpressionAction : : execute ( Block & block , bool dry_run , ExtraBlockPtr & not_processed ) const
2013-05-24 10:49:19 +00:00
{
2018-11-26 01:39:04 +00:00
size_t input_rows_count = block . rows ( ) ;
2018-04-24 07:16:39 +00:00
2017-04-01 07:20:54 +00:00
if ( type = = REMOVE_COLUMN | | type = = COPY_COLUMN )
if ( ! block . has ( source_name ) )
throw Exception ( " Not found column ' " + source_name + " '. There are columns: " + block . dumpNames ( ) , ErrorCodes : : NOT_FOUND_COLUMN_IN_BLOCK ) ;
2018-08-09 19:17:55 +00:00
if ( type = = ADD_COLUMN | | ( type = = COPY_COLUMN & & ! can_replace ) | | type = = APPLY_FUNCTION )
2017-04-01 07:20:54 +00:00
if ( block . has ( result_name ) )
throw Exception ( " Column ' " + result_name + " ' already exists " , ErrorCodes : : DUPLICATE_COLUMN ) ;
switch ( type )
{
case APPLY_FUNCTION :
{
ColumnNumbers arguments ( argument_names . size ( ) ) ;
for ( size_t i = 0 ; i < argument_names . size ( ) ; + + i )
{
if ( ! block . has ( argument_names [ i ] ) )
throw Exception ( " Not found column: ' " + argument_names [ i ] + " ' " , ErrorCodes : : NOT_FOUND_COLUMN_IN_BLOCK ) ;
arguments [ i ] = block . getPositionByName ( argument_names [ i ] ) ;
}
size_t num_columns_without_result = block . columns ( ) ;
block . insert ( { nullptr , result_type , result_name } ) ;
ProfileEvents : : increment ( ProfileEvents : : FunctionExecute ) ;
2018-08-29 20:05:31 +00:00
if ( is_function_compiled )
ProfileEvents : : increment ( ProfileEvents : : CompiledFunctionExecute ) ;
2018-12-02 11:00:23 +00:00
function - > execute ( block , arguments , num_columns_without_result , input_rows_count , dry_run ) ;
2017-04-01 07:20:54 +00:00
break ;
}
case ARRAY_JOIN :
{
2020-02-05 10:12:19 +00:00
array_join - > execute ( block , dry_run ) ;
2017-04-01 07:20:54 +00:00
break ;
}
case JOIN :
{
2020-01-13 18:00:32 +00:00
join - > joinBlock ( block , not_processed ) ;
2020-01-15 20:33:29 +00:00
break ;
2017-04-01 07:20:54 +00:00
}
case PROJECT :
{
Block new_block ;
2020-03-08 23:48:08 +00:00
for ( const auto & elem : projection )
2017-04-01 07:20:54 +00:00
{
2020-03-08 23:48:08 +00:00
const std : : string & name = elem . first ;
const std : : string & alias = elem . second ;
2017-04-01 07:20:54 +00:00
ColumnWithTypeAndName column = block . getByName ( name ) ;
2020-03-08 23:48:08 +00:00
if ( ! alias . empty ( ) )
2017-04-01 07:20:54 +00:00
column . name = alias ;
new_block . insert ( std : : move ( column ) ) ;
}
block . swap ( new_block ) ;
break ;
}
2018-09-03 17:24:46 +00:00
case ADD_ALIASES :
{
2020-03-08 23:48:08 +00:00
for ( const auto & elem : projection )
2018-09-03 17:24:46 +00:00
{
2020-03-08 23:48:08 +00:00
const std : : string & name = elem . first ;
const std : : string & alias = elem . second ;
2018-09-03 17:24:46 +00:00
const ColumnWithTypeAndName & column = block . getByName ( name ) ;
2020-03-08 23:48:08 +00:00
if ( ! alias . empty ( ) & & ! block . has ( alias ) )
2018-09-03 17:24:46 +00:00
block . insert ( { column . column , column . type , alias } ) ;
}
break ;
}
2017-04-01 07:20:54 +00:00
case REMOVE_COLUMN :
block . erase ( source_name ) ;
break ;
case ADD_COLUMN :
2018-04-24 07:16:39 +00:00
block . insert ( { added_column - > cloneResized ( input_rows_count ) , result_type , result_name } ) ;
2017-04-01 07:20:54 +00:00
break ;
case COPY_COLUMN :
2018-08-09 19:17:55 +00:00
if ( can_replace & & block . has ( result_name ) )
{
auto & result = block . getByName ( result_name ) ;
2019-07-31 15:05:21 +00:00
const auto & source = block . getByName ( source_name ) ;
result . type = source . type ;
result . column = source . column ;
2018-08-09 19:17:55 +00:00
}
else
2019-07-31 12:33:58 +00:00
{
const auto & source_column = block . getByName ( source_name ) ;
block . insert ( { source_column . column , source_column . type , result_name } ) ;
}
2018-08-09 19:17:55 +00:00
2017-04-01 07:20:54 +00:00
break ;
}
2013-05-24 10:49:19 +00:00
}
2015-04-16 09:55:24 +00:00
void ExpressionAction : : executeOnTotals ( Block & block ) const
{
2017-04-01 07:20:54 +00:00
if ( type ! = JOIN )
2018-12-02 11:00:23 +00:00
execute ( block , false ) ;
2017-04-01 07:20:54 +00:00
else
2019-09-16 12:37:46 +00:00
join - > joinTotals ( block ) ;
2015-04-16 09:55:24 +00:00
}
2014-06-12 18:41:09 +00:00
std : : string ExpressionAction : : toString ( ) const
2013-05-24 10:49:19 +00:00
{
2017-04-01 07:20:54 +00:00
std : : stringstream ss ;
switch ( type )
{
case ADD_COLUMN :
ss < < " ADD " < < result_name < < " "
< < ( result_type ? result_type - > getName ( ) : " (no type) " ) < < " "
< < ( added_column ? added_column - > getName ( ) : " (no column) " ) ;
break ;
case REMOVE_COLUMN :
ss < < " REMOVE " < < source_name ;
break ;
case COPY_COLUMN :
ss < < " COPY " < < result_name < < " = " < < source_name ;
2018-08-09 19:17:55 +00:00
if ( can_replace )
ss < < " (can replace) " ;
2017-04-01 07:20:54 +00:00
break ;
case APPLY_FUNCTION :
2018-08-29 20:05:31 +00:00
ss < < " FUNCTION " < < result_name < < " " < < ( is_function_compiled ? " [compiled] " : " " )
2017-04-01 07:20:54 +00:00
< < ( result_type ? result_type - > getName ( ) : " (no type) " ) < < " = "
2018-09-21 14:15:21 +00:00
< < ( function_base ? function_base - > getName ( ) : " (no function) " ) < < " ( " ;
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < argument_names . size ( ) ; + + i )
{
if ( i )
ss < < " , " ;
ss < < argument_names [ i ] ;
}
ss < < " ) " ;
break ;
case ARRAY_JOIN :
2020-02-05 10:12:19 +00:00
ss < < ( array_join - > is_left ? " LEFT " : " " ) < < " ARRAY JOIN " ;
for ( NameSet : : const_iterator it = array_join - > columns . begin ( ) ; it ! = array_join - > columns . end ( ) ; + + it )
2017-04-01 07:20:54 +00:00
{
2020-02-05 10:12:19 +00:00
if ( it ! = array_join - > columns . begin ( ) )
2017-04-01 07:20:54 +00:00
ss < < " , " ;
ss < < * it ;
}
break ;
case JOIN :
ss < < " JOIN " ;
2019-09-03 14:36:02 +00:00
for ( NamesAndTypesList : : const_iterator it = table_join - > columnsAddedByJoin ( ) . begin ( ) ;
it ! = table_join - > columnsAddedByJoin ( ) . end ( ) ; + + it )
2017-04-01 07:20:54 +00:00
{
2019-09-03 14:36:02 +00:00
if ( it ! = table_join - > columnsAddedByJoin ( ) . begin ( ) )
2017-04-01 07:20:54 +00:00
ss < < " , " ;
ss < < it - > name ;
}
break ;
2018-09-04 19:48:05 +00:00
case PROJECT : [[fallthrough]] ;
2018-09-03 17:24:46 +00:00
case ADD_ALIASES :
ss < < ( type = = PROJECT ? " PROJECT " : " ADD_ALIASES " ) ;
2017-04-01 07:20:54 +00:00
for ( size_t i = 0 ; i < projection . size ( ) ; + + i )
{
if ( i )
ss < < " , " ;
ss < < projection [ i ] . first ;
2020-03-08 23:48:08 +00:00
if ( ! projection [ i ] . second . empty ( ) & & projection [ i ] . second ! = projection [ i ] . first )
2017-04-01 07:20:54 +00:00
ss < < " AS " < < projection [ i ] . second ;
}
break ;
}
return ss . str ( ) ;
2013-05-24 10:49:19 +00:00
}
2013-06-05 10:34:59 +00:00
void ExpressionActions : : checkLimits ( Block & block ) const
2013-05-28 12:19:25 +00:00
{
2018-03-11 00:15:26 +00:00
if ( settings . max_temporary_columns & & block . columns ( ) > settings . max_temporary_columns )
2017-04-01 07:20:54 +00:00
throw Exception ( " Too many temporary columns: " + block . dumpNames ( )
2018-03-11 00:15:26 +00:00
+ " . Maximum: " + settings . max_temporary_columns . toString ( ) ,
2018-03-09 23:23:15 +00:00
ErrorCodes : : TOO_MANY_TEMPORARY_COLUMNS ) ;
2017-04-01 07:20:54 +00:00
2018-03-11 00:15:26 +00:00
if ( settings . max_temporary_non_const_columns )
2017-04-01 07:20:54 +00:00
{
size_t non_const_columns = 0 ;
for ( size_t i = 0 , size = block . columns ( ) ; i < size ; + + i )
2019-06-27 19:28:52 +00:00
if ( block . safeGetByPosition ( i ) . column & & ! isColumnConst ( * block . safeGetByPosition ( i ) . column ) )
2017-04-01 07:20:54 +00:00
+ + non_const_columns ;
2018-03-11 00:15:26 +00:00
if ( non_const_columns > settings . max_temporary_non_const_columns )
2017-04-01 07:20:54 +00:00
{
std : : stringstream list_of_non_const_columns ;
for ( size_t i = 0 , size = block . columns ( ) ; i < size ; + + i )
2019-06-27 19:28:52 +00:00
if ( block . safeGetByPosition ( i ) . column & & ! isColumnConst ( * block . safeGetByPosition ( i ) . column ) )
2017-04-01 07:20:54 +00:00
list_of_non_const_columns < < " \n " < < block . safeGetByPosition ( i ) . name ;
throw Exception ( " Too many temporary non-const columns: " + list_of_non_const_columns . str ( )
2018-03-11 00:15:26 +00:00
+ " . Maximum: " + settings . max_temporary_non_const_columns . toString ( ) ,
2018-03-09 23:23:15 +00:00
ErrorCodes : : TOO_MANY_TEMPORARY_NON_CONST_COLUMNS ) ;
2017-04-01 07:20:54 +00:00
}
}
2013-05-28 12:19:25 +00:00
}
2015-07-17 01:27:35 +00:00
void ExpressionActions : : addInput ( const ColumnWithTypeAndName & column )
2013-06-11 16:21:25 +00:00
{
2017-04-01 07:20:54 +00:00
input_columns . emplace_back ( column . name , column . type ) ;
sample_block . insert ( column ) ;
2013-06-11 16:21:25 +00:00
}
2017-12-25 21:57:29 +00:00
void ExpressionActions : : addInput ( const NameAndTypePair & column )
2013-06-11 16:21:25 +00:00
{
2017-04-01 07:20:54 +00:00
addInput ( ColumnWithTypeAndName ( nullptr , column . type , column . name ) ) ;
2013-06-11 16:21:25 +00:00
}
2014-06-12 18:41:09 +00:00
void ExpressionActions : : add ( const ExpressionAction & action , Names & out_new_columns )
2013-06-11 16:21:25 +00:00
{
2018-02-19 20:23:25 +00:00
addImpl ( action , out_new_columns ) ;
2013-06-11 16:21:25 +00:00
}
2014-06-12 18:41:09 +00:00
void ExpressionActions : : add ( const ExpressionAction & action )
2013-06-04 13:34:46 +00:00
{
2017-04-01 07:20:54 +00:00
Names new_names ;
2018-02-19 20:23:25 +00:00
addImpl ( action , new_names ) ;
2013-06-04 13:34:46 +00:00
}
2018-02-19 20:23:25 +00:00
void ExpressionActions : : addImpl ( ExpressionAction action , Names & new_names )
2013-05-28 12:19:25 +00:00
{
2020-03-08 23:48:08 +00:00
if ( ! action . result_name . empty ( ) )
2017-04-01 07:20:54 +00:00
new_names . push_back ( action . result_name ) ;
2020-02-05 10:12:19 +00:00
if ( action . array_join )
new_names . insert ( new_names . end ( ) , action . array_join - > columns . begin ( ) , action . array_join - > columns . end ( ) ) ;
2014-06-12 18:41:09 +00:00
2019-08-28 18:23:20 +00:00
/// Compiled functions are custom functions and they don't need building
2018-10-25 16:52:28 +00:00
if ( action . type = = ExpressionAction : : APPLY_FUNCTION & & ! action . is_function_compiled )
2018-02-19 20:23:25 +00:00
{
if ( sample_block . has ( action . result_name ) )
throw Exception ( " Column ' " + action . result_name + " ' already exists " , ErrorCodes : : DUPLICATE_COLUMN ) ;
2014-06-12 18:41:09 +00:00
2018-02-19 20:23:25 +00:00
ColumnsWithTypeAndName arguments ( action . argument_names . size ( ) ) ;
for ( size_t i = 0 ; i < action . argument_names . size ( ) ; + + i )
{
if ( ! sample_block . has ( action . argument_names [ i ] ) )
throw Exception ( " Unknown identifier: ' " + action . argument_names [ i ] + " ' " , ErrorCodes : : UNKNOWN_IDENTIFIER ) ;
arguments [ i ] = sample_block . getByName ( action . argument_names [ i ] ) ;
}
2014-06-12 18:41:09 +00:00
2018-09-21 14:15:21 +00:00
action . function_base = action . function_builder - > build ( arguments ) ;
action . result_type = action . function_base - > getReturnType ( ) ;
2018-02-19 20:23:25 +00:00
}
2014-06-12 18:41:09 +00:00
2018-11-12 18:19:16 +00:00
if ( action . type = = ExpressionAction : : ADD_ALIASES )
for ( const auto & name_with_alias : action . projection )
new_names . emplace_back ( name_with_alias . second ) ;
2019-08-16 10:29:33 +00:00
action . prepare ( sample_block , settings , names_not_for_constant_folding ) ;
2017-04-01 07:20:54 +00:00
actions . push_back ( action ) ;
2013-05-28 12:19:25 +00:00
}
2013-05-28 14:47:37 +00:00
void ExpressionActions : : prependProjectInput ( )
{
2017-04-01 07:20:54 +00:00
actions . insert ( actions . begin ( ) , ExpressionAction : : project ( getRequiredColumns ( ) ) ) ;
2013-05-28 14:47:37 +00:00
}
2018-08-27 17:42:13 +00:00
void ExpressionActions : : prependArrayJoin ( const ExpressionAction & action , const Block & sample_block_before )
2014-04-30 19:19:29 +00:00
{
2017-04-01 07:20:54 +00:00
if ( action . type ! = ExpressionAction : : ARRAY_JOIN )
throw Exception ( " ARRAY_JOIN action expected " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-02-05 10:12:19 +00:00
NameSet array_join_set ( action . array_join - > columns . begin ( ) , action . array_join - > columns . end ( ) ) ;
2017-04-01 07:20:54 +00:00
for ( auto & it : input_columns )
{
if ( array_join_set . count ( it . name ) )
{
array_join_set . erase ( it . name ) ;
it . type = std : : make_shared < DataTypeArray > ( it . type ) ;
}
}
for ( const std : : string & name : array_join_set )
{
2018-08-27 17:42:13 +00:00
input_columns . emplace_back ( name , sample_block_before . getByName ( name ) . type ) ;
2017-04-01 07:20:54 +00:00
actions . insert ( actions . begin ( ) , ExpressionAction : : removeColumn ( name ) ) ;
}
actions . insert ( actions . begin ( ) , action ) ;
optimizeArrayJoin ( ) ;
2014-04-30 19:19:29 +00:00
}
2014-06-12 18:41:09 +00:00
bool ExpressionActions : : popUnusedArrayJoin ( const Names & required_columns , ExpressionAction & out_action )
2014-04-30 19:19:29 +00:00
{
2017-04-01 07:20:54 +00:00
if ( actions . empty ( ) | | actions . back ( ) . type ! = ExpressionAction : : ARRAY_JOIN )
return false ;
NameSet required_set ( required_columns . begin ( ) , required_columns . end ( ) ) ;
2020-02-05 10:12:19 +00:00
for ( const std : : string & name : actions . back ( ) . array_join - > columns )
2017-04-01 07:20:54 +00:00
{
if ( required_set . count ( name ) )
return false ;
}
2020-02-05 10:12:19 +00:00
for ( const std : : string & name : actions . back ( ) . array_join - > columns )
2017-04-01 07:20:54 +00:00
{
DataTypePtr & type = sample_block . getByName ( name ) . type ;
type = std : : make_shared < DataTypeArray > ( type ) ;
}
out_action = actions . back ( ) ;
actions . pop_back ( ) ;
return true ;
2014-04-30 19:19:29 +00:00
}
2018-12-02 11:00:23 +00:00
void ExpressionActions : : execute ( Block & block , bool dry_run ) const
2013-05-28 12:05:47 +00:00
{
2017-04-01 07:20:54 +00:00
for ( const auto & action : actions )
{
2018-12-02 11:00:23 +00:00
action . execute ( block , dry_run ) ;
2017-04-01 07:20:54 +00:00
checkLimits ( block ) ;
}
2013-05-28 12:05:47 +00:00
}
2020-01-15 20:33:29 +00:00
/// @warning It's a tricky method that allows to continue ONLY ONE action in reason of one-to-many ALL JOIN logic.
void ExpressionActions : : execute ( Block & block , ExtraBlockPtr & not_processed , size_t & start_action ) const
2020-01-13 18:00:32 +00:00
{
2020-01-15 20:33:29 +00:00
size_t i = start_action ;
start_action = 0 ;
for ( ; i < actions . size ( ) ; + + i )
2020-01-13 18:00:32 +00:00
{
2020-01-15 20:33:29 +00:00
actions [ i ] . execute ( block , false , not_processed ) ;
2020-01-13 18:00:32 +00:00
checkLimits ( block ) ;
if ( not_processed )
2020-01-15 20:33:29 +00:00
start_action = i ;
2020-01-13 18:00:32 +00:00
}
}
2019-04-09 14:51:38 +00:00
bool ExpressionActions : : hasTotalsInJoin ( ) const
2015-04-16 09:55:24 +00:00
{
2019-04-09 14:51:38 +00:00
for ( const auto & action : actions )
2019-09-16 12:37:46 +00:00
if ( action . table_join & & action . join - > hasTotals ( ) )
2019-09-03 14:36:02 +00:00
return true ;
return false ;
2019-04-09 14:51:38 +00:00
}
void ExpressionActions : : executeOnTotals ( Block & block ) const
{
/// If there is `totals` in the subquery for JOIN, but we do not have totals, then take the block with the default values instead of `totals`.
if ( ! block )
{
if ( hasTotalsInJoin ( ) )
2017-04-01 07:20:54 +00:00
{
for ( const auto & name_and_type : input_columns )
{
2017-12-15 18:23:05 +00:00
auto column = name_and_type . type - > createColumn ( ) ;
column - > insertDefault ( ) ;
block . insert ( ColumnWithTypeAndName ( std : : move ( column ) , name_and_type . type , name_and_type . name ) ) ;
2017-04-01 07:20:54 +00:00
}
}
else
2017-04-02 17:37:49 +00:00
return ; /// There's nothing to JOIN.
2017-04-01 07:20:54 +00:00
}
for ( const auto & action : actions )
action . executeOnTotals ( block ) ;
2015-04-16 09:55:24 +00:00
}
2017-12-25 21:57:29 +00:00
std : : string ExpressionActions : : getSmallestColumn ( const NamesAndTypesList & columns )
2013-06-10 14:24:40 +00:00
{
2017-12-09 06:32:22 +00:00
std : : optional < size_t > min_size ;
String res ;
2017-04-01 07:20:54 +00:00
2017-12-09 06:32:22 +00:00
for ( const auto & column : columns )
2017-04-01 07:20:54 +00:00
{
2017-12-09 06:32:22 +00:00
/// @todo resolve evil constant
size_t size = column . type - > haveMaximumSizeOfValue ( ) ? column . type - > getMaximumSizeOfValueInMemory ( ) : 100 ;
if ( ! min_size | | size < * min_size )
2017-04-01 07:20:54 +00:00
{
2017-12-09 06:32:22 +00:00
min_size = size ;
res = column . name ;
2017-04-01 07:20:54 +00:00
}
}
2017-12-09 06:32:22 +00:00
if ( ! min_size )
throw Exception ( " No available columns " , ErrorCodes : : LOGICAL_ERROR ) ;
2017-04-01 07:20:54 +00:00
return res ;
2013-06-10 14:24:40 +00:00
}
2013-05-28 11:54:37 +00:00
void ExpressionActions : : finalize ( const Names & output_columns )
2013-05-24 10:49:19 +00:00
{
2017-04-01 07:20:54 +00:00
NameSet final_columns ;
2020-03-08 23:48:08 +00:00
for ( const auto & name : output_columns )
2017-04-01 07:20:54 +00:00
{
if ( ! sample_block . has ( name ) )
throw Exception ( " Unknown column: " + name + " , there are only columns "
+ sample_block . dumpNames ( ) , ErrorCodes : : UNKNOWN_IDENTIFIER ) ;
final_columns . insert ( name ) ;
}
2018-04-29 01:00:26 +00:00
# if USE_EMBEDDED_COMPILER
2018-04-26 11:09:10 +00:00
/// This has to be done before removing redundant actions and inserting REMOVE_COLUMNs
/// because inlining may change dependency sets.
2018-05-02 21:47:28 +00:00
if ( settings . compile_expressions )
2019-02-11 14:36:54 +00:00
compileFunctions ( actions , output_columns , sample_block , compilation_cache , settings . min_count_to_compile_expression ) ;
2018-04-29 01:00:26 +00:00
# endif
2018-04-26 11:09:10 +00:00
2017-04-02 17:37:49 +00:00
/// Which columns are needed to perform actions from the current to the last.
2017-04-01 07:20:54 +00:00
NameSet needed_columns = final_columns ;
2017-04-02 17:37:49 +00:00
/// Which columns nobody will touch from the current action to the last.
2017-04-01 07:20:54 +00:00
NameSet unmodified_columns ;
2017-12-25 21:57:19 +00:00
{
2017-12-25 21:57:29 +00:00
NamesAndTypesList sample_columns = sample_block . getNamesAndTypesList ( ) ;
2020-03-08 23:48:08 +00:00
for ( const auto & sample_column : sample_columns )
unmodified_columns . insert ( sample_column . name ) ;
2017-12-25 21:57:19 +00:00
}
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// Let's go from the end and maintain set of required columns at this stage.
/// We will throw out unnecessary actions, although usually they are absent by construction.
2017-04-01 07:20:54 +00:00
for ( int i = static_cast < int > ( actions . size ( ) ) - 1 ; i > = 0 ; - - i )
{
ExpressionAction & action = actions [ i ] ;
Names in = action . getNeededColumns ( ) ;
if ( action . type = = ExpressionAction : : PROJECT )
{
needed_columns = NameSet ( in . begin ( ) , in . end ( ) ) ;
unmodified_columns . clear ( ) ;
}
2018-09-03 17:24:46 +00:00
else if ( action . type = = ExpressionAction : : ADD_ALIASES )
{
needed_columns . insert ( in . begin ( ) , in . end ( ) ) ;
2018-09-05 12:19:15 +00:00
for ( auto & name_wit_alias : action . projection )
{
auto it = unmodified_columns . find ( name_wit_alias . second ) ;
if ( it ! = unmodified_columns . end ( ) )
unmodified_columns . erase ( it ) ;
}
2018-09-03 17:24:46 +00:00
}
2017-04-01 07:20:54 +00:00
else if ( action . type = = ExpressionAction : : ARRAY_JOIN )
{
2020-02-05 10:12:19 +00:00
action . array_join - > finalize ( needed_columns , unmodified_columns , final_columns ) ;
2017-04-01 07:20:54 +00:00
}
else
{
std : : string out = action . result_name ;
if ( ! out . empty ( ) )
{
2017-04-02 17:37:49 +00:00
/// If the result is not used and there are no side effects, throw out the action.
2017-04-01 07:20:54 +00:00
if ( ! needed_columns . count ( out ) & &
( action . type = = ExpressionAction : : APPLY_FUNCTION
| | action . type = = ExpressionAction : : ADD_COLUMN
| | action . type = = ExpressionAction : : COPY_COLUMN ) )
{
actions . erase ( actions . begin ( ) + i ) ;
if ( unmodified_columns . count ( out ) )
{
sample_block . erase ( out ) ;
unmodified_columns . erase ( out ) ;
}
continue ;
}
unmodified_columns . erase ( out ) ;
needed_columns . erase ( out ) ;
2017-04-02 17:37:49 +00:00
/** If the function is a constant expression, then replace the action by adding a column-constant - result.
* That is , we perform constant folding .
2017-04-01 07:20:54 +00:00
*/
2019-08-16 09:38:12 +00:00
if ( action . type = = ExpressionAction : : APPLY_FUNCTION & & sample_block . has ( out ) )
2017-04-01 07:20:54 +00:00
{
auto & result = sample_block . getByName ( out ) ;
2019-08-16 10:29:33 +00:00
if ( result . column & & names_not_for_constant_folding . count ( result . name ) = = 0 )
2017-04-01 07:20:54 +00:00
{
action . type = ExpressionAction : : ADD_COLUMN ;
action . result_type = result . type ;
action . added_column = result . column ;
2018-02-02 08:33:36 +00:00
action . function_builder = nullptr ;
2018-09-21 14:15:21 +00:00
action . function_base = nullptr ;
2017-04-01 07:20:54 +00:00
action . function = nullptr ;
action . argument_names . clear ( ) ;
in . clear ( ) ;
}
}
}
needed_columns . insert ( in . begin ( ) , in . end ( ) ) ;
}
}
2019-07-31 10:22:56 +00:00
/// 1) Sometimes we don't need any columns to perform actions and sometimes actions doesn't produce any columns as result.
/// But Block class doesn't store any information about structure itself, it uses information from column.
/// If we remove all columns from input or output block we will lose information about amount of rows in it.
/// To avoid this situation we always leaving one of the columns in required columns (input)
/// and output column. We choose that "redundant" column by size with help of getSmallestColumn.
///
/// 2) Sometimes we have to read data from different Storages to execute query.
/// For example in 'remote' function which requires to read data from local table (for example MergeTree) and
/// remote table (doesn't know anything about it).
///
/// If we have combination of two previous cases, our heuristic from (1) can choose absolutely different columns,
2020-01-11 09:50:41 +00:00
/// so generated streams with these actions will have different headers. To avoid this we additionally rename our "redundant" column
2019-07-31 12:33:58 +00:00
/// to DUMMY_COLUMN_NAME with help of COPY_COLUMN action and consequent remove of original column.
/// It doesn't affect any logic, but all streams will have same "redundant" column in header called "_dummy".
2019-07-31 10:22:56 +00:00
/// Also, it seems like we will always have same type (UInt8) of "redundant" column, but it's not obvious.
2019-07-31 12:35:23 +00:00
bool dummy_column_copied = false ;
2019-07-31 10:22:56 +00:00
2017-04-02 17:37:49 +00:00
/// We will not throw out all the input columns, so as not to lose the number of rows in the block.
2017-04-01 07:20:54 +00:00
if ( needed_columns . empty ( ) & & ! input_columns . empty ( ) )
2019-07-31 10:22:56 +00:00
{
auto colname = getSmallestColumn ( input_columns ) ;
needed_columns . insert ( colname ) ;
2019-07-31 12:33:58 +00:00
actions . insert ( actions . begin ( ) , ExpressionAction : : copyColumn ( colname , DUMMY_COLUMN_NAME , true ) ) ;
2019-07-31 12:35:23 +00:00
dummy_column_copied = true ;
2019-07-31 10:22:56 +00:00
}
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// We will not leave the block empty so as not to lose the number of rows in it.
2018-02-22 10:54:28 +00:00
if ( final_columns . empty ( ) & & ! input_columns . empty ( ) )
2019-07-31 10:22:56 +00:00
{
auto colname = getSmallestColumn ( input_columns ) ;
final_columns . insert ( DUMMY_COLUMN_NAME ) ;
2019-07-31 12:35:23 +00:00
if ( ! dummy_column_copied ) /// otherwise we already have this column
2019-07-31 12:33:58 +00:00
actions . insert ( actions . begin ( ) , ExpressionAction : : copyColumn ( colname , DUMMY_COLUMN_NAME , true ) ) ;
2019-07-31 10:22:56 +00:00
}
2017-04-01 07:20:54 +00:00
2017-12-25 21:57:29 +00:00
for ( NamesAndTypesList : : iterator it = input_columns . begin ( ) ; it ! = input_columns . end ( ) ; )
2017-04-01 07:20:54 +00:00
{
2017-12-25 21:57:29 +00:00
NamesAndTypesList : : iterator it0 = it ;
2017-12-25 21:57:19 +00:00
+ + it ;
if ( ! needed_columns . count ( it0 - > name ) )
2017-04-01 07:20:54 +00:00
{
2017-12-25 21:57:19 +00:00
if ( unmodified_columns . count ( it0 - > name ) )
sample_block . erase ( it0 - > name ) ;
input_columns . erase ( it0 ) ;
2017-04-01 07:20:54 +00:00
}
}
2019-07-31 12:33:58 +00:00
/* std::cerr << "\n";
for ( const auto & action : actions )
2019-07-31 10:22:56 +00:00
std : : cerr < < action . toString ( ) < < " \n " ;
2019-07-31 12:33:58 +00:00
std : : cerr < < " \n " ; */
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// Deletes unnecessary temporary columns.
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// If the column after performing the function `refcount = 0`, it can be deleted.
2017-04-01 07:20:54 +00:00
std : : map < String , int > columns_refcount ;
for ( const auto & name : final_columns )
+ + columns_refcount [ name ] ;
for ( const auto & action : actions )
{
if ( ! action . source_name . empty ( ) )
+ + columns_refcount [ action . source_name ] ;
for ( const auto & name : action . argument_names )
+ + columns_refcount [ name ] ;
for ( const auto & name_alias : action . projection )
+ + columns_refcount [ name_alias . first ] ;
}
Actions new_actions ;
new_actions . reserve ( actions . size ( ) ) ;
for ( const auto & action : actions )
{
new_actions . push_back ( action ) ;
auto process = [ & ] ( const String & name )
{
auto refcount = - - columns_refcount [ name ] ;
if ( refcount < = 0 )
{
new_actions . push_back ( ExpressionAction : : removeColumn ( name ) ) ;
if ( sample_block . has ( name ) )
sample_block . erase ( name ) ;
}
} ;
if ( ! action . source_name . empty ( ) )
process ( action . source_name ) ;
for ( const auto & name : action . argument_names )
process ( name ) ;
2017-04-02 17:37:49 +00:00
/// For `projection`, there is no reduction in `refcount`, because the `project` action replaces the names of the columns, in effect, already deleting them under the old names.
2017-04-01 07:20:54 +00:00
}
actions . swap ( new_actions ) ;
/* std::cerr << "\n";
for ( const auto & action : actions )
std : : cerr < < action . toString ( ) < < " \n " ;
std : : cerr < < " \n " ; */
2018-04-25 11:16:51 +00:00
optimizeArrayJoin ( ) ;
2017-04-01 07:20:54 +00:00
checkLimits ( sample_block ) ;
2013-05-24 10:49:19 +00:00
}
2014-08-22 19:51:55 +00:00
2013-05-24 10:49:19 +00:00
std : : string ExpressionActions : : dumpActions ( ) const
{
2017-04-01 07:20:54 +00:00
std : : stringstream ss ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
ss < < " input: \n " ;
2020-03-08 23:48:08 +00:00
for ( const auto & input_column : input_columns )
ss < < input_column . name < < " " < < input_column . type - > getName ( ) < < " \n " ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
ss < < " \n actions: \n " ;
2020-03-08 23:48:08 +00:00
for ( const auto & action : actions )
ss < < action . toString ( ) < < ' \n ' ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
ss < < " \n output: \n " ;
2017-12-25 21:57:29 +00:00
NamesAndTypesList output_columns = sample_block . getNamesAndTypesList ( ) ;
2020-03-08 23:48:08 +00:00
for ( const auto & output_column : output_columns )
ss < < output_column . name < < " " < < output_column . type - > getName ( ) < < " \n " ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
return ss . str ( ) ;
2013-05-24 10:49:19 +00:00
}
2013-06-10 16:03:23 +00:00
void ExpressionActions : : optimizeArrayJoin ( )
{
2020-03-23 02:12:31 +00:00
const size_t none = actions . size ( ) ;
size_t first_array_join = none ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// Columns that need to be evaluated for arrayJoin.
/// Actions for adding them can not be moved to the left of the arrayJoin.
2017-04-01 07:20:54 +00:00
NameSet array_joined_columns ;
2017-04-02 17:37:49 +00:00
/// Columns needed to evaluate arrayJoin or those that depend on it.
/// Actions to delete them can not be moved to the left of the arrayJoin.
2017-04-01 07:20:54 +00:00
NameSet array_join_dependencies ;
for ( size_t i = 0 ; i < actions . size ( ) ; + + i )
{
2017-04-02 17:37:49 +00:00
/// Do not move the action to the right of the projection (the more that they are not usually there).
2017-04-01 07:20:54 +00:00
if ( actions [ i ] . type = = ExpressionAction : : PROJECT )
break ;
bool depends_on_array_join = false ;
Names needed ;
if ( actions [ i ] . type = = ExpressionAction : : ARRAY_JOIN )
{
depends_on_array_join = true ;
needed = actions [ i ] . getNeededColumns ( ) ;
}
else
{
2020-03-23 02:12:31 +00:00
if ( first_array_join = = none )
2017-04-01 07:20:54 +00:00
continue ;
needed = actions [ i ] . getNeededColumns ( ) ;
2020-03-08 23:48:08 +00:00
for ( const auto & elem : needed )
2017-04-01 07:20:54 +00:00
{
2020-03-08 23:48:08 +00:00
if ( array_joined_columns . count ( elem ) )
2017-04-01 07:20:54 +00:00
{
depends_on_array_join = true ;
break ;
}
}
}
if ( depends_on_array_join )
{
2020-03-23 02:12:31 +00:00
if ( first_array_join = = none )
2017-04-01 07:20:54 +00:00
first_array_join = i ;
2020-03-08 23:48:08 +00:00
if ( ! actions [ i ] . result_name . empty ( ) )
2017-04-01 07:20:54 +00:00
array_joined_columns . insert ( actions [ i ] . result_name ) ;
2020-02-05 10:12:19 +00:00
if ( actions [ i ] . array_join )
array_joined_columns . insert ( actions [ i ] . array_join - > columns . begin ( ) , actions [ i ] . array_join - > columns . end ( ) ) ;
2017-04-01 07:20:54 +00:00
array_join_dependencies . insert ( needed . begin ( ) , needed . end ( ) ) ;
}
else
{
bool can_move = false ;
if ( actions [ i ] . type = = ExpressionAction : : REMOVE_COLUMN )
{
2017-04-02 17:37:49 +00:00
/// If you delete a column that is not needed for arrayJoin (and those who depend on it), you can delete it before arrayJoin.
2017-04-01 07:20:54 +00:00
can_move = ! array_join_dependencies . count ( actions [ i ] . source_name ) ;
}
else
{
2017-04-02 17:37:49 +00:00
/// If the action does not delete the columns and does not depend on the result of arrayJoin, you can make it until arrayJoin.
2017-04-01 07:20:54 +00:00
can_move = true ;
}
2017-04-02 17:37:49 +00:00
/// Move the current action to the position just before the first arrayJoin.
2017-04-01 07:20:54 +00:00
if ( can_move )
{
2017-04-02 17:37:49 +00:00
/// Move the i-th element to the position `first_array_join`.
2017-04-01 07:20:54 +00:00
std : : rotate ( actions . begin ( ) + first_array_join , actions . begin ( ) + i , actions . begin ( ) + i + 1 ) ;
+ + first_array_join ;
}
}
}
2013-06-10 16:03:23 +00:00
}
2014-04-30 19:19:29 +00:00
2019-09-16 12:37:46 +00:00
JoinPtr ExpressionActions : : getTableJoinAlgo ( ) const
2015-04-18 22:30:43 +00:00
{
2017-04-01 07:20:54 +00:00
for ( const auto & action : actions )
2019-09-16 12:37:46 +00:00
if ( action . join )
return action . join ;
2017-04-01 07:20:54 +00:00
return { } ;
2015-04-18 22:30:43 +00:00
}
2019-10-11 17:27:54 +00:00
bool ExpressionActions : : resultIsAlwaysEmpty ( ) const
{
/// Check that has join which returns empty result.
for ( auto & action : actions )
{
if ( action . type = = action . JOIN & & action . join & & action . join - > alwaysReturnsEmptySet ( ) )
return true ;
}
return false ;
}
bool ExpressionActions : : checkColumnIsAlwaysFalse ( const String & column_name ) const
{
/// Check has column in (empty set).
String set_to_check ;
2019-11-01 10:58:29 +00:00
for ( auto it = actions . rbegin ( ) ; it ! = actions . rend ( ) ; + + it )
2019-10-11 17:27:54 +00:00
{
2019-11-01 10:58:29 +00:00
auto & action = * it ;
2019-10-11 17:55:33 +00:00
if ( action . type = = action . APPLY_FUNCTION & & action . function_base )
2019-10-11 17:27:54 +00:00
{
2019-10-11 17:55:33 +00:00
auto name = action . function_base - > getName ( ) ;
if ( ( name = = " in " | | name = = " globalIn " )
& & action . result_name = = column_name
& & action . argument_names . size ( ) > 1 )
{
set_to_check = action . argument_names [ 1 ] ;
2019-11-01 10:58:29 +00:00
break ;
2019-10-11 17:55:33 +00:00
}
2019-10-11 17:27:54 +00:00
}
}
if ( ! set_to_check . empty ( ) )
{
for ( auto & action : actions )
{
if ( action . type = = action . ADD_COLUMN & & action . result_name = = set_to_check )
{
2019-10-27 18:12:40 +00:00
// Constant ColumnSet cannot be empty, so we only need to check non-constant ones.
if ( auto * column_set = checkAndGetColumn < const ColumnSet > ( action . added_column . get ( ) ) )
2019-10-11 17:27:54 +00:00
{
2019-11-01 13:56:33 +00:00
if ( column_set - > getData ( ) - > isCreated ( ) & & column_set - > getData ( ) - > getTotalRowCount ( ) = = 0 )
2019-10-11 17:27:54 +00:00
return true ;
}
}
}
}
return false ;
}
2018-09-03 10:14:05 +00:00
/// It is not important to calculate the hash of individual strings or their concatenation
2018-09-07 20:17:40 +00:00
UInt128 ExpressionAction : : ActionHash : : operator ( ) ( const ExpressionAction & action ) const
2018-08-28 17:06:42 +00:00
{
2018-08-30 16:31:20 +00:00
SipHash hash ;
hash . update ( action . type ) ;
hash . update ( action . is_function_compiled ) ;
2018-11-23 18:52:00 +00:00
switch ( action . type )
2018-08-28 17:06:42 +00:00
{
case ADD_COLUMN :
2018-08-30 16:31:20 +00:00
hash . update ( action . result_name ) ;
2018-08-29 15:14:44 +00:00
if ( action . result_type )
2018-08-30 16:31:20 +00:00
hash . update ( action . result_type - > getName ( ) ) ;
2018-08-29 15:14:44 +00:00
if ( action . added_column )
2018-08-30 16:31:20 +00:00
hash . update ( action . added_column - > getName ( ) ) ;
2018-08-28 17:06:42 +00:00
break ;
case REMOVE_COLUMN :
2018-08-30 16:31:20 +00:00
hash . update ( action . source_name ) ;
2018-08-28 17:06:42 +00:00
break ;
case COPY_COLUMN :
2018-08-30 16:31:20 +00:00
hash . update ( action . result_name ) ;
hash . update ( action . source_name ) ;
2018-08-28 17:06:42 +00:00
break ;
case APPLY_FUNCTION :
2018-08-30 16:31:20 +00:00
hash . update ( action . result_name ) ;
2018-08-29 15:14:44 +00:00
if ( action . result_type )
2018-08-30 16:31:20 +00:00
hash . update ( action . result_type - > getName ( ) ) ;
2018-09-21 14:15:21 +00:00
if ( action . function_base )
2018-08-29 15:14:44 +00:00
{
2018-09-21 14:15:21 +00:00
hash . update ( action . function_base - > getName ( ) ) ;
for ( const auto & arg_type : action . function_base - > getArgumentTypes ( ) )
2018-08-30 16:31:20 +00:00
hash . update ( arg_type - > getName ( ) ) ;
2018-08-29 15:14:44 +00:00
}
2018-08-28 17:06:42 +00:00
for ( const auto & arg_name : action . argument_names )
2018-08-30 16:31:20 +00:00
hash . update ( arg_name ) ;
2018-08-28 17:06:42 +00:00
break ;
case ARRAY_JOIN :
2020-02-05 10:12:19 +00:00
hash . update ( action . array_join - > is_left ) ;
for ( const auto & col : action . array_join - > columns )
2018-08-30 16:31:20 +00:00
hash . update ( col ) ;
2018-08-28 17:06:42 +00:00
break ;
case JOIN :
2019-09-03 14:36:02 +00:00
for ( const auto & col : action . table_join - > columnsAddedByJoin ( ) )
2018-08-30 16:31:20 +00:00
hash . update ( col . name ) ;
2018-08-28 17:06:42 +00:00
break ;
case PROJECT :
for ( const auto & pair_of_strs : action . projection )
{
2018-08-30 16:31:20 +00:00
hash . update ( pair_of_strs . first ) ;
hash . update ( pair_of_strs . second ) ;
2018-08-28 17:06:42 +00:00
}
break ;
2018-09-06 09:52:22 +00:00
case ADD_ALIASES :
break ;
2018-08-28 17:06:42 +00:00
}
2018-09-07 20:17:40 +00:00
UInt128 result ;
hash . get128 ( result . low , result . high ) ;
return result ;
2018-08-28 17:06:42 +00:00
}
bool ExpressionAction : : operator = = ( const ExpressionAction & other ) const
{
2018-08-29 15:14:44 +00:00
if ( result_type ! = other . result_type )
{
if ( result_type = = nullptr | | other . result_type = = nullptr )
return false ;
else if ( ! result_type - > equals ( * other . result_type ) )
return false ;
}
2018-09-21 14:15:21 +00:00
if ( function_base ! = other . function_base )
2018-08-29 15:14:44 +00:00
{
2018-09-21 14:15:21 +00:00
if ( function_base = = nullptr | | other . function_base = = nullptr )
2018-08-29 15:14:44 +00:00
return false ;
2018-09-21 14:15:21 +00:00
else if ( function_base - > getName ( ) ! = other . function_base - > getName ( ) )
2018-08-29 15:14:44 +00:00
return false ;
2018-09-21 14:15:21 +00:00
const auto & my_arg_types = function_base - > getArgumentTypes ( ) ;
const auto & other_arg_types = other . function_base - > getArgumentTypes ( ) ;
2018-08-29 15:14:44 +00:00
if ( my_arg_types . size ( ) ! = other_arg_types . size ( ) )
return false ;
for ( size_t i = 0 ; i < my_arg_types . size ( ) ; + + i )
if ( ! my_arg_types [ i ] - > equals ( * other_arg_types [ i ] ) )
return false ;
}
if ( added_column ! = other . added_column )
{
if ( added_column = = nullptr | | other . added_column = = nullptr )
return false ;
else if ( added_column - > getName ( ) ! = other . added_column - > getName ( ) )
return false ;
}
2020-02-05 10:12:19 +00:00
bool same_array_join = ! array_join & & ! other . array_join ;
if ( array_join & & other . array_join )
same_array_join = ( array_join - > columns = = other . array_join - > columns ) & &
( array_join - > is_left = = other . array_join - > is_left ) ;
2018-08-29 15:14:44 +00:00
return source_name = = other . source_name
2018-08-28 17:06:42 +00:00
& & result_name = = other . result_name
& & argument_names = = other . argument_names
2020-02-05 10:12:19 +00:00
& & same_array_join
2019-09-03 14:36:02 +00:00
& & AnalyzedJoin : : sameJoin ( table_join . get ( ) , other . table_join . get ( ) )
2018-08-29 20:05:31 +00:00
& & projection = = other . projection
& & is_function_compiled = = other . is_function_compiled ;
2018-08-28 17:06:42 +00:00
}
2015-04-18 22:30:43 +00:00
2014-04-30 19:19:29 +00:00
void ExpressionActionsChain : : addStep ( )
{
2017-04-01 07:20:54 +00:00
if ( steps . empty ( ) )
throw Exception ( " Cannot add action to empty ExpressionActionsChain " , ErrorCodes : : LOGICAL_ERROR ) ;
2014-04-30 19:19:29 +00:00
2017-12-18 02:37:08 +00:00
ColumnsWithTypeAndName columns = steps . back ( ) . actions - > getSampleBlock ( ) . getColumnsWithTypeAndName ( ) ;
2018-08-30 16:31:20 +00:00
steps . push_back ( Step ( std : : make_shared < ExpressionActions > ( columns , context ) ) ) ;
2014-04-30 19:19:29 +00:00
}
void ExpressionActionsChain : : finalize ( )
{
2017-04-02 17:37:49 +00:00
/// Finalize all steps. Right to left to define unnecessary input columns.
2017-04-01 07:20:54 +00:00
for ( int i = static_cast < int > ( steps . size ( ) ) - 1 ; i > = 0 ; - - i )
{
Names required_output = steps [ i ] . required_output ;
2018-04-12 09:45:24 +00:00
std : : unordered_map < String , size_t > required_output_indexes ;
for ( size_t j = 0 ; j < required_output . size ( ) ; + + j )
required_output_indexes [ required_output [ j ] ] = j ;
2018-06-29 11:42:44 +00:00
auto & can_remove_required_output = steps [ i ] . can_remove_required_output ;
2018-04-12 09:45:24 +00:00
2017-04-01 07:20:54 +00:00
if ( i + 1 < static_cast < int > ( steps . size ( ) ) )
{
2018-04-12 09:45:24 +00:00
const NameSet & additional_input = steps [ i + 1 ] . additional_input ;
2017-04-01 07:20:54 +00:00
for ( const auto & it : steps [ i + 1 ] . actions - > getRequiredColumnsWithTypes ( ) )
2018-04-12 09:45:24 +00:00
{
if ( additional_input . count ( it . name ) = = 0 )
{
auto iter = required_output_indexes . find ( it . name ) ;
if ( iter = = required_output_indexes . end ( ) )
required_output . push_back ( it . name ) ;
else if ( ! can_remove_required_output . empty ( ) )
2018-06-29 11:42:44 +00:00
can_remove_required_output [ iter - > second ] = false ;
2018-04-12 09:45:24 +00:00
}
}
2017-04-01 07:20:54 +00:00
}
steps [ i ] . actions - > finalize ( required_output ) ;
}
2017-04-02 17:37:49 +00:00
/// When possible, move the ARRAY JOIN from earlier steps to later steps.
2017-04-01 07:20:54 +00:00
for ( size_t i = 1 ; i < steps . size ( ) ; + + i )
{
ExpressionAction action ;
if ( steps [ i - 1 ] . actions - > popUnusedArrayJoin ( steps [ i - 1 ] . required_output , action ) )
steps [ i ] . actions - > prependArrayJoin ( action , steps [ i - 1 ] . actions - > getSampleBlock ( ) ) ;
}
2017-04-02 17:37:49 +00:00
/// Adding the ejection of unnecessary columns to the beginning of each step.
2017-04-01 07:20:54 +00:00
for ( size_t i = 1 ; i < steps . size ( ) ; + + i )
{
size_t columns_from_previous = steps [ i - 1 ] . actions - > getSampleBlock ( ) . columns ( ) ;
2017-04-02 17:37:49 +00:00
/// If unnecessary columns are formed at the output of the previous step, we'll add them to the beginning of this step.
/// Except when we drop all the columns and lose the number of rows in the block.
2017-04-01 07:20:54 +00:00
if ( ! steps [ i ] . actions - > getRequiredColumnsWithTypes ( ) . empty ( )
& & columns_from_previous > steps [ i ] . actions - > getRequiredColumnsWithTypes ( ) . size ( ) )
steps [ i ] . actions - > prependProjectInput ( ) ;
}
2014-04-30 19:19:29 +00:00
}
std : : string ExpressionActionsChain : : dumpChain ( )
{
2017-04-01 07:20:54 +00:00
std : : stringstream ss ;
for ( size_t i = 0 ; i < steps . size ( ) ; + + i )
{
ss < < " step " < < i < < " \n " ;
ss < < " required output: \n " ;
for ( const std : : string & name : steps [ i ] . required_output )
ss < < name < < " \n " ;
ss < < " \n " < < steps [ i ] . actions - > dumpActions ( ) < < " \n " ;
}
return ss . str ( ) ;
2014-04-30 19:19:29 +00:00
}
2013-05-24 10:49:19 +00:00
}