2013-05-24 10:49:19 +00:00
# pragma once
2017-04-01 09:19:00 +00:00
# include <Interpreters/Settings.h>
# include <Core/Names.h>
# include <Core/ColumnWithTypeAndName.h>
# include <Core/Block.h>
2014-06-12 21:12:47 +00:00
# include <unordered_set>
# include <unordered_map>
2013-05-24 10:49:19 +00:00
namespace DB
{
2013-05-28 14:24:20 +00:00
2016-01-11 21:46:36 +00:00
namespace ErrorCodes
{
2017-04-01 07:20:54 +00:00
extern const int LOGICAL_ERROR ;
2016-01-11 21:46:36 +00:00
}
using NameWithAlias = std : : pair < std : : string , std : : string > ;
using NamesWithAliases = std : : vector < NameWithAlias > ;
2013-06-11 16:21:25 +00:00
2014-06-13 02:05:05 +00:00
class Join ;
2017-01-14 09:00:19 +00:00
class IFunction ;
using FunctionPtr = std : : shared_ptr < IFunction > ;
class IDataType ;
2017-12-18 01:11:48 +00:00
using DataTypePtr = std : : shared_ptr < const IDataType > ;
2017-01-14 09:00:19 +00:00
class IBlockInputStream ;
using BlockInputStreamPtr = std : : shared_ptr < IBlockInputStream > ;
2014-06-12 18:41:09 +00:00
2017-06-02 21:37:28 +00:00
/** Action on the block.
2013-05-24 10:49:19 +00:00
*/
2014-06-12 18:41:09 +00:00
struct ExpressionAction
2013-05-24 10:49:19 +00:00
{
public :
2017-04-01 07:20:54 +00:00
enum Type
{
ADD_COLUMN ,
REMOVE_COLUMN ,
COPY_COLUMN ,
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
APPLY_FUNCTION ,
2014-06-12 18:41:09 +00:00
2017-06-02 21:37:28 +00:00
/** Replaces the specified columns with arrays into columns with elements.
* Duplicates the values in the remaining columns by the number of elements in the arrays .
* Arrays must be parallel ( have the same lengths ) .
2017-04-01 07:20:54 +00:00
*/
ARRAY_JOIN ,
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
JOIN ,
2014-06-12 18:41:09 +00:00
2017-06-02 21:37:28 +00:00
/// Reorder and rename the columns, delete the extra ones. The same column names are allowed in the result.
2017-04-01 07:20:54 +00:00
PROJECT ,
} ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
Type type ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
/// For ADD/REMOVE/COPY_COLUMN.
std : : string source_name ;
std : : string result_name ;
DataTypePtr result_type ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
/// For ADD_COLUMN.
ColumnPtr added_column ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
/// For APPLY_FUNCTION and LEFT ARRAY JOIN.
mutable FunctionPtr function ; /// mutable - to allow execute.
Names argument_names ;
Names prerequisite_names ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
/// For ARRAY_JOIN
NameSet array_joined_columns ;
bool array_join_is_left = false ;
2014-06-12 19:23:06 +00:00
2017-04-01 07:20:54 +00:00
/// For JOIN
std : : shared_ptr < const Join > join ;
2017-12-25 21:57:29 +00:00
NamesAndTypesList columns_added_by_join ;
2014-06-12 19:23:06 +00:00
2017-04-01 07:20:54 +00:00
/// For PROJECT.
NamesWithAliases projection ;
2014-06-12 18:41:09 +00:00
2017-06-02 21:37:28 +00:00
/// If result_name_ == "", as name "function_name(arguments separated by commas) is used".
2017-04-01 07:20:54 +00:00
static ExpressionAction applyFunction (
const FunctionPtr & function_ , const std : : vector < std : : string > & argument_names_ , std : : string result_name_ = " " ) ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
static ExpressionAction addColumn ( const ColumnWithTypeAndName & added_column_ ) ;
static ExpressionAction removeColumn ( const std : : string & removed_name ) ;
static ExpressionAction copyColumn ( const std : : string & from_name , const std : : string & to_name ) ;
static ExpressionAction project ( const NamesWithAliases & projected_columns_ ) ;
static ExpressionAction project ( const Names & projected_columns_ ) ;
static ExpressionAction arrayJoin ( const NameSet & array_joined_columns , bool array_join_is_left , const Context & context ) ;
2017-12-25 21:57:29 +00:00
static ExpressionAction ordinaryJoin ( std : : shared_ptr < const Join > join_ , const NamesAndTypesList & columns_added_by_join_ ) ;
2014-06-13 02:05:05 +00:00
2017-06-02 21:37:28 +00:00
/// Which columns necessary to perform this action.
/// If this `Action` is not already added to `ExpressionActions`, the returned list may be incomplete, because `prerequisites` are not taken into account.
2017-04-01 07:20:54 +00:00
Names getNeededColumns ( ) const ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
std : : string toString ( ) const ;
2014-06-12 18:41:09 +00:00
private :
2017-04-01 07:20:54 +00:00
friend class ExpressionActions ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
std : : vector < ExpressionAction > getPrerequisites ( Block & sample_block ) ;
void prepare ( Block & sample_block ) ;
void execute ( Block & block ) const ;
void executeOnTotals ( Block & block ) const ;
2014-06-12 18:41:09 +00:00
} ;
2017-06-02 21:37:28 +00:00
/** Contains a sequence of actions on the block.
2014-06-12 18:41:09 +00:00
*/
class ExpressionActions
{
public :
2017-04-01 07:20:54 +00:00
using Actions = std : : vector < ExpressionAction > ;
2014-06-12 18:41:09 +00:00
2017-12-25 21:57:29 +00:00
ExpressionActions ( const NamesAndTypesList & input_columns_ , const Settings & settings_ )
2017-04-01 07:20:54 +00:00
: input_columns ( input_columns_ ) , settings ( settings_ )
{
for ( const auto & input_elem : input_columns )
sample_block . insert ( ColumnWithTypeAndName ( nullptr , input_elem . type , input_elem . name ) ) ;
}
2014-06-12 18:41:09 +00:00
2017-06-02 21:37:28 +00:00
/// For constant columns the columns themselves can be contained in `input_columns_`.
2017-04-01 07:20:54 +00:00
ExpressionActions ( const ColumnsWithTypeAndName & input_columns_ , const Settings & settings_ )
: settings ( settings_ )
{
for ( const auto & input_elem : input_columns_ )
{
input_columns . emplace_back ( input_elem . name , input_elem . type ) ;
sample_block . insert ( input_elem ) ;
}
}
2014-06-12 18:41:09 +00:00
2017-06-02 21:37:28 +00:00
/// Add the input column.
/// The name of the column must not match the names of the intermediate columns that occur when evaluating the expression.
/// The expression must not have any PROJECT actions.
2017-04-01 07:20:54 +00:00
void addInput ( const ColumnWithTypeAndName & column ) ;
2017-12-25 21:57:29 +00:00
void addInput ( const NameAndTypePair & column ) ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
void add ( const ExpressionAction & action ) ;
2014-06-12 18:41:09 +00:00
2017-06-02 21:37:28 +00:00
/// Adds new column names to out_new_columns
/// (formed as a result of the added action and its prerequisites).
2017-04-01 07:20:54 +00:00
void add ( const ExpressionAction & action , Names & out_new_columns ) ;
2014-06-12 18:41:09 +00:00
2017-06-02 21:37:28 +00:00
/// Adds to the beginning the removal of all extra columns.
2017-04-01 07:20:54 +00:00
void prependProjectInput ( ) ;
2014-04-30 19:19:29 +00:00
2017-06-02 21:37:28 +00:00
/// Add the specified ARRAY JOIN action to the beginning. Change the appropriate input types to arrays.
/// If there are unknown columns in the ARRAY JOIN list, take their types from sample_block, and immediately after ARRAY JOIN remove them.
2017-04-01 07:20:54 +00:00
void prependArrayJoin ( const ExpressionAction & action , const Block & sample_block ) ;
2014-04-30 19:19:29 +00:00
2017-06-02 21:37:28 +00:00
/// If the last action is ARRAY JOIN, and it does not affect the columns from required_columns, discard and return it.
/// Change the corresponding output types to arrays.
2017-04-01 07:20:54 +00:00
bool popUnusedArrayJoin ( const Names & required_columns , ExpressionAction & out_action ) ;
2014-06-12 18:41:09 +00:00
2017-06-02 21:37:28 +00:00
/// - Adds actions to delete all but the specified columns.
/// - Removes unused input columns.
/// - Can somehow optimize the expression.
/// - Does not reorder the columns.
/// - Does not remove "unexpected" columns (for example, added by functions).
/// - If output_columns is empty, leaves one arbitrary column (so that the number of rows in the block is not lost).
2017-04-01 07:20:54 +00:00
void finalize ( const Names & output_columns ) ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
const Actions & getActions ( ) const { return actions ; }
2017-02-09 17:29:36 +00:00
2017-06-02 21:37:28 +00:00
/// Get a list of input columns.
2017-04-01 07:20:54 +00:00
Names getRequiredColumns ( ) const
{
Names names ;
2017-12-25 21:57:29 +00:00
for ( NamesAndTypesList : : const_iterator it = input_columns . begin ( ) ; it ! = input_columns . end ( ) ; + + it )
2017-04-01 07:20:54 +00:00
names . push_back ( it - > name ) ;
return names ;
}
2014-06-12 18:41:09 +00:00
2017-12-25 21:57:29 +00:00
const NamesAndTypesList & getRequiredColumnsWithTypes ( ) const { return input_columns ; }
2013-05-24 10:49:19 +00:00
2017-06-02 21:37:28 +00:00
/// Execute the expression on the block. The block must contain all the columns returned by getRequiredColumns.
2017-04-01 07:20:54 +00:00
void execute ( Block & block ) const ;
2013-05-24 10:49:19 +00:00
2017-06-02 21:37:28 +00:00
/** Execute the expression on the block of total values.
* Almost the same as ` execute ` . The difference is only when JOIN is executed .
2017-04-01 07:20:54 +00:00
*/
void executeOnTotals ( Block & block ) const ;
2015-04-16 09:55:24 +00:00
2017-06-02 21:37:28 +00:00
/// Obtain a sample block that contains the names and types of result columns.
2017-04-01 07:20:54 +00:00
const Block & getSampleBlock ( ) const { return sample_block ; }
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
std : : string getID ( ) const ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
std : : string dumpActions ( ) const ;
2014-06-12 18:41:09 +00:00
2017-12-25 21:57:29 +00:00
static std : : string getSmallestColumn ( const NamesAndTypesList & columns ) ;
2013-05-24 10:49:19 +00:00
2017-04-01 07:20:54 +00:00
BlockInputStreamPtr createStreamWithNonJoinedDataIfFullOrRightJoin ( size_t max_block_size ) const ;
2015-04-18 22:30:43 +00:00
2013-05-24 10:49:19 +00:00
private :
2017-12-25 21:57:29 +00:00
NamesAndTypesList input_columns ;
2017-04-01 07:20:54 +00:00
Actions actions ;
Block sample_block ;
Settings settings ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
void checkLimits ( Block & block ) const ;
2014-06-12 18:41:09 +00:00
2017-06-02 21:37:28 +00:00
/// Adds all `prerequisites` first, then the action itself.
/// current_names - columns whose `prerequisites` are currently being processed.
2017-04-01 07:20:54 +00:00
void addImpl ( ExpressionAction action , NameSet & current_names , Names & new_names ) ;
2014-06-12 18:41:09 +00:00
2017-06-02 21:37:28 +00:00
/// Try to improve something without changing the lists of input and output columns.
2017-04-01 07:20:54 +00:00
void optimize ( ) ;
2017-06-02 21:37:28 +00:00
/// Move all arrayJoin as close as possible to the end.
2017-04-01 07:20:54 +00:00
void optimizeArrayJoin ( ) ;
2013-05-24 10:49:19 +00:00
} ;
2016-01-13 00:32:59 +00:00
using ExpressionActionsPtr = std : : shared_ptr < ExpressionActions > ;
2013-05-24 10:49:19 +00:00
2017-06-02 21:37:28 +00:00
/** The sequence of transformations over the block.
* It is assumed that the result of each step is fed to the input of the next step .
* Used to execute parts of the query individually .
2014-06-12 18:41:09 +00:00
*
2017-06-02 21:37:28 +00:00
* For example , you can create a chain of two steps :
* 1 ) evaluate the expression in the WHERE clause ,
* 2 ) calculate the expression in the SELECT section ,
* and between the two steps do the filtering by value in the WHERE clause .
2013-06-20 12:27:33 +00:00
*/
2013-05-28 14:24:20 +00:00
struct ExpressionActionsChain
{
2017-04-01 07:20:54 +00:00
struct Step
{
ExpressionActionsPtr actions ;
Names required_output ;
2014-06-12 18:41:09 +00:00
2017-09-08 03:47:27 +00:00
Step ( const ExpressionActionsPtr & actions_ = nullptr , const Names & required_output_ = Names ( ) )
2017-04-01 07:20:54 +00:00
: actions ( actions_ ) , required_output ( required_output_ ) { }
} ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
using Steps = std : : vector < Step > ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
Settings settings ;
Steps steps ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
void addStep ( ) ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
void finalize ( ) ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
void clear ( )
{
steps . clear ( ) ;
}
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
ExpressionActionsPtr getLastActions ( )
{
if ( steps . empty ( ) )
throw Exception ( " Empty ExpressionActionsChain " , ErrorCodes : : LOGICAL_ERROR ) ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
return steps . back ( ) . actions ;
}
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
Step & getLastStep ( )
{
if ( steps . empty ( ) )
throw Exception ( " Empty ExpressionActionsChain " , ErrorCodes : : LOGICAL_ERROR ) ;
2014-06-12 18:41:09 +00:00
2017-04-01 07:20:54 +00:00
return steps . back ( ) ;
}
2014-04-30 19:19:29 +00:00
2017-04-01 07:20:54 +00:00
std : : string dumpChain ( ) ;
2013-05-28 14:24:20 +00:00
} ;
2013-05-24 10:49:19 +00:00
}