2019-07-04 12:12:39 +00:00
# include <any>
2020-04-21 19:01:34 +00:00
# include <limits>
2021-09-06 10:59:18 +00:00
# include <unordered_map>
# include <vector>
2019-07-04 12:12:39 +00:00
2022-10-04 08:20:13 +00:00
# include <Common/StackTrace.h>
2022-04-27 15:05:45 +00:00
# include <Common/logger_useful.h>
2017-01-21 04:24:28 +00:00
2017-07-21 06:35:58 +00:00
# include <Columns/ColumnConst.h>
2017-04-01 09:19:00 +00:00
# include <Columns/ColumnString.h>
2020-04-08 18:59:52 +00:00
# include <Columns/ColumnVector.h>
2017-04-01 09:19:00 +00:00
# include <Columns/ColumnFixedString.h>
2017-04-04 06:52:39 +00:00
# include <Columns/ColumnNullable.h>
# include <DataTypes/DataTypeNullable.h>
2020-04-08 18:59:52 +00:00
# include <DataTypes/DataTypeLowCardinality.h>
2014-06-12 02:31:30 +00:00
2020-04-07 09:48:47 +00:00
# include <Interpreters/HashJoin.h>
2022-08-04 15:20:19 +00:00
# include <Interpreters/JoinUtils.h>
2020-04-07 09:48:47 +00:00
# include <Interpreters/TableJoin.h>
2019-07-02 14:38:31 +00:00
# include <Interpreters/joinDispatch.h>
2017-04-01 09:19:00 +00:00
# include <Interpreters/NullableUtils.h>
2017-03-30 14:09:24 +00:00
2022-01-21 05:36:36 +00:00
# include <Storages/IStorage.h>
2018-02-19 21:01:46 +00:00
2017-04-01 09:19:00 +00:00
# include <Core/ColumnNumbers.h>
2022-01-21 05:36:36 +00:00
# include <Common/Exception.h>
2017-07-13 20:58:19 +00:00
# include <Common/typeid_cast.h>
2019-08-21 02:28:04 +00:00
# include <Common/assert_cast.h>
2021-11-08 12:44:13 +00:00
2014-06-12 02:31:30 +00:00
namespace DB
{
2014-06-13 02:05:05 +00:00
2016-01-12 02:21:15 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int NOT_IMPLEMENTED ;
2020-06-02 20:26:41 +00:00
extern const int NO_SUCH_COLUMN_IN_TABLE ;
extern const int INCOMPATIBLE_TYPE_OF_JOIN ;
2019-08-05 14:03:14 +00:00
extern const int UNSUPPORTED_JOIN_KEYS ;
2016-01-12 02:21:15 +00:00
extern const int LOGICAL_ERROR ;
2020-07-16 08:56:45 +00:00
extern const int SYNTAX_ERROR ;
2016-01-12 02:21:15 +00:00
extern const int SET_SIZE_LIMIT_EXCEEDED ;
extern const int TYPE_MISMATCH ;
2020-07-11 07:12:42 +00:00
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH ;
2016-01-12 02:21:15 +00:00
}
2020-04-06 13:39:57 +00:00
namespace
{
struct NotProcessedCrossJoin : public ExtraBlock
{
size_t left_position ;
size_t right_block ;
} ;
}
2019-03-27 13:37:11 +00:00
2021-02-04 14:46:36 +00:00
namespace JoinStuff
{
2021-08-17 10:27:23 +00:00
/// for single disjunct
2021-02-04 14:46:36 +00:00
bool JoinUsedFlags : : getUsedSafe ( size_t i ) const
{
2021-08-17 10:27:23 +00:00
return getUsedSafe ( nullptr , i ) ;
2021-02-04 14:46:36 +00:00
}
2021-08-17 10:27:23 +00:00
/// for multiple disjuncts
bool JoinUsedFlags : : getUsedSafe ( const Block * block_ptr , size_t row_idx ) const
{
if ( auto it = flags . find ( block_ptr ) ; it ! = flags . end ( ) )
return it - > second [ row_idx ] . load ( ) ;
return ! need_flags ;
2021-02-04 14:46:36 +00:00
}
2021-08-17 10:27:23 +00:00
/// for single disjunct
2022-07-29 16:30:50 +00:00
template < JoinKind KIND , JoinStrictness STRICTNESS >
2021-02-04 14:46:36 +00:00
void JoinUsedFlags : : reinit ( size_t size )
{
if constexpr ( MapGetter < KIND , STRICTNESS > : : flagged )
{
2021-08-17 10:27:23 +00:00
assert ( flags [ nullptr ] . size ( ) < = size ) ;
2021-02-04 14:46:36 +00:00
need_flags = true ;
2021-08-17 10:27:23 +00:00
flags [ nullptr ] = std : : vector < std : : atomic_bool > ( size ) ;
2021-02-04 14:46:36 +00:00
}
}
2021-08-17 10:27:23 +00:00
/// for multiple disjuncts
2022-07-29 16:30:50 +00:00
template < JoinKind KIND , JoinStrictness STRICTNESS >
2021-08-17 10:27:23 +00:00
void JoinUsedFlags : : reinit ( const Block * block_ptr )
{
if constexpr ( MapGetter < KIND , STRICTNESS > : : flagged )
{
assert ( flags [ block_ptr ] . size ( ) < = block_ptr - > rows ( ) ) ;
need_flags = true ;
flags [ block_ptr ] = std : : vector < std : : atomic_bool > ( block_ptr - > rows ( ) ) ;
2021-02-04 14:46:36 +00:00
}
}
2021-06-25 12:03:10 +00:00
template < bool use_flags , bool multiple_disjuncts , typename FindResult >
void JoinUsedFlags : : setUsed ( const FindResult & f )
2021-02-04 14:46:36 +00:00
{
2021-08-17 10:27:23 +00:00
if constexpr ( ! use_flags )
return ;
2021-02-04 14:46:36 +00:00
/// Could be set simultaneously from different threads.
2021-08-17 10:27:23 +00:00
if constexpr ( multiple_disjuncts )
2021-06-25 12:03:10 +00:00
{
2021-08-17 10:27:23 +00:00
auto & mapped = f . getMapped ( ) ;
flags [ mapped . block ] [ mapped . row_num ] . store ( true , std : : memory_order_relaxed ) ;
}
else
{
flags [ nullptr ] [ f . getOffset ( ) ] . store ( true , std : : memory_order_relaxed ) ;
2021-06-25 12:03:10 +00:00
}
2021-02-04 14:46:36 +00:00
}
2022-02-19 20:01:47 +00:00
template < bool use_flags , bool multiple_disjuncts >
void JoinUsedFlags : : setUsed ( const Block * block , size_t row_num , size_t offset )
{
if constexpr ( ! use_flags )
return ;
/// Could be set simultaneously from different threads.
if constexpr ( multiple_disjuncts )
{
flags [ block ] [ row_num ] . store ( true , std : : memory_order_relaxed ) ;
}
else
{
flags [ nullptr ] [ offset ] . store ( true , std : : memory_order_relaxed ) ;
}
}
2021-06-25 12:03:10 +00:00
template < bool use_flags , bool multiple_disjuncts , typename FindResult >
bool JoinUsedFlags : : getUsed ( const FindResult & f )
{
2021-08-17 10:27:23 +00:00
if constexpr ( ! use_flags )
return true ;
2021-02-04 14:46:36 +00:00
2021-08-17 10:27:23 +00:00
if constexpr ( multiple_disjuncts )
2021-06-25 12:03:10 +00:00
{
2021-08-17 10:27:23 +00:00
auto & mapped = f . getMapped ( ) ;
return flags [ mapped . block ] [ mapped . row_num ] . load ( ) ;
}
else
{
return flags [ nullptr ] [ f . getOffset ( ) ] . load ( ) ;
2021-06-25 12:03:10 +00:00
}
}
2021-02-04 14:46:36 +00:00
2021-06-25 12:03:10 +00:00
template < bool use_flags , bool multiple_disjuncts , typename FindResult >
bool JoinUsedFlags : : setUsedOnce ( const FindResult & f )
2021-02-04 14:46:36 +00:00
{
2021-08-17 10:27:23 +00:00
if constexpr ( ! use_flags )
return true ;
if constexpr ( multiple_disjuncts )
{
auto & mapped = f . getMapped ( ) ;
/// fast check to prevent heavy CAS with seq_cst order
if ( flags [ mapped . block ] [ mapped . row_num ] . load ( std : : memory_order_relaxed ) )
return false ;
bool expected = false ;
return flags [ mapped . block ] [ mapped . row_num ] . compare_exchange_strong ( expected , true ) ;
}
else
2021-06-25 12:03:10 +00:00
{
2021-08-17 10:27:23 +00:00
auto off = f . getOffset ( ) ;
2021-06-25 12:03:10 +00:00
/// fast check to prevent heavy CAS with seq_cst order
2021-08-17 10:27:23 +00:00
if ( flags [ nullptr ] [ off ] . load ( std : : memory_order_relaxed ) )
2021-06-25 12:03:10 +00:00
return false ;
2021-02-04 14:46:36 +00:00
2021-06-25 12:03:10 +00:00
bool expected = false ;
2021-08-17 10:27:23 +00:00
return flags [ nullptr ] [ off ] . compare_exchange_strong ( expected , true ) ;
2021-06-25 12:03:10 +00:00
}
2021-02-04 14:46:36 +00:00
}
}
2022-12-19 12:31:58 +00:00
static void correctNullabilityInplace ( ColumnWithTypeAndName & column , bool nullable )
2019-03-27 13:37:11 +00:00
{
if ( nullable )
2019-10-29 19:39:42 +00:00
{
2019-09-11 18:03:21 +00:00
JoinCommon : : convertColumnToNullable ( column ) ;
2019-10-29 19:39:42 +00:00
}
else
{
/// We have to replace values masked by NULLs with defaults.
if ( column . column )
2020-04-22 06:01:33 +00:00
if ( const auto * nullable_column = checkAndGetColumn < ColumnNullable > ( * column . column ) )
2022-07-07 12:26:34 +00:00
column . column = JoinCommon : : filterWithBlanks ( column . column , nullable_column - > getNullMapColumn ( ) . getData ( ) , true ) ;
2019-10-29 19:39:42 +00:00
JoinCommon : : removeColumnNullability ( column ) ;
}
2019-03-27 13:37:11 +00:00
}
2022-12-19 12:31:58 +00:00
static void correctNullabilityInplace ( ColumnWithTypeAndName & column , bool nullable , const ColumnUInt8 & negative_null_map )
2019-03-27 13:37:11 +00:00
{
if ( nullable )
{
2021-05-15 14:09:51 +00:00
JoinCommon : : convertColumnToNullable ( column ) ;
2020-03-09 00:28:05 +00:00
if ( column . type - > isNullable ( ) & & ! negative_null_map . empty ( ) )
2019-03-27 13:37:11 +00:00
{
2020-05-14 08:30:18 +00:00
MutableColumnPtr mutable_column = IColumn : : mutate ( std : : move ( column . column ) ) ;
2019-08-21 02:28:04 +00:00
assert_cast < ColumnNullable & > ( * mutable_column ) . applyNegatedNullMap ( negative_null_map ) ;
2019-03-27 13:37:11 +00:00
column . column = std : : move ( mutable_column ) ;
}
}
2019-10-29 19:39:42 +00:00
else
JoinCommon : : removeColumnNullability ( column ) ;
2019-03-27 13:37:11 +00:00
}
2020-07-10 18:10:06 +00:00
HashJoin : : HashJoin ( std : : shared_ptr < TableJoin > table_join_ , const Block & right_sample_block_ , bool any_take_last_row_ )
: table_join ( table_join_ )
, kind ( table_join - > kind ( ) )
, strictness ( table_join - > strictness ( ) )
, any_take_last_row ( any_take_last_row_ )
, asof_inequality ( table_join - > getAsofInequality ( ) )
, data ( std : : make_shared < RightTableData > ( ) )
, right_sample_block ( right_sample_block_ )
, log ( & Poco : : Logger : : get ( " HashJoin " ) )
2020-03-13 08:15:43 +00:00
{
2023-01-23 12:34:36 +00:00
LOG_DEBUG ( log , " ({}) Datatype: {}, kind: {}, strictness: {}, right header: {} " , fmt : : ptr ( this ) , data - > type , kind , strictness , right_sample_block . dumpStructure ( ) ) ;
LOG_DEBUG ( log , " ({}) Keys: {} " , fmt : : ptr ( this ) , TableJoin : : formatClauses ( table_join - > getClauses ( ) , true ) ) ;
2020-03-13 08:15:43 +00:00
2021-09-29 10:57:48 +00:00
if ( isCrossOrComma ( kind ) )
2021-06-25 12:03:10 +00:00
{
2021-09-10 14:52:44 +00:00
data - > type = Type : : CROSS ;
sample_block_with_columns_to_add = right_sample_block ;
2021-06-25 12:03:10 +00:00
}
2022-10-17 16:12:00 +00:00
else if ( table_join - > getClauses ( ) . empty ( ) )
{
data - > type = Type : : EMPTY ;
sample_block_with_columns_to_add = right_sample_block ;
}
2021-09-10 14:52:44 +00:00
else if ( table_join - > oneDisjunct ( ) )
2021-06-25 12:03:10 +00:00
{
2021-09-06 10:59:18 +00:00
const auto & key_names_right = table_join - > getOnlyClause ( ) . key_names_right ;
2021-06-25 12:03:10 +00:00
JoinCommon : : splitAdditionalColumns ( key_names_right , right_sample_block , right_table_keys , sample_block_with_columns_to_add ) ;
required_right_keys = table_join - > getRequiredRightKeys ( right_table_keys , required_right_keys_sources ) ;
}
2021-09-10 14:52:44 +00:00
else
{
/// required right keys concept does not work well if multiple disjuncts, we need all keys
sample_block_with_columns_to_add = right_table_keys = materializeBlock ( right_sample_block ) ;
}
2021-08-18 08:18:33 +00:00
2021-04-15 02:16:09 +00:00
JoinCommon : : convertToFullColumnsInplace ( right_table_keys ) ;
2020-07-10 18:10:06 +00:00
initRightBlockStructure ( data - > sample_block ) ;
2020-03-13 08:15:43 +00:00
2020-07-10 18:10:06 +00:00
JoinCommon : : createMissedColumns ( sample_block_with_columns_to_add ) ;
2020-03-13 08:15:43 +00:00
2021-09-06 10:59:18 +00:00
size_t disjuncts_num = table_join - > getClauses ( ) . size ( ) ;
2021-09-09 09:47:08 +00:00
data - > maps . resize ( disjuncts_num ) ;
2021-09-06 10:59:18 +00:00
key_sizes . reserve ( disjuncts_num ) ;
2020-03-13 08:15:43 +00:00
2021-09-06 10:59:18 +00:00
for ( const auto & clause : table_join - > getClauses ( ) )
2020-07-10 18:10:06 +00:00
{
2021-09-06 10:59:18 +00:00
const auto & key_names_right = clause . key_names_right ;
ColumnRawPtrs key_columns = JoinCommon : : extractKeysForJoin ( right_table_keys , key_names_right ) ;
2020-07-10 18:10:06 +00:00
2022-08-04 15:15:49 +00:00
if ( strictness = = JoinStrictness : : Asof )
2021-06-25 12:03:10 +00:00
{
2021-09-06 10:59:18 +00:00
assert ( disjuncts_num = = 1 ) ;
2021-06-25 12:03:10 +00:00
/// @note ASOF JOIN is not INNER. It's better avoid use of 'INNER ASOF' combination in messages.
/// In fact INNER means 'LEFT SEMI ASOF' while LEFT means 'LEFT OUTER ASOF'.
if ( ! isLeft ( kind ) & & ! isInner ( kind ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Wrong ASOF JOIN type. Only ASOF and LEFT ASOF joins are supported " ) ;
2021-06-25 12:03:10 +00:00
if ( key_columns . size ( ) < = 1 )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : SYNTAX_ERROR , " ASOF join needs at least one equi-join column " ) ;
2021-06-25 12:03:10 +00:00
size_t asof_size ;
2022-02-18 10:15:11 +00:00
asof_type = SortedLookupVectorBase : : getTypeSize ( * key_columns . back ( ) , asof_size ) ;
2021-06-25 12:03:10 +00:00
key_columns . pop_back ( ) ;
/// this is going to set up the appropriate hash table for the direct lookup part of the join
/// However, this does not depend on the size of the asof join key (as that goes into the BST)
/// Therefore, add it back in such that it can be extracted appropriately from the full stored
/// key_columns and key_sizes
2021-09-09 09:47:08 +00:00
auto & asof_key_sizes = key_sizes . emplace_back ( ) ;
2021-11-08 12:44:13 +00:00
data - > type = chooseMethod ( kind , key_columns , asof_key_sizes ) ;
2021-09-09 09:47:08 +00:00
asof_key_sizes . push_back ( asof_size ) ;
2021-06-25 12:03:10 +00:00
}
else
{
/// Choose data structure to use for JOIN.
2021-11-08 12:44:13 +00:00
auto current_join_method = chooseMethod ( kind , key_columns , key_sizes . emplace_back ( ) ) ;
2021-09-06 10:59:18 +00:00
if ( data - > type = = Type : : EMPTY )
data - > type = current_join_method ;
else if ( data - > type ! = current_join_method )
data - > type = Type : : hashed ;
2021-06-25 12:03:10 +00:00
}
2020-07-10 18:10:06 +00:00
}
2021-06-25 12:03:10 +00:00
2021-09-06 10:59:18 +00:00
for ( auto & maps : data - > maps )
dataMapInit ( maps ) ;
2020-07-10 18:10:06 +00:00
}
2017-01-21 04:24:28 +00:00
2022-07-29 16:30:50 +00:00
HashJoin : : Type HashJoin : : chooseMethod ( JoinKind kind , const ColumnRawPtrs & key_columns , Sizes & key_sizes )
2015-03-02 01:10:58 +00:00
{
size_t keys_size = key_columns . size ( ) ;
2017-04-01 07:20:54 +00:00
2015-07-23 20:23:24 +00:00
if ( keys_size = = 0 )
2021-11-08 12:44:13 +00:00
{
if ( isCrossOrComma ( kind ) )
return Type : : CROSS ;
return Type : : EMPTY ;
}
2017-04-01 07:20:54 +00:00
2017-03-28 06:51:22 +00:00
bool all_fixed = true ;
size_t keys_bytes = 0 ;
key_sizes . resize ( keys_size ) ;
2015-03-02 01:10:58 +00:00
for ( size_t j = 0 ; j < keys_size ; + + j )
{
2017-12-09 10:14:45 +00:00
if ( ! key_columns [ j ] - > isFixedAndContiguous ( ) )
2015-03-02 01:10:58 +00:00
{
2017-03-28 06:51:22 +00:00
all_fixed = false ;
2015-03-02 01:10:58 +00:00
break ;
}
2017-12-09 10:14:45 +00:00
key_sizes [ j ] = key_columns [ j ] - > sizeOfValueIfFixed ( ) ;
2015-03-02 01:10:58 +00:00
keys_bytes + = key_sizes [ j ] ;
}
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// If there is one numeric key that fits in 64 bits
2017-12-09 10:14:45 +00:00
if ( keys_size = = 1 & & key_columns [ 0 ] - > isNumeric ( ) )
2017-03-28 06:51:22 +00:00
{
2017-12-09 10:14:45 +00:00
size_t size_of_field = key_columns [ 0 ] - > sizeOfValueIfFixed ( ) ;
2017-03-28 06:51:22 +00:00
if ( size_of_field = = 1 )
return Type : : key8 ;
if ( size_of_field = = 2 )
return Type : : key16 ;
if ( size_of_field = = 4 )
return Type : : key32 ;
if ( size_of_field = = 8 )
return Type : : key64 ;
2017-11-09 23:37:57 +00:00
if ( size_of_field = = 16 )
return Type : : keys128 ;
2020-09-04 13:33:02 +00:00
if ( size_of_field = = 32 )
return Type : : keys256 ;
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32. " ) ;
2017-03-28 06:51:22 +00:00
}
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/// If the keys fit in N bits, we will use a hash table for N-bit-packed keys
2017-03-28 06:51:22 +00:00
if ( all_fixed & & keys_bytes < = 16 )
return Type : : keys128 ;
if ( all_fixed & & keys_bytes < = 32 )
return Type : : keys256 ;
2017-04-01 07:20:54 +00:00
2017-03-28 06:51:22 +00:00
/// If there is single string key, use hash table of it's values.
2017-07-21 06:35:58 +00:00
if ( keys_size = = 1
& & ( typeid_cast < const ColumnString * > ( key_columns [ 0 ] )
2019-08-21 02:28:04 +00:00
| | ( isColumnConst ( * key_columns [ 0 ] ) & & typeid_cast < const ColumnString * > ( & assert_cast < const ColumnConst * > ( key_columns [ 0 ] ) - > getDataColumn ( ) ) ) ) )
2017-03-28 06:51:22 +00:00
return Type : : key_string ;
2017-04-01 07:20:54 +00:00
2017-03-28 06:51:22 +00:00
if ( keys_size = = 1 & & typeid_cast < const ColumnFixedString * > ( key_columns [ 0 ] ) )
return Type : : key_fixed_string ;
2017-04-01 07:20:54 +00:00
2017-03-28 06:51:22 +00:00
/// Otherwise, will use set of cryptographic hashes of unambiguously serialized values.
return Type : : hashed ;
2015-03-02 01:10:58 +00:00
}
2022-06-14 14:13:01 +00:00
template < typename KeyGetter , bool is_asof_join >
2019-03-28 18:35:50 +00:00
static KeyGetter createKeyGetter ( const ColumnRawPtrs & key_columns , const Sizes & key_sizes )
2019-03-26 22:05:51 +00:00
{
2019-12-02 18:07:27 +00:00
if constexpr ( is_asof_join )
2019-03-26 22:05:51 +00:00
{
2019-03-28 18:35:50 +00:00
auto key_column_copy = key_columns ;
auto key_size_copy = key_sizes ;
key_column_copy . pop_back ( ) ;
key_size_copy . pop_back ( ) ;
return KeyGetter ( key_column_copy , key_size_copy , nullptr ) ;
2019-03-20 16:58:28 +00:00
}
2019-03-28 18:35:50 +00:00
else
return KeyGetter ( key_columns , key_sizes , nullptr ) ;
2019-03-20 16:58:28 +00:00
}
2021-08-17 10:27:23 +00:00
template < typename Mapped , bool need_offset = false >
using FindResultImpl = ColumnsHashing : : columns_hashing_impl : : FindResultImpl < Mapped , true > ;
2021-11-08 12:44:13 +00:00
/// Dummy key getter, always find nothing, used for JOIN ON NULL
template < typename Mapped >
class KeyGetterEmpty
{
public :
struct MappedType
{
using mapped_type = Mapped ;
} ;
using FindResult = ColumnsHashing : : columns_hashing_impl : : FindResultImpl < Mapped , true > ;
KeyGetterEmpty ( ) = default ;
FindResult findKey ( MappedType , size_t , const Arena & ) { return FindResult ( ) ; }
} ;
2020-04-07 09:48:47 +00:00
template < HashJoin : : Type type , typename Value , typename Mapped >
2019-01-24 14:56:04 +00:00
struct KeyGetterForTypeImpl ;
2017-03-28 06:51:22 +00:00
2021-02-04 14:46:36 +00:00
constexpr bool use_offset = true ;
2020-04-07 09:48:47 +00:00
template < typename Value , typename Mapped > struct KeyGetterForTypeImpl < HashJoin : : Type : : key8 , Value , Mapped >
2019-01-24 14:56:04 +00:00
{
2021-02-04 14:46:36 +00:00
using Type = ColumnsHashing : : HashMethodOneNumber < Value , Mapped , UInt8 , false , use_offset > ;
2019-01-24 14:56:04 +00:00
} ;
2020-04-07 09:48:47 +00:00
template < typename Value , typename Mapped > struct KeyGetterForTypeImpl < HashJoin : : Type : : key16 , Value , Mapped >
2019-01-24 14:56:04 +00:00
{
2021-02-04 14:46:36 +00:00
using Type = ColumnsHashing : : HashMethodOneNumber < Value , Mapped , UInt16 , false , use_offset > ;
2019-01-24 14:56:04 +00:00
} ;
2020-04-07 09:48:47 +00:00
template < typename Value , typename Mapped > struct KeyGetterForTypeImpl < HashJoin : : Type : : key32 , Value , Mapped >
2019-01-24 14:56:04 +00:00
{
2021-02-04 14:46:36 +00:00
using Type = ColumnsHashing : : HashMethodOneNumber < Value , Mapped , UInt32 , false , use_offset > ;
2019-01-24 14:56:04 +00:00
} ;
2020-04-07 09:48:47 +00:00
template < typename Value , typename Mapped > struct KeyGetterForTypeImpl < HashJoin : : Type : : key64 , Value , Mapped >
2019-01-24 14:56:04 +00:00
{
2021-02-04 14:46:36 +00:00
using Type = ColumnsHashing : : HashMethodOneNumber < Value , Mapped , UInt64 , false , use_offset > ;
2019-01-24 14:56:04 +00:00
} ;
2020-04-07 09:48:47 +00:00
template < typename Value , typename Mapped > struct KeyGetterForTypeImpl < HashJoin : : Type : : key_string , Value , Mapped >
2019-01-24 14:56:04 +00:00
{
2021-02-04 14:46:36 +00:00
using Type = ColumnsHashing : : HashMethodString < Value , Mapped , true , false , use_offset > ;
2019-01-24 14:56:04 +00:00
} ;
2020-04-07 09:48:47 +00:00
template < typename Value , typename Mapped > struct KeyGetterForTypeImpl < HashJoin : : Type : : key_fixed_string , Value , Mapped >
2019-01-24 14:56:04 +00:00
{
2021-02-04 14:46:36 +00:00
using Type = ColumnsHashing : : HashMethodFixedString < Value , Mapped , true , false , use_offset > ;
2019-01-24 14:56:04 +00:00
} ;
2020-04-07 09:48:47 +00:00
template < typename Value , typename Mapped > struct KeyGetterForTypeImpl < HashJoin : : Type : : keys128 , Value , Mapped >
2019-01-24 14:56:04 +00:00
{
2021-02-04 14:46:36 +00:00
using Type = ColumnsHashing : : HashMethodKeysFixed < Value , UInt128 , Mapped , false , false , false , use_offset > ;
2019-01-24 14:56:04 +00:00
} ;
2020-04-07 09:48:47 +00:00
template < typename Value , typename Mapped > struct KeyGetterForTypeImpl < HashJoin : : Type : : keys256 , Value , Mapped >
2019-01-24 14:56:04 +00:00
{
2021-04-25 09:30:43 +00:00
using Type = ColumnsHashing : : HashMethodKeysFixed < Value , UInt256 , Mapped , false , false , false , use_offset > ;
2019-01-24 14:56:04 +00:00
} ;
2020-04-07 09:48:47 +00:00
template < typename Value , typename Mapped > struct KeyGetterForTypeImpl < HashJoin : : Type : : hashed , Value , Mapped >
2019-01-24 14:56:04 +00:00
{
2021-02-04 14:46:36 +00:00
using Type = ColumnsHashing : : HashMethodHashed < Value , Mapped , false , use_offset > ;
2019-01-24 14:56:04 +00:00
} ;
2020-04-07 09:48:47 +00:00
template < HashJoin : : Type type , typename Data >
2019-01-24 14:56:04 +00:00
struct KeyGetterForType
{
using Value = typename Data : : value_type ;
using Mapped_t = typename Data : : mapped_type ;
using Mapped = std : : conditional_t < std : : is_const_v < Data > , const Mapped_t , Mapped_t > ;
using Type = typename KeyGetterForTypeImpl < type , Value , Mapped > : : Type ;
} ;
2017-03-28 06:51:22 +00:00
2021-09-06 10:59:18 +00:00
void HashJoin : : dataMapInit ( MapsVariant & map )
2014-06-18 20:08:31 +00:00
{
2022-07-08 13:11:27 +00:00
2022-08-04 15:15:49 +00:00
if ( kind = = JoinKind : : Cross )
2015-07-23 20:23:24 +00:00
return ;
2021-06-25 12:03:10 +00:00
joinDispatchInit ( kind , strictness , map ) ;
joinDispatch ( kind , strictness , map , [ & ] ( auto , auto , auto & map_ ) { map_ . create ( data - > type ) ; } ) ;
2014-06-18 20:08:31 +00:00
}
2021-01-04 23:49:31 +00:00
bool HashJoin : : empty ( ) const
{
return data - > type = = Type : : EMPTY ;
}
bool HashJoin : : alwaysReturnsEmptySet ( ) const
2020-12-30 13:52:37 +00:00
{
2022-07-08 13:11:27 +00:00
return isInnerOrRight ( getKind ( ) ) & & data - > empty ;
2020-12-30 13:52:37 +00:00
}
2021-02-24 16:19:04 +00:00
size_t HashJoin : : getTotalRowCount ( ) const
2020-12-30 13:52:37 +00:00
{
2022-12-20 12:50:27 +00:00
if ( ! data )
return 0 ;
2015-04-17 08:46:06 +00:00
size_t res = 0 ;
2017-04-01 07:20:54 +00:00
2019-12-19 15:50:28 +00:00
if ( data - > type = = Type : : CROSS )
2015-07-23 20:23:24 +00:00
{
2019-12-19 15:50:28 +00:00
for ( const auto & block : data - > blocks )
2017-01-02 20:42:49 +00:00
res + = block . rows ( ) ;
2015-07-23 20:23:24 +00:00
}
2022-07-08 13:11:27 +00:00
else
2015-07-23 20:23:24 +00:00
{
2021-06-25 12:03:10 +00:00
for ( const auto & map : data - > maps )
{
joinDispatch ( kind , strictness , map , [ & ] ( auto , auto , auto & map_ ) { res + = map_ . getTotalRowCount ( data - > type ) ; } ) ;
}
2015-07-23 20:23:24 +00:00
}
2017-04-01 07:20:54 +00:00
2015-04-17 08:46:06 +00:00
return res ;
2014-06-18 20:08:31 +00:00
}
2021-02-24 16:19:04 +00:00
size_t HashJoin : : getTotalByteCount ( ) const
2014-06-18 20:08:31 +00:00
{
2022-12-20 12:50:27 +00:00
if ( ! data )
return 0 ;
2022-12-19 15:19:49 +00:00
# ifdef NDEBUG
size_t debug_blocks_allocated_size = 0 ;
for ( const auto & block : data - > blocks )
debug_blocks_allocated_size + = block . allocatedBytes ( ) ;
if ( data - > blocks_allocated_size ! = debug_blocks_allocated_size )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " data->blocks_allocated_size != debug_blocks_allocated_size ({} != {}) " ,
data - > blocks_allocated_size , debug_blocks_allocated_size ) ;
size_t debug_blocks_nullmaps_allocated_size = 0 ;
for ( const auto & nullmap : data - > blocks_nullmaps )
debug_blocks_nullmaps_allocated_size + = nullmap . second - > allocatedBytes ( ) ;
if ( data - > blocks_nullmaps_allocated_size ! = debug_blocks_nullmaps_allocated_size )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " data->blocks_nullmaps_allocated_size != debug_blocks_nullmaps_allocated_size ({} != {}) " ,
data - > blocks_nullmaps_allocated_size , debug_blocks_nullmaps_allocated_size ) ;
# endif
2015-04-17 08:46:06 +00:00
size_t res = 0 ;
2017-04-01 07:20:54 +00:00
2022-12-19 15:19:49 +00:00
res + = data - > blocks_allocated_size ;
res + = data - > blocks_nullmaps_allocated_size ;
res + = data - > pool . size ( ) ;
if ( data - > type ! = Type : : CROSS )
2015-07-23 20:23:24 +00:00
{
2021-06-25 12:03:10 +00:00
for ( const auto & map : data - > maps )
{
joinDispatch ( kind , strictness , map , [ & ] ( auto , auto , auto & map_ ) { res + = map_ . getTotalByteCountImpl ( data - > type ) ; } ) ;
}
2015-07-23 20:23:24 +00:00
}
2015-04-17 08:46:06 +00:00
return res ;
2014-06-18 20:08:31 +00:00
}
2017-03-30 14:09:24 +00:00
namespace
{
2017-04-02 17:37:49 +00:00
/// Inserting an element into a hash table of the form `key -> reference to a string`, which will then be used by JOIN.
2017-03-30 14:09:24 +00:00
template < typename Map , typename KeyGetter >
2019-12-02 18:07:27 +00:00
struct Inserter
2017-03-30 14:09:24 +00:00
{
2020-04-07 09:48:47 +00:00
static ALWAYS_INLINE void insertOne ( const HashJoin & join , Map & map , KeyGetter & key_getter , Block * stored_block , size_t i ,
2019-12-02 18:07:27 +00:00
Arena & pool )
2017-03-30 14:09:24 +00:00
{
2019-01-24 14:56:04 +00:00
auto emplace_result = key_getter . emplaceKey ( map , i , pool ) ;
2017-04-01 07:20:54 +00:00
2019-05-07 18:21:44 +00:00
if ( emplace_result . isInserted ( ) | | join . anyTakeLastRow ( ) )
2019-01-24 14:56:04 +00:00
new ( & emplace_result . getMapped ( ) ) typename Map : : mapped_type ( stored_block , i ) ;
2017-03-30 14:09:24 +00:00
}
2017-04-01 07:20:54 +00:00
2020-04-07 09:48:47 +00:00
static ALWAYS_INLINE void insertAll ( const HashJoin & , Map & map , KeyGetter & key_getter , Block * stored_block , size_t i , Arena & pool )
2017-03-30 14:09:24 +00:00
{
2019-01-24 14:56:04 +00:00
auto emplace_result = key_getter . emplaceKey ( map , i , pool ) ;
2017-04-01 07:20:54 +00:00
2019-01-24 14:56:04 +00:00
if ( emplace_result . isInserted ( ) )
new ( & emplace_result . getMapped ( ) ) typename Map : : mapped_type ( stored_block , i ) ;
2017-03-30 14:09:24 +00:00
else
{
2019-05-14 14:40:43 +00:00
/// The first element of the list is stored in the value of the hash table, the rest in the pool.
2019-05-14 14:39:03 +00:00
emplace_result . getMapped ( ) . insert ( { stored_block , i } , pool ) ;
2017-03-30 14:09:24 +00:00
}
}
2017-04-01 07:20:54 +00:00
2020-04-07 09:48:47 +00:00
static ALWAYS_INLINE void insertAsof ( HashJoin & join , Map & map , KeyGetter & key_getter , Block * stored_block , size_t i , Arena & pool ,
2020-08-03 23:11:39 +00:00
const IColumn & asof_column )
2019-03-20 16:58:28 +00:00
{
auto emplace_result = key_getter . emplaceKey ( map , i , pool ) ;
2022-02-18 16:45:17 +00:00
typename Map : : mapped_type * time_series_map = & emplace_result . getMapped ( ) ;
2019-03-20 16:58:28 +00:00
2020-08-03 23:11:39 +00:00
TypeIndex asof_type = * join . getAsofType ( ) ;
2019-03-26 22:05:51 +00:00
if ( emplace_result . isInserted ( ) )
2022-02-18 15:16:29 +00:00
time_series_map = new ( time_series_map ) typename Map : : mapped_type ( createAsofRowRef ( asof_type , join . getAsofInequality ( ) ) ) ;
2022-02-18 10:02:14 +00:00
( * time_series_map ) - > insert ( asof_column , stored_block , i ) ;
2019-03-20 16:58:28 +00:00
}
} ;
2017-04-01 07:20:54 +00:00
2022-07-29 16:30:50 +00:00
template < JoinStrictness STRICTNESS , typename KeyGetter , typename Map , bool has_null_map >
2021-02-04 14:46:36 +00:00
size_t NO_INLINE insertFromBlockImplTypeCase (
2020-04-07 09:48:47 +00:00
HashJoin & join , Map & map , size_t rows , const ColumnRawPtrs & key_columns ,
2021-07-21 17:03:33 +00:00
const Sizes & key_sizes , Block * stored_block , ConstNullMapPtr null_map , UInt8ColumnDataPtr join_mask , Arena & pool )
2017-03-30 14:09:24 +00:00
{
2021-02-04 14:46:36 +00:00
[[maybe_unused]] constexpr bool mapped_one = std : : is_same_v < typename Map : : mapped_type , RowRef > ;
2022-07-29 16:30:50 +00:00
constexpr bool is_asof_join = STRICTNESS = = JoinStrictness : : Asof ;
2019-12-02 18:07:27 +00:00
2019-03-28 18:35:50 +00:00
const IColumn * asof_column [[maybe_unused]] = nullptr ;
2019-12-02 18:07:27 +00:00
if constexpr ( is_asof_join )
2020-08-03 23:11:39 +00:00
asof_column = key_columns . back ( ) ;
2019-03-28 18:35:50 +00:00
2019-12-02 18:07:27 +00:00
auto key_getter = createKeyGetter < KeyGetter , is_asof_join > ( key_columns , key_sizes ) ;
2017-04-01 07:20:54 +00:00
2017-03-30 14:09:24 +00:00
for ( size_t i = 0 ; i < rows ; + + i )
{
if ( has_null_map & & ( * null_map ) [ i ] )
continue ;
2017-04-01 07:20:54 +00:00
2021-07-21 17:03:33 +00:00
/// Check condition for right table from ON section
if ( join_mask & & ! ( * join_mask ) [ i ] )
continue ;
2019-12-02 18:07:27 +00:00
if constexpr ( is_asof_join )
2020-08-03 23:11:39 +00:00
Inserter < Map , KeyGetter > : : insertAsof ( join , map , key_getter , stored_block , i , pool , * asof_column ) ;
2019-12-02 18:07:27 +00:00
else if constexpr ( mapped_one )
Inserter < Map , KeyGetter > : : insertOne ( join , map , key_getter , stored_block , i , pool ) ;
2019-03-29 21:20:23 +00:00
else
2019-12-02 18:07:27 +00:00
Inserter < Map , KeyGetter > : : insertAll ( join , map , key_getter , stored_block , i , pool ) ;
2017-03-30 14:09:24 +00:00
}
2021-02-04 14:46:36 +00:00
return map . getBufferSizeInCells ( ) ;
2017-03-30 14:09:24 +00:00
}
2017-04-01 07:20:54 +00:00
2022-07-29 16:30:50 +00:00
template < JoinStrictness STRICTNESS , typename KeyGetter , typename Map >
2021-02-04 14:46:36 +00:00
size_t insertFromBlockImplType (
2020-04-07 09:48:47 +00:00
HashJoin & join , Map & map , size_t rows , const ColumnRawPtrs & key_columns ,
2021-07-21 17:03:33 +00:00
const Sizes & key_sizes , Block * stored_block , ConstNullMapPtr null_map , UInt8ColumnDataPtr join_mask , Arena & pool )
2017-03-30 14:09:24 +00:00
{
if ( null_map )
2021-07-21 17:03:33 +00:00
return insertFromBlockImplTypeCase < STRICTNESS , KeyGetter , Map , true > (
join , map , rows , key_columns , key_sizes , stored_block , null_map , join_mask , pool ) ;
2017-03-30 14:09:24 +00:00
else
2021-07-21 17:03:33 +00:00
return insertFromBlockImplTypeCase < STRICTNESS , KeyGetter , Map , false > (
join , map , rows , key_columns , key_sizes , stored_block , null_map , join_mask , pool ) ;
2017-03-30 14:09:24 +00:00
}
2017-04-01 07:20:54 +00:00
2022-07-29 16:30:50 +00:00
template < JoinStrictness STRICTNESS , typename Maps >
2021-02-04 14:46:36 +00:00
size_t insertFromBlockImpl (
2020-04-07 09:48:47 +00:00
HashJoin & join , HashJoin : : Type type , Maps & maps , size_t rows , const ColumnRawPtrs & key_columns ,
2021-07-21 17:03:33 +00:00
const Sizes & key_sizes , Block * stored_block , ConstNullMapPtr null_map , UInt8ColumnDataPtr join_mask , Arena & pool )
2017-03-30 14:09:24 +00:00
{
switch ( type )
{
2021-02-04 14:46:36 +00:00
case HashJoin : : Type : : EMPTY : return 0 ;
case HashJoin : : Type : : CROSS : return 0 ; /// Do nothing. We have already saved block, and it is enough.
2017-04-01 07:20:54 +00:00
2017-03-30 14:09:24 +00:00
# define M(TYPE) \
2020-04-07 09:48:47 +00:00
case HashJoin : : Type : : TYPE : \
2021-02-04 14:46:36 +00:00
return insertFromBlockImplType < STRICTNESS , typename KeyGetterForType < HashJoin : : Type : : TYPE , std : : remove_reference_t < decltype ( * maps . TYPE ) > > : : Type > ( \
2021-07-21 17:03:33 +00:00
join , * maps . TYPE , rows , key_columns , key_sizes , stored_block , null_map , join_mask , pool ) ; \
2017-03-30 14:09:24 +00:00
break ;
APPLY_FOR_JOIN_VARIANTS ( M )
# undef M
}
2022-10-07 19:20:14 +00:00
UNREACHABLE ( ) ;
2017-03-30 14:09:24 +00:00
}
}
2020-04-07 09:48:47 +00:00
void HashJoin : : initRightBlockStructure ( Block & saved_block_sample )
2019-11-01 17:41:07 +00:00
{
2022-05-30 13:50:22 +00:00
if ( isCrossOrComma ( kind ) )
{
/// cross join doesn't have keys, just add all columns
saved_block_sample = sample_block_with_columns_to_add . cloneEmpty ( ) ;
return ;
}
2021-09-06 10:59:18 +00:00
bool multiple_disjuncts = ! table_join - > oneDisjunct ( ) ;
2020-02-18 12:41:23 +00:00
/// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any).
2022-09-26 13:46:50 +00:00
bool save_key_columns = table_join - > isEnabledAlgorithm ( JoinAlgorithm : : AUTO ) | |
table_join - > isEnabledAlgorithm ( JoinAlgorithm : : GRACE_HASH ) | |
isRightOrFull ( kind ) | |
multiple_disjuncts ;
2020-02-18 12:41:23 +00:00
if ( save_key_columns )
2019-03-31 23:09:00 +00:00
{
2019-11-01 17:41:07 +00:00
saved_block_sample = right_table_keys . cloneEmpty ( ) ;
}
2022-07-29 16:30:50 +00:00
else if ( strictness = = JoinStrictness : : Asof )
2019-11-01 17:41:07 +00:00
{
/// Save ASOF key
saved_block_sample . insert ( right_table_keys . safeGetByPosition ( right_table_keys . columns ( ) - 1 ) ) ;
}
2019-03-31 23:09:00 +00:00
2019-11-01 17:41:07 +00:00
/// Save non key columns
for ( auto & column : sample_block_with_columns_to_add )
2021-06-25 12:03:10 +00:00
{
2022-11-01 18:24:13 +00:00
if ( auto * col = saved_block_sample . findByName ( column . name ) )
* col = column ;
else
2021-06-25 12:03:10 +00:00
saved_block_sample . insert ( column ) ;
}
2019-11-01 17:41:07 +00:00
}
2019-03-31 23:09:00 +00:00
2022-12-20 12:50:27 +00:00
Block HashJoin : : prepareRightBlock ( const Block & block , const Block & saved_block_sample_ )
2019-11-01 17:41:07 +00:00
{
2019-11-03 19:33:59 +00:00
Block structured_block ;
2022-12-20 12:50:27 +00:00
for ( const auto & sample_column : saved_block_sample_ . getColumnsWithTypeAndName ( ) )
2019-11-03 19:33:59 +00:00
{
2019-12-17 15:02:42 +00:00
ColumnWithTypeAndName column = block . getByName ( sample_column . name ) ;
2019-11-03 19:33:59 +00:00
if ( sample_column . column - > isNullable ( ) )
JoinCommon : : convertColumnToNullable ( column ) ;
2019-11-01 17:41:07 +00:00
2022-12-28 16:02:32 +00:00
if ( column . column - > lowCardinality ( ) & & ! sample_column . column - > lowCardinality ( ) )
{
column . column = column . column - > convertToFullColumnIfLowCardinality ( ) ;
column . type = removeLowCardinality ( column . type ) ;
}
2022-12-20 12:50:27 +00:00
/// There's no optimization for right side const columns. Remove constness if any.
column . column = recursiveRemoveSparse ( column . column - > convertToFullColumnIfConst ( ) ) ;
structured_block . insert ( std : : move ( column ) ) ;
2019-03-31 23:09:00 +00:00
}
2019-11-01 17:41:07 +00:00
2019-12-03 13:31:52 +00:00
return structured_block ;
2019-03-31 23:09:00 +00:00
}
2022-12-20 12:50:27 +00:00
Block HashJoin : : prepareRightBlock ( const Block & block ) const
{
return prepareRightBlock ( block , savedBlockSample ( ) ) ;
}
2020-04-07 09:48:47 +00:00
bool HashJoin : : addJoinedBlock ( const Block & source_block , bool check_limits )
2015-05-26 00:37:48 +00:00
{
2022-12-20 12:50:27 +00:00
if ( ! data )
throw Exception ( " Join data was released " , ErrorCodes : : LOGICAL_ERROR ) ;
2020-04-21 19:09:18 +00:00
/// RowRef::SizeT is uint32_t (not size_t) for hash table Cell memory efficiency.
2020-04-21 19:01:34 +00:00
/// It's possible to split bigger blocks and insert them by parts here. But it would be a dead code.
if ( unlikely ( source_block . rows ( ) > std : : numeric_limits < RowRef : : SizeT > : : max ( ) ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " Too many rows in right table block for HashJoin: {} " , source_block . rows ( ) ) ;
2020-04-21 19:01:34 +00:00
2022-12-27 10:13:12 +00:00
size_t rows = source_block . rows ( ) ;
2019-12-17 15:02:42 +00:00
2022-12-27 17:49:17 +00:00
ColumnPtrMap all_key_columns = JoinCommon : : materializeColumnsInplaceMap ( source_block , table_join - > getAllNames ( JoinTableSide : : Right ) ) ;
2021-07-21 17:03:33 +00:00
2022-12-27 10:13:12 +00:00
Block block_to_save = prepareRightBlock ( source_block ) ;
2019-12-03 14:30:51 +00:00
size_t total_rows = 0 ;
size_t total_bytes = 0 ;
2019-07-04 12:38:54 +00:00
{
2021-09-29 17:30:07 +00:00
if ( storage_join_lock )
2023-01-23 21:13:58 +00:00
throw DB : : Exception ( ErrorCodes : : LOGICAL_ERROR , " addJoinedBlock called when HashJoin locked to prevent updates " ) ;
2019-12-03 13:31:52 +00:00
2022-12-27 10:13:12 +00:00
data - > blocks_allocated_size + = block_to_save . allocatedBytes ( ) ;
data - > blocks . emplace_back ( std : : move ( block_to_save ) ) ;
2019-12-19 15:50:28 +00:00
Block * stored_block = & data - > blocks . back ( ) ;
2019-12-03 13:31:52 +00:00
2019-12-03 14:30:51 +00:00
if ( rows )
2019-12-19 15:50:28 +00:00
data - > empty = false ;
2019-10-12 10:06:07 +00:00
2021-09-06 10:59:18 +00:00
bool multiple_disjuncts = ! table_join - > oneDisjunct ( ) ;
const auto & onexprs = table_join - > getClauses ( ) ;
for ( size_t onexpr_idx = 0 ; onexpr_idx < onexprs . size ( ) ; + + onexpr_idx )
2015-07-23 20:23:24 +00:00
{
2021-09-06 10:59:18 +00:00
ColumnRawPtrs key_columns ;
for ( const auto & name : onexprs [ onexpr_idx ] . key_names_right )
2022-12-27 17:49:17 +00:00
key_columns . push_back ( all_key_columns [ name ] . get ( ) ) ;
2021-06-25 12:03:10 +00:00
/// We will insert to the map only keys, where all components are not NULL.
2021-09-06 10:59:18 +00:00
ConstNullMapPtr null_map { } ;
ColumnPtr null_map_holder = extractNestedColumnsAndNullMap ( key_columns , null_map ) ;
2021-06-25 12:03:10 +00:00
2021-09-06 10:59:18 +00:00
/// If RIGHT or FULL save blocks with nulls for NotJoinedBlocks
2021-06-25 12:03:10 +00:00
UInt8 save_nullmap = 0 ;
2021-09-06 10:59:18 +00:00
if ( isRightOrFull ( kind ) & & null_map )
2019-12-03 14:30:51 +00:00
{
2021-09-06 10:59:18 +00:00
/// Save rows with NULL keys
for ( size_t i = 0 ; ! save_nullmap & & i < null_map - > size ( ) ; + + i )
save_nullmap | = ( * null_map ) [ i ] ;
2021-06-25 12:03:10 +00:00
}
2019-07-04 12:38:54 +00:00
2022-12-27 10:13:12 +00:00
auto join_mask_col = JoinCommon : : getColumnAsMask ( source_block , onexprs [ onexpr_idx ] . condColumnNames ( ) . second ) ;
2021-08-05 11:35:16 +00:00
/// Save blocks that do not hold conditions in ON section
ColumnUInt8 : : MutablePtr not_joined_map = nullptr ;
2022-12-28 13:20:58 +00:00
if ( ! multiple_disjuncts & & isRightOrFull ( kind ) & & join_mask_col . hasData ( ) )
2021-08-05 11:35:16 +00:00
{
2021-11-08 12:36:34 +00:00
const auto & join_mask = join_mask_col . getData ( ) ;
2021-08-05 11:35:16 +00:00
/// Save rows that do not hold conditions
2022-12-20 12:50:27 +00:00
not_joined_map = ColumnUInt8 : : create ( rows , 0 ) ;
2021-11-08 12:36:34 +00:00
for ( size_t i = 0 , sz = join_mask - > size ( ) ; i < sz ; + + i )
2021-08-05 11:35:16 +00:00
{
/// Condition hold, do not save row
2021-11-08 12:36:34 +00:00
if ( ( * join_mask ) [ i ] )
2021-08-05 11:35:16 +00:00
continue ;
2019-12-03 14:30:51 +00:00
2021-08-05 11:35:16 +00:00
/// NULL key will be saved anyway because, do not save twice
2021-09-06 10:59:18 +00:00
if ( save_nullmap & & ( * null_map ) [ i ] )
2021-08-05 11:35:16 +00:00
continue ;
2021-07-21 17:03:33 +00:00
2021-08-05 11:35:16 +00:00
not_joined_map - > getData ( ) [ i ] = 1 ;
}
}
2022-07-29 16:30:50 +00:00
if ( kind ! = JoinKind : : Cross )
2021-09-06 10:59:18 +00:00
{
joinDispatch ( kind , strictness , data - > maps [ onexpr_idx ] , [ & ] ( auto kind_ , auto strictness_ , auto & map )
{
size_t size = insertFromBlockImpl < strictness_ > (
* this , data - > type , map , rows , key_columns , key_sizes [ onexpr_idx ] , stored_block , null_map ,
2021-11-08 12:36:34 +00:00
/// If mask is false constant, rows are added to hashmap anyway. It's not a happy-flow, so this case is not optimized
join_mask_col . getData ( ) ,
2021-09-06 10:59:18 +00:00
data - > pool ) ;
if ( multiple_disjuncts )
used_flags . reinit < kind_ , strictness_ > ( stored_block ) ;
else
/// Number of buckets + 1 value from zero storage
used_flags . reinit < kind_ , strictness_ > ( size + 1 ) ;
} ) ;
}
if ( ! multiple_disjuncts & & save_nullmap )
2022-12-19 15:19:49 +00:00
{
data - > blocks_nullmaps_allocated_size + = null_map_holder - > allocatedBytes ( ) ;
2021-09-06 10:59:18 +00:00
data - > blocks_nullmaps . emplace_back ( stored_block , null_map_holder ) ;
2022-12-19 15:19:49 +00:00
}
2021-09-06 10:59:18 +00:00
if ( ! multiple_disjuncts & & not_joined_map )
2022-12-19 15:19:49 +00:00
{
data - > blocks_nullmaps_allocated_size + = not_joined_map - > allocatedBytes ( ) ;
2021-08-05 11:35:16 +00:00
data - > blocks_nullmaps . emplace_back ( stored_block , std : : move ( not_joined_map ) ) ;
2022-12-19 15:19:49 +00:00
}
2021-09-06 10:59:18 +00:00
if ( ! check_limits )
return true ;
2020-02-17 17:08:31 +00:00
2021-09-06 10:59:18 +00:00
/// TODO: Do not calculate them every time
total_rows = getTotalRowCount ( ) ;
total_bytes = getTotalByteCount ( ) ;
2021-06-25 12:03:10 +00:00
}
2019-07-04 12:38:54 +00:00
}
2019-07-03 19:06:34 +00:00
2019-12-03 14:30:51 +00:00
return table_join - > sizeLimits ( ) . check ( total_rows , total_bytes , " JOIN " , ErrorCodes : : SET_SIZE_LIMIT_EXCEEDED ) ;
2014-06-12 02:31:30 +00:00
}
2017-03-30 14:09:24 +00:00
namespace
2014-06-19 18:15:46 +00:00
{
2019-03-20 14:41:35 +00:00
2021-09-06 10:59:18 +00:00
struct JoinOnKeyColumns
{
2021-09-09 09:47:08 +00:00
Names key_names ;
2021-09-06 10:59:18 +00:00
Columns materialized_keys_holder ;
ColumnRawPtrs key_columns ;
ConstNullMapPtr null_map ;
2021-09-09 09:47:08 +00:00
ColumnPtr null_map_holder ;
2021-09-06 10:59:18 +00:00
/// Only rows where mask == true can be joined
2021-11-08 12:36:34 +00:00
JoinCommon : : JoinMask join_mask_column ;
2021-09-06 10:59:18 +00:00
Sizes key_sizes ;
explicit JoinOnKeyColumns ( const Block & block , const Names & key_names_ , const String & cond_column_name , const Sizes & key_sizes_ )
: key_names ( key_names_ )
, materialized_keys_holder ( JoinCommon : : materializeColumns ( block , key_names ) ) /// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them.
, key_columns ( JoinCommon : : getRawPointers ( materialized_keys_holder ) )
2021-09-09 09:47:08 +00:00
, null_map ( nullptr )
2021-09-06 10:59:18 +00:00
, null_map_holder ( extractNestedColumnsAndNullMap ( key_columns , null_map ) )
, join_mask_column ( JoinCommon : : getColumnAsMask ( block , cond_column_name ) )
, key_sizes ( key_sizes_ )
{
}
2021-11-08 12:36:34 +00:00
bool isRowFiltered ( size_t i ) const { return join_mask_column . isRowFiltered ( i ) ; }
2021-09-06 10:59:18 +00:00
} ;
2019-03-20 14:41:35 +00:00
2019-03-20 14:49:05 +00:00
class AddedColumns
{
public :
2021-04-29 14:30:02 +00:00
struct TypeAndName
{
DataTypePtr type ;
String name ;
String qualified_name ;
TypeAndName ( DataTypePtr type_ , const String & name_ , const String & qualified_name_ )
2021-06-15 08:34:53 +00:00
: type ( type_ ) , name ( name_ ) , qualified_name ( qualified_name_ )
{
}
2021-04-29 14:30:02 +00:00
} ;
2019-03-20 14:41:35 +00:00
2021-06-15 08:34:53 +00:00
AddedColumns (
const Block & block_with_columns_to_add ,
const Block & block ,
2022-05-30 12:59:47 +00:00
const Block & saved_block_sample ,
2021-06-15 08:34:53 +00:00
const HashJoin & join ,
2021-09-06 10:59:18 +00:00
std : : vector < JoinOnKeyColumns > & & join_on_keys_ ,
2021-06-15 08:34:53 +00:00
bool is_asof_join ,
bool is_join_get_ )
2021-09-06 10:59:18 +00:00
: join_on_keys ( join_on_keys_ )
2019-11-06 19:39:52 +00:00
, rows_to_add ( block . rows ( ) )
2022-05-30 15:57:10 +00:00
, sample_block ( saved_block_sample )
2021-06-15 08:34:53 +00:00
, is_join_get ( is_join_get_ )
2019-03-20 14:49:05 +00:00
{
2020-08-03 23:11:39 +00:00
size_t num_columns_to_add = block_with_columns_to_add . columns ( ) ;
if ( is_asof_join )
+ + num_columns_to_add ;
2019-03-20 14:41:35 +00:00
2019-03-20 15:15:44 +00:00
columns . reserve ( num_columns_to_add ) ;
type_name . reserve ( num_columns_to_add ) ;
right_indexes . reserve ( num_columns_to_add ) ;
2019-03-20 14:41:35 +00:00
2020-04-22 06:01:33 +00:00
for ( const auto & src_column : block_with_columns_to_add )
2019-03-20 15:15:44 +00:00
{
2021-05-10 13:39:32 +00:00
/// Column names `src_column.name` and `qualified_name` can differ for StorageJoin,
2021-04-29 14:30:02 +00:00
/// because it uses not qualified right block column names
auto qualified_name = join . getTableJoin ( ) . renamedRightColumnName ( src_column . name ) ;
2020-04-07 14:52:32 +00:00
/// Don't insert column if it's in left block
2021-04-29 14:30:02 +00:00
if ( ! block . has ( qualified_name ) )
addColumn ( src_column , qualified_name ) ;
2019-03-31 21:14:43 +00:00
}
2020-08-03 23:11:39 +00:00
if ( is_asof_join )
{
2021-09-06 10:59:18 +00:00
assert ( join_on_keys . size ( ) = = 1 ) ;
2020-08-03 23:11:39 +00:00
const ColumnWithTypeAndName & right_asof_column = join . rightAsofKeyColumn ( ) ;
2021-04-29 14:30:02 +00:00
addColumn ( right_asof_column , right_asof_column . name ) ;
2021-09-09 09:47:08 +00:00
left_asof_key = join_on_keys [ 0 ] . key_columns . back ( ) ;
2020-08-03 23:11:39 +00:00
}
2019-03-31 21:14:43 +00:00
2019-04-01 10:35:37 +00:00
for ( auto & tn : type_name )
2021-04-29 14:30:02 +00:00
right_indexes . push_back ( saved_block_sample . getPositionByName ( tn . name ) ) ;
2019-03-20 14:49:05 +00:00
}
2017-04-01 07:20:54 +00:00
2019-03-20 15:15:44 +00:00
size_t size ( ) const { return columns . size ( ) ; }
2019-03-20 14:49:05 +00:00
ColumnWithTypeAndName moveColumn ( size_t i )
{
2021-04-29 14:30:02 +00:00
return ColumnWithTypeAndName ( std : : move ( columns [ i ] ) , type_name [ i ] . type , type_name [ i ] . qualified_name ) ;
2019-03-20 14:49:05 +00:00
}
2019-03-20 14:41:35 +00:00
2022-05-30 15:57:10 +00:00
static void assertBlockEqualsStructureUpToLowCard ( const Block & lhs_block , const Block & rhs_block )
{
if ( lhs_block . columns ( ) ! = rhs_block . columns ( ) )
2022-10-19 10:20:41 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Different number of columns in blocks [{}] and [{}] " ,
lhs_block . dumpStructure ( ) , rhs_block . dumpStructure ( ) ) ;
2022-05-30 15:57:10 +00:00
for ( size_t i = 0 ; i < lhs_block . columns ( ) ; + + i )
{
const auto & lhs = lhs_block . getByPosition ( i ) ;
const auto & rhs = rhs_block . getByPosition ( i ) ;
if ( lhs . name ! = rhs . name )
2023-01-11 16:17:21 +00:00
throw DB : : Exception ( ErrorCodes : : LOGICAL_ERROR , " Block structure mismatch: [{}] != [{}] ({} != {}) " ,
lhs_block . dumpStructure ( ) , rhs_block . dumpStructure ( ) , lhs . name , rhs . name ) ;
2022-05-30 15:57:10 +00:00
const auto & ltype = recursiveRemoveLowCardinality ( lhs . type ) ;
const auto & rtype = recursiveRemoveLowCardinality ( rhs . type ) ;
if ( ! ltype - > equals ( * rtype ) )
2023-01-11 16:17:21 +00:00
throw DB : : Exception ( ErrorCodes : : LOGICAL_ERROR , " Block structure mismatch: [{}] != [{}] ({} != {}) " ,
lhs_block . dumpStructure ( ) , rhs_block . dumpStructure ( ) , ltype - > getName ( ) , rtype - > getName ( ) ) ;
2022-05-30 15:57:10 +00:00
const auto & lcol = recursiveRemoveLowCardinality ( lhs . column ) ;
const auto & rcol = recursiveRemoveLowCardinality ( rhs . column ) ;
if ( lcol - > getDataType ( ) ! = rcol - > getDataType ( ) )
2023-01-11 16:17:21 +00:00
throw DB : : Exception ( ErrorCodes : : LOGICAL_ERROR , " Block structure mismatch: [{}] != [{}] ({} != {}) " ,
lhs_block . dumpStructure ( ) , rhs_block . dumpStructure ( ) , lcol - > getDataType ( ) , rcol - > getDataType ( ) ) ;
2022-05-30 15:57:10 +00:00
}
}
2019-11-08 16:13:43 +00:00
template < bool has_defaults >
2022-05-31 16:02:06 +00:00
void appendFromBlock ( const Block & block , size_t row_num )
2019-03-20 14:49:05 +00:00
{
2019-11-08 16:13:43 +00:00
if constexpr ( has_defaults )
applyLazyDefaults ( ) ;
2022-05-31 16:02:06 +00:00
# ifndef NDEBUG
2022-05-30 15:57:10 +00:00
/// Like assertBlocksHaveEqualStructure but doesn't check low cardinality
assertBlockEqualsStructureUpToLowCard ( sample_block , block ) ;
2022-05-31 16:02:06 +00:00
# else
UNUSED ( assertBlockEqualsStructureUpToLowCard ) ;
# endif
2022-05-30 12:59:47 +00:00
2021-06-15 08:34:53 +00:00
if ( is_join_get )
{
/// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin.
for ( size_t j = 0 , size = right_indexes . size ( ) ; j < size ; + + j )
{
2022-06-14 13:14:18 +00:00
const auto & column_from_block = block . getByPosition ( right_indexes [ j ] ) ;
2022-03-25 11:12:02 +00:00
if ( auto * nullable_col = typeid_cast < ColumnNullable * > ( columns [ j ] . get ( ) ) ;
nullable_col & & ! column_from_block . column - > isNullable ( ) )
nullable_col - > insertFromNotNullable ( * column_from_block . column , row_num ) ;
2022-06-14 13:14:18 +00:00
else if ( auto * lowcard_col = typeid_cast < ColumnLowCardinality * > ( columns [ j ] . get ( ) ) ;
lowcard_col & & ! typeid_cast < const ColumnLowCardinality * > ( column_from_block . column . get ( ) ) )
lowcard_col - > insertFromFullColumn ( * column_from_block . column , row_num ) ;
2021-06-15 08:34:53 +00:00
else
2022-03-25 11:12:02 +00:00
columns [ j ] - > insertFrom ( * column_from_block . column , row_num ) ;
2021-06-15 08:34:53 +00:00
}
}
else
{
for ( size_t j = 0 , size = right_indexes . size ( ) ; j < size ; + + j )
{
2022-06-14 13:14:18 +00:00
const auto & column_from_block = block . getByPosition ( right_indexes [ j ] ) ;
if ( auto * lowcard_col = typeid_cast < ColumnLowCardinality * > ( columns [ j ] . get ( ) ) ;
lowcard_col & & ! typeid_cast < const ColumnLowCardinality * > ( column_from_block . column . get ( ) ) )
lowcard_col - > insertFromFullColumn ( * column_from_block . column , row_num ) ;
else
columns [ j ] - > insertFrom ( * column_from_block . column , row_num ) ;
2021-06-15 08:34:53 +00:00
}
}
2019-03-31 21:14:43 +00:00
}
2019-03-20 16:58:28 +00:00
2019-03-20 14:49:05 +00:00
void appendDefaultRow ( )
2019-03-20 14:41:35 +00:00
{
2019-11-06 19:39:52 +00:00
+ + lazy_defaults_count ;
2019-03-20 14:49:05 +00:00
}
2019-03-20 14:41:35 +00:00
2019-11-06 19:39:52 +00:00
void applyLazyDefaults ( )
{
if ( lazy_defaults_count )
{
2021-01-28 08:26:10 +00:00
for ( size_t j = 0 , size = right_indexes . size ( ) ; j < size ; + + j )
2021-04-29 14:30:02 +00:00
JoinCommon : : addDefaultValues ( * columns [ j ] , type_name [ j ] . type , lazy_defaults_count ) ;
2019-11-06 19:39:52 +00:00
lazy_defaults_count = 0 ;
}
}
2020-08-03 23:11:39 +00:00
const IColumn & leftAsofKey ( ) const { return * left_asof_key ; }
2021-09-06 10:59:18 +00:00
std : : vector < JoinOnKeyColumns > join_on_keys ;
2021-11-08 12:36:34 +00:00
2019-11-06 19:39:52 +00:00
size_t rows_to_add ;
std : : unique_ptr < IColumn : : Offsets > offsets_to_replicate ;
2020-08-03 23:11:39 +00:00
bool need_filter = false ;
2019-11-06 19:39:52 +00:00
2019-03-31 19:03:57 +00:00
private :
2021-04-29 14:30:02 +00:00
std : : vector < TypeAndName > type_name ;
2019-03-31 19:03:57 +00:00
MutableColumns columns ;
std : : vector < size_t > right_indexes ;
2019-11-06 19:39:52 +00:00
size_t lazy_defaults_count = 0 ;
2020-08-03 23:11:39 +00:00
/// for ASOF
const IColumn * left_asof_key = nullptr ;
2022-05-30 12:59:47 +00:00
Block sample_block ;
2021-11-08 12:36:34 +00:00
2021-06-15 08:34:53 +00:00
bool is_join_get ;
2019-03-31 23:09:00 +00:00
2021-04-29 14:30:02 +00:00
void addColumn ( const ColumnWithTypeAndName & src_column , const std : : string & qualified_name )
2019-03-31 23:09:00 +00:00
{
2022-03-25 11:12:02 +00:00
2019-03-31 23:09:00 +00:00
columns . push_back ( src_column . column - > cloneEmpty ( ) ) ;
columns . back ( ) - > reserve ( src_column . column - > size ( ) ) ;
2021-04-29 14:30:02 +00:00
type_name . emplace_back ( src_column . type , src_column . name , qualified_name ) ;
2019-03-31 23:09:00 +00:00
}
2019-03-20 15:15:44 +00:00
} ;
2017-04-01 07:20:54 +00:00
2022-07-29 16:30:50 +00:00
template < JoinKind KIND , JoinStrictness STRICTNESS >
2021-06-25 12:03:10 +00:00
struct JoinFeatures
{
2022-07-29 16:30:50 +00:00
static constexpr bool is_any_join = STRICTNESS = = JoinStrictness : : Any ;
static constexpr bool is_any_or_semi_join = STRICTNESS = = JoinStrictness : : Any | | STRICTNESS = = JoinStrictness : : RightAny | | ( STRICTNESS = = JoinStrictness : : Semi & & KIND = = JoinKind : : Left ) ;
static constexpr bool is_all_join = STRICTNESS = = JoinStrictness : : All ;
static constexpr bool is_asof_join = STRICTNESS = = JoinStrictness : : Asof ;
static constexpr bool is_semi_join = STRICTNESS = = JoinStrictness : : Semi ;
static constexpr bool is_anti_join = STRICTNESS = = JoinStrictness : : Anti ;
static constexpr bool left = KIND = = JoinKind : : Left ;
static constexpr bool right = KIND = = JoinKind : : Right ;
static constexpr bool inner = KIND = = JoinKind : : Inner ;
static constexpr bool full = KIND = = JoinKind : : Full ;
2021-06-25 12:03:10 +00:00
static constexpr bool need_replication = is_all_join | | ( is_any_join & & right ) | | ( is_semi_join & & right ) ;
static constexpr bool need_filter = ! need_replication & & ( inner | | right | | ( is_semi_join & & left ) | | ( is_anti_join & & left ) ) ;
static constexpr bool add_missing = ( left | | full ) & & ! is_semi_join ;
static constexpr bool need_flags = MapGetter < KIND , STRICTNESS > : : flagged ;
} ;
template < bool multiple_disjuncts >
class KnownRowsHolder ;
2017-04-01 07:20:54 +00:00
2021-07-23 19:55:36 +00:00
/// Keep already joined rows to prevent duplication if many disjuncts
/// if for a particular pair of rows condition looks like TRUE or TRUE or TRUE
/// we want to have it once in resultset
2021-06-25 12:03:10 +00:00
template < >
class KnownRowsHolder < true >
{
public :
2021-07-23 19:55:36 +00:00
using Type = std : : pair < const Block * , DB : : RowRef : : SizeT > ;
2021-06-25 12:03:10 +00:00
private :
2021-07-23 19:55:36 +00:00
static const size_t MAX_LINEAR = 16 ; // threshold to switch from Array to Set
using ArrayHolder = std : : array < Type , MAX_LINEAR > ;
using SetHolder = std : : set < Type > ;
using SetHolderPtr = std : : unique_ptr < SetHolder > ;
2021-06-25 12:03:10 +00:00
2021-07-23 19:55:36 +00:00
ArrayHolder array_holder ;
SetHolderPtr set_holder_ptr ;
2021-06-25 12:03:10 +00:00
size_t items ;
public :
KnownRowsHolder ( )
: items ( 0 )
{
}
template < class InputIt >
void add ( InputIt from , InputIt to )
{
2021-07-23 19:55:36 +00:00
const size_t new_items = std : : distance ( from , to ) ;
2021-06-25 12:03:10 +00:00
if ( items + new_items < = MAX_LINEAR )
{
2021-07-23 19:55:36 +00:00
std : : copy ( from , to , & array_holder [ items ] ) ;
2021-06-25 12:03:10 +00:00
}
else
{
if ( items < = MAX_LINEAR )
{
2021-07-23 19:55:36 +00:00
set_holder_ptr = std : : make_unique < SetHolder > ( ) ;
set_holder_ptr - > insert ( std : : cbegin ( array_holder ) , std : : cbegin ( array_holder ) + items ) ;
2021-06-25 12:03:10 +00:00
}
2021-07-23 19:55:36 +00:00
set_holder_ptr - > insert ( from , to ) ;
2021-06-25 12:03:10 +00:00
}
items + = new_items ;
}
template < class Needle >
bool isKnown ( const Needle & needle )
{
return items < = MAX_LINEAR
2021-07-23 19:55:36 +00:00
? std : : find ( std : : cbegin ( array_holder ) , std : : cbegin ( array_holder ) + items , needle ) ! = std : : cbegin ( array_holder ) + items
: set_holder_ptr - > find ( needle ) ! = set_holder_ptr - > end ( ) ;
2021-06-25 12:03:10 +00:00
}
} ;
template < >
class KnownRowsHolder < false >
{
public :
template < class InputIt >
void add ( InputIt , InputIt )
{
}
template < class Needle >
static bool isKnown ( const Needle & )
{
return false ;
}
} ;
template < typename Map , bool add_missing , bool multiple_disjuncts >
2021-08-17 10:27:23 +00:00
void addFoundRowAll (
const typename Map : : mapped_type & mapped ,
AddedColumns & added ,
IColumn : : Offset & current_offset ,
KnownRowsHolder < multiple_disjuncts > & known_rows [[maybe_unused]] ,
JoinStuff : : JoinUsedFlags * used_flags [[maybe_unused]] )
2019-03-20 14:49:05 +00:00
{
2019-12-02 11:45:21 +00:00
if constexpr ( add_missing )
added . applyLazyDefaults ( ) ;
2017-04-01 07:20:54 +00:00
2021-06-25 12:03:10 +00:00
if constexpr ( multiple_disjuncts )
2019-03-19 16:53:36 +00:00
{
2021-06-25 12:03:10 +00:00
std : : unique_ptr < std : : vector < KnownRowsHolder < true > : : Type > > new_known_rows_ptr ;
for ( auto it = mapped . begin ( ) ; it . ok ( ) ; + + it )
{
if ( ! known_rows . isKnown ( std : : make_pair ( it - > block , it - > row_num ) ) )
{
added . appendFromBlock < false > ( * it - > block , it - > row_num ) ;
+ + current_offset ;
if ( ! new_known_rows_ptr )
{
new_known_rows_ptr = std : : make_unique < std : : vector < KnownRowsHolder < true > : : Type > > ( ) ;
}
new_known_rows_ptr - > push_back ( std : : make_pair ( it - > block , it - > row_num ) ) ;
2021-08-17 10:27:23 +00:00
if ( used_flags )
{
used_flags - > JoinStuff : : JoinUsedFlags : : setUsedOnce < true , multiple_disjuncts > (
FindResultImpl < const RowRef , false > ( * it , true , 0 ) ) ;
}
2021-06-25 12:03:10 +00:00
}
}
if ( new_known_rows_ptr )
{
known_rows . add ( std : : cbegin ( * new_known_rows_ptr ) , std : : cend ( * new_known_rows_ptr ) ) ;
}
}
else
{
for ( auto it = mapped . begin ( ) ; it . ok ( ) ; + + it )
{
added . appendFromBlock < false > ( * it - > block , it - > row_num ) ;
+ + current_offset ;
}
2019-03-19 16:53:36 +00:00
}
2022-05-16 18:59:27 +00:00
}
2017-04-01 07:20:54 +00:00
2019-12-03 12:55:20 +00:00
template < bool add_missing , bool need_offset >
2019-03-20 14:49:05 +00:00
void addNotFoundRow ( AddedColumns & added [[maybe_unused]], IColumn::Offset & current_offset [[maybe_unused]] )
{
2019-11-08 16:13:43 +00:00
if constexpr ( add_missing )
2017-03-30 14:09:24 +00:00
{
2019-03-20 14:49:05 +00:00
added . appendDefaultRow ( ) ;
2019-12-03 12:55:20 +00:00
if constexpr ( need_offset )
+ + current_offset ;
2019-03-20 14:49:05 +00:00
}
}
2019-03-19 16:53:36 +00:00
2019-12-03 12:55:20 +00:00
template < bool need_filter >
void setUsed ( IColumn : : Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]] )
{
if constexpr ( need_filter )
filter [ pos ] = 1 ;
}
2019-03-20 14:49:05 +00:00
/// Joins right table columns which indexes are present in right_indexes using specified map.
/// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS).
2022-07-29 16:30:50 +00:00
template < JoinKind KIND , JoinStrictness STRICTNESS , typename KeyGetter , typename Map , bool need_filter , bool has_null_map , bool multiple_disjuncts >
2021-02-04 14:46:36 +00:00
NO_INLINE IColumn : : Filter joinRightColumns (
2021-06-25 12:03:10 +00:00
std : : vector < KeyGetter > & & key_getter_vector ,
const std : : vector < const Map * > & mapv ,
2021-02-04 14:46:36 +00:00
AddedColumns & added_columns ,
JoinStuff : : JoinUsedFlags & used_flags [[maybe_unused]] )
2019-03-20 14:49:05 +00:00
{
2021-08-17 10:27:23 +00:00
constexpr JoinFeatures < KIND , STRICTNESS > jf ;
2019-11-07 21:32:44 +00:00
2019-11-06 19:39:52 +00:00
size_t rows = added_columns . rows_to_add ;
2019-12-03 12:55:20 +00:00
IColumn : : Filter filter ;
if constexpr ( need_filter )
filter = IColumn : : Filter ( rows , 0 ) ;
2017-04-01 07:20:54 +00:00
2019-03-20 14:49:05 +00:00
Arena pool ;
2021-06-25 12:03:10 +00:00
if constexpr ( jf . need_replication )
2019-11-11 11:42:10 +00:00
added_columns . offsets_to_replicate = std : : make_unique < IColumn : : Offsets > ( rows ) ;
2019-03-20 16:58:28 +00:00
IColumn : : Offset current_offset = 0 ;
2019-03-20 14:49:05 +00:00
for ( size_t i = 0 ; i < rows ; + + i )
{
2021-06-25 12:03:10 +00:00
bool right_row_found = false ;
bool null_element_found = false ;
KnownRowsHolder < multiple_disjuncts > known_rows ;
2021-09-06 10:59:18 +00:00
for ( size_t onexpr_idx = 0 ; onexpr_idx < added_columns . join_on_keys . size ( ) ; + + onexpr_idx )
2017-03-30 14:09:24 +00:00
{
2021-09-06 10:59:18 +00:00
const auto & join_keys = added_columns . join_on_keys [ onexpr_idx ] ;
2021-06-25 12:03:10 +00:00
if constexpr ( has_null_map )
2019-11-11 11:42:10 +00:00
{
2021-09-06 10:59:18 +00:00
if ( join_keys . null_map & & ( * join_keys . null_map ) [ i ] )
2021-06-25 12:03:10 +00:00
{
null_element_found = true ;
continue ;
}
2019-11-11 11:42:10 +00:00
}
2021-09-06 10:59:18 +00:00
bool row_acceptable = ! join_keys . isRowFiltered ( i ) ;
2021-06-25 12:03:10 +00:00
using FindResult = typename KeyGetter : : FindResult ;
2021-09-06 10:59:18 +00:00
auto find_result = row_acceptable ? key_getter_vector [ onexpr_idx ] . findKey ( * ( mapv [ onexpr_idx ] ) , i , pool ) : FindResult ( ) ;
2019-11-11 11:42:10 +00:00
2021-06-25 12:03:10 +00:00
if ( find_result . isFound ( ) )
2017-03-30 14:09:24 +00:00
{
2021-06-25 12:03:10 +00:00
right_row_found = true ;
auto & mapped = find_result . getMapped ( ) ;
if constexpr ( jf . is_asof_join )
{
const IColumn & left_asof_key = added_columns . leftAsofKey ( ) ;
2022-03-23 11:19:38 +00:00
auto row_ref = mapped - > findAsof ( left_asof_key , i ) ;
if ( row_ref . block )
2021-06-25 12:03:10 +00:00
{
setUsed < need_filter > ( filter , i ) ;
2021-08-17 10:27:23 +00:00
if constexpr ( multiple_disjuncts )
2022-03-23 11:19:38 +00:00
used_flags . template setUsed < jf . need_flags , multiple_disjuncts > ( row_ref . block , row_ref . row_num , 0 ) ;
2021-08-17 10:27:23 +00:00
else
used_flags . template setUsed < jf . need_flags , multiple_disjuncts > ( find_result ) ;
2022-03-23 11:19:38 +00:00
added_columns . appendFromBlock < jf . add_missing > ( * row_ref . block , row_ref . row_num ) ;
2021-06-25 12:03:10 +00:00
}
else
addNotFoundRow < jf . add_missing , jf . need_replication > ( added_columns , current_offset ) ;
}
else if constexpr ( jf . is_all_join )
2019-03-30 21:30:21 +00:00
{
2019-12-03 12:55:20 +00:00
setUsed < need_filter > ( filter , i ) ;
2021-06-25 12:03:10 +00:00
used_flags . template setUsed < jf . need_flags , multiple_disjuncts > ( find_result ) ;
2021-08-17 10:27:23 +00:00
auto used_flags_opt = jf . need_flags ? & used_flags : nullptr ;
addFoundRowAll < Map , jf . add_missing > ( mapped , added_columns , current_offset , known_rows , used_flags_opt ) ;
2019-03-30 21:30:21 +00:00
}
2021-06-25 12:03:10 +00:00
else if constexpr ( ( jf . is_any_join | | jf . is_semi_join ) & & jf . right )
{
/// Use first appeared left key + it needs left columns replication
bool used_once = used_flags . template setUsedOnce < jf . need_flags , multiple_disjuncts > ( find_result ) ;
if ( used_once )
{
2021-08-17 10:27:23 +00:00
auto used_flags_opt = jf . need_flags ? & used_flags : nullptr ;
2021-06-25 12:03:10 +00:00
setUsed < need_filter > ( filter , i ) ;
2021-08-17 10:27:23 +00:00
addFoundRowAll < Map , jf . add_missing > ( mapped , added_columns , current_offset , known_rows , used_flags_opt ) ;
2021-06-25 12:03:10 +00:00
}
}
2022-07-29 16:30:50 +00:00
else if constexpr ( jf . is_any_join & & KIND = = JoinKind : : Inner )
2021-06-25 12:03:10 +00:00
{
bool used_once = used_flags . template setUsedOnce < jf . need_flags , multiple_disjuncts > ( find_result ) ;
/// Use first appeared left key only
if ( used_once )
{
setUsed < need_filter > ( filter , i ) ;
added_columns . appendFromBlock < jf . add_missing > ( * mapped . block , mapped . row_num ) ;
}
2021-02-04 14:46:36 +00:00
2021-06-25 12:03:10 +00:00
break ;
}
else if constexpr ( jf . is_any_join & & jf . full )
{
/// TODO
}
else if constexpr ( jf . is_anti_join )
{
if constexpr ( jf . right & & jf . need_flags )
used_flags . template setUsed < jf . need_flags , multiple_disjuncts > ( find_result ) ;
}
else /// ANY LEFT, SEMI LEFT, old ANY (RightAny)
2019-03-26 20:13:15 +00:00
{
2019-12-03 12:55:20 +00:00
setUsed < need_filter > ( filter , i ) ;
2021-06-25 12:03:10 +00:00
used_flags . template setUsed < jf . need_flags , multiple_disjuncts > ( find_result ) ;
added_columns . appendFromBlock < jf . add_missing > ( * mapped . block , mapped . row_num ) ;
2021-08-25 10:18:29 +00:00
if ( jf . is_any_or_semi_join )
2021-06-25 12:03:10 +00:00
{
break ;
}
2019-03-20 16:58:28 +00:00
}
2017-03-30 14:09:24 +00:00
}
2021-09-06 10:59:18 +00:00
}
2021-06-25 12:03:10 +00:00
if constexpr ( has_null_map )
{
if ( ! right_row_found & & null_element_found )
2019-11-11 11:42:10 +00:00
{
2021-06-25 12:03:10 +00:00
addNotFoundRow < jf . add_missing , jf . need_replication > ( added_columns , current_offset ) ;
2021-02-04 14:46:36 +00:00
2021-06-25 12:03:10 +00:00
if constexpr ( jf . need_replication )
2019-11-07 21:32:44 +00:00
{
2021-09-06 10:59:18 +00:00
( * added_columns . offsets_to_replicate ) [ i ] = current_offset ;
2019-03-20 16:58:28 +00:00
}
2021-06-25 12:03:10 +00:00
continue ;
2019-11-11 11:42:10 +00:00
}
2017-03-30 14:09:24 +00:00
}
2021-06-25 12:03:10 +00:00
if ( ! right_row_found )
2019-12-02 18:07:27 +00:00
{
2021-06-25 12:03:10 +00:00
if constexpr ( jf . is_anti_join & & jf . left )
2019-12-03 12:55:20 +00:00
setUsed < need_filter > ( filter , i ) ;
2021-06-25 12:03:10 +00:00
addNotFoundRow < jf . add_missing , jf . need_replication > ( added_columns , current_offset ) ;
2017-03-30 14:09:24 +00:00
}
2017-04-01 07:20:54 +00:00
2021-06-25 12:03:10 +00:00
if constexpr ( jf . need_replication )
{
2021-09-06 10:59:18 +00:00
( * added_columns . offsets_to_replicate ) [ i ] = current_offset ;
2021-06-25 12:03:10 +00:00
}
2019-03-19 16:53:36 +00:00
}
2019-01-29 12:38:53 +00:00
2019-11-06 19:39:52 +00:00
added_columns . applyLazyDefaults ( ) ;
2019-11-11 11:42:10 +00:00
return filter ;
2019-03-20 14:49:05 +00:00
}
2019-03-20 13:38:17 +00:00
2022-07-29 16:30:50 +00:00
template < JoinKind KIND , JoinStrictness STRICTNESS , typename KeyGetter , typename Map , bool need_filter , bool has_null_map >
2021-06-25 12:03:10 +00:00
IColumn : : Filter joinRightColumnsSwitchMultipleDisjuncts (
std : : vector < KeyGetter > & & key_getter_vector ,
const std : : vector < const Map * > & mapv ,
AddedColumns & added_columns ,
JoinStuff : : JoinUsedFlags & used_flags [[maybe_unused]] )
{
return mapv . size ( ) > 1
2022-05-16 14:27:27 +00:00
? joinRightColumns < KIND , STRICTNESS , KeyGetter , Map , need_filter , has_null_map , true > ( std : : forward < std : : vector < KeyGetter > > ( key_getter_vector ) , mapv , added_columns , used_flags )
: joinRightColumns < KIND , STRICTNESS , KeyGetter , Map , need_filter , has_null_map , false > ( std : : forward < std : : vector < KeyGetter > > ( key_getter_vector ) , mapv , added_columns , used_flags ) ;
2021-06-25 12:03:10 +00:00
}
2022-07-29 16:30:50 +00:00
template < JoinKind KIND , JoinStrictness STRICTNESS , typename KeyGetter , typename Map >
2021-02-04 14:46:36 +00:00
IColumn : : Filter joinRightColumnsSwitchNullability (
2021-06-25 12:03:10 +00:00
std : : vector < KeyGetter > & & key_getter_vector ,
2021-09-06 10:59:18 +00:00
const std : : vector < const Map * > & mapv ,
2021-06-25 12:03:10 +00:00
AddedColumns & added_columns ,
JoinStuff : : JoinUsedFlags & used_flags )
2019-03-20 14:49:05 +00:00
{
2021-09-06 10:59:18 +00:00
bool has_null_map = std : : any_of ( added_columns . join_on_keys . begin ( ) , added_columns . join_on_keys . end ( ) ,
[ ] ( const auto & k ) { return k . null_map ; } ) ;
2019-12-03 12:55:20 +00:00
if ( added_columns . need_filter )
{
2021-09-06 10:59:18 +00:00
if ( has_null_map )
return joinRightColumnsSwitchMultipleDisjuncts < KIND , STRICTNESS , KeyGetter , Map , true , true > ( std : : forward < std : : vector < KeyGetter > > ( key_getter_vector ) , mapv , added_columns , used_flags ) ;
2019-12-03 12:55:20 +00:00
else
2021-09-06 10:59:18 +00:00
return joinRightColumnsSwitchMultipleDisjuncts < KIND , STRICTNESS , KeyGetter , Map , true , false > ( std : : forward < std : : vector < KeyGetter > > ( key_getter_vector ) , mapv , added_columns , used_flags ) ;
2019-12-03 12:55:20 +00:00
}
2019-03-20 14:49:05 +00:00
else
2019-12-03 12:55:20 +00:00
{
2021-09-06 10:59:18 +00:00
if ( has_null_map )
return joinRightColumnsSwitchMultipleDisjuncts < KIND , STRICTNESS , KeyGetter , Map , false , true > ( std : : forward < std : : vector < KeyGetter > > ( key_getter_vector ) , mapv , added_columns , used_flags ) ;
2019-12-03 12:55:20 +00:00
else
2021-09-06 10:59:18 +00:00
return joinRightColumnsSwitchMultipleDisjuncts < KIND , STRICTNESS , KeyGetter , Map , false , false > ( std : : forward < std : : vector < KeyGetter > > ( key_getter_vector ) , mapv , added_columns , used_flags ) ;
2019-12-03 12:55:20 +00:00
}
2019-03-20 14:49:05 +00:00
}
2019-03-19 16:53:36 +00:00
2022-07-29 16:30:50 +00:00
template < JoinKind KIND , JoinStrictness STRICTNESS , typename Maps >
2021-02-04 14:46:36 +00:00
IColumn : : Filter switchJoinRightColumns (
2021-09-06 10:59:18 +00:00
const std : : vector < const Maps * > & mapv ,
2021-06-23 10:13:11 +00:00
AddedColumns & added_columns ,
HashJoin : : Type type ,
JoinStuff : : JoinUsedFlags & used_flags )
2019-03-20 14:49:05 +00:00
{
2022-07-29 16:30:50 +00:00
constexpr bool is_asof_join = STRICTNESS = = JoinStrictness : : Asof ;
2019-03-20 14:49:05 +00:00
switch ( type )
{
2021-11-08 12:44:13 +00:00
case HashJoin : : Type : : EMPTY :
{
if constexpr ( ! is_asof_join )
{
using KeyGetter = KeyGetterEmpty < typename Maps : : MappedType > ;
std : : vector < KeyGetter > key_getter_vector ;
key_getter_vector . emplace_back ( ) ;
using MapTypeVal = typename KeyGetter : : MappedType ;
std : : vector < const MapTypeVal * > a_map_type_vector ;
a_map_type_vector . emplace_back ( ) ;
return joinRightColumnsSwitchNullability < KIND , STRICTNESS , KeyGetter > (
std : : move ( key_getter_vector ) , a_map_type_vector , added_columns , used_flags ) ;
}
throw Exception ( ErrorCodes : : UNSUPPORTED_JOIN_KEYS , " Unsupported JOIN keys. Type: {} " , type ) ;
}
2019-03-20 14:49:05 +00:00
# define M(TYPE) \
2020-04-07 09:48:47 +00:00
case HashJoin : : Type : : TYPE : \
2021-06-25 12:03:10 +00:00
{ \
2021-09-06 10:59:18 +00:00
using MapTypeVal = const typename std : : remove_reference_t < decltype ( Maps : : TYPE ) > : : element_type ; \
using KeyGetter = typename KeyGetterForType < HashJoin : : Type : : TYPE , MapTypeVal > : : Type ; \
2021-11-08 12:44:13 +00:00
std : : vector < const MapTypeVal * > a_map_type_vector ( mapv . size ( ) ) ; \
2021-06-25 12:03:10 +00:00
std : : vector < KeyGetter > key_getter_vector ; \
2021-09-06 10:59:18 +00:00
for ( size_t d = 0 ; d < added_columns . join_on_keys . size ( ) ; + + d ) \
{ \
const auto & join_on_key = added_columns . join_on_keys [ d ] ; \
2021-06-25 12:03:10 +00:00
a_map_type_vector [ d ] = mapv [ d ] - > TYPE . get ( ) ; \
2021-09-06 10:59:18 +00:00
key_getter_vector . push_back ( std : : move ( createKeyGetter < KeyGetter , is_asof_join > ( join_on_key . key_columns , join_on_key . key_sizes ) ) ) ; \
2021-06-25 12:03:10 +00:00
} \
2021-06-23 10:13:11 +00:00
return joinRightColumnsSwitchNullability < KIND , STRICTNESS , KeyGetter > ( \
2021-09-06 10:59:18 +00:00
std : : move ( key_getter_vector ) , a_map_type_vector , added_columns , used_flags ) ; \
2021-06-25 12:03:10 +00:00
}
2019-03-20 14:49:05 +00:00
APPLY_FOR_JOIN_VARIANTS ( M )
# undef M
default :
2021-11-08 12:44:13 +00:00
throw Exception ( ErrorCodes : : UNSUPPORTED_JOIN_KEYS , " Unsupported JOIN keys (type: {}) " , type) ;
2017-03-28 06:51:22 +00:00
}
}
2019-03-20 14:49:05 +00:00
} /// nameless
2022-07-29 16:30:50 +00:00
template < JoinKind KIND , JoinStrictness STRICTNESS , typename Maps >
2020-04-07 09:48:47 +00:00
void HashJoin : : joinBlockImpl (
2018-11-30 14:49:35 +00:00
Block & block ,
const Block & block_with_columns_to_add ,
2021-09-06 10:59:18 +00:00
const std : : vector < const Maps * > & maps_ ,
2021-06-15 08:34:53 +00:00
bool is_join_get ) const
2014-06-12 04:04:47 +00:00
{
2021-06-25 12:03:10 +00:00
constexpr JoinFeatures < KIND , STRICTNESS > jf ;
2017-04-01 07:20:54 +00:00
2021-09-06 10:59:18 +00:00
std : : vector < JoinOnKeyColumns > join_on_keys ;
const auto & onexprs = table_join - > getClauses ( ) ;
for ( size_t i = 0 ; i < onexprs . size ( ) ; + + i )
2021-06-25 12:03:10 +00:00
{
2021-09-06 10:59:18 +00:00
const auto & key_names = ! is_join_get ? onexprs [ i ] . key_names_left : onexprs [ i ] . key_names_right ;
join_on_keys . emplace_back ( block , key_names , onexprs [ i ] . condColumnNames ( ) . first , key_sizes [ i ] ) ;
2021-06-25 12:03:10 +00:00
}
2015-07-23 21:19:30 +00:00
size_t existing_columns = block . columns ( ) ;
2017-04-01 07:20:54 +00:00
2017-04-02 17:37:49 +00:00
/** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized.
* Because if they are constants , then in the " not joined " rows , they may have different values
* - default values , which can differ from the values of these constants .
2015-07-23 21:19:30 +00:00
*/
2021-06-25 12:03:10 +00:00
if constexpr ( jf . right | | jf . full )
2015-07-23 21:19:30 +00:00
{
2019-09-12 12:59:53 +00:00
materializeBlockInplace ( block ) ;
2015-07-23 21:19:30 +00:00
}
2017-04-01 07:20:54 +00:00
2017-12-14 12:21:01 +00:00
/** For LEFT/INNER JOIN, the saved blocks do not contain keys.
* For FULL / RIGHT JOIN , the saved blocks contain keys ;
* but they will not be used at this stage of joining ( and will be in ` AdderNonJoined ` ) , and they need to be skipped .
2019-03-31 23:09:00 +00:00
* For ASOF , the last column is used as the ASOF column
2017-12-14 12:21:01 +00:00
*/
2021-07-21 17:03:33 +00:00
AddedColumns added_columns (
block_with_columns_to_add ,
block ,
savedBlockSample ( ) ,
* this ,
2021-09-06 10:59:18 +00:00
std : : move ( join_on_keys ) ,
2021-06-25 12:03:10 +00:00
jf . is_asof_join ,
2021-07-21 17:03:33 +00:00
is_join_get ) ;
2019-12-03 12:55:20 +00:00
bool has_required_right_keys = ( required_right_keys . columns ( ) ! = 0 ) ;
2021-09-06 10:59:18 +00:00
added_columns . need_filter = jf . need_filter | | has_required_right_keys ;
2019-03-20 12:08:38 +00:00
2022-07-08 13:11:27 +00:00
IColumn : : Filter row_filter = switchJoinRightColumns < KIND , STRICTNESS > ( maps_ , added_columns , data - > type , used_flags ) ;
2017-04-01 07:20:54 +00:00
2019-11-06 19:39:52 +00:00
for ( size_t i = 0 ; i < added_columns . size ( ) ; + + i )
block . insert ( added_columns . moveColumn ( i ) ) ;
2019-03-27 13:37:11 +00:00
2019-07-31 20:19:52 +00:00
std : : vector < size_t > right_keys_to_replicate [[maybe_unused]] ;
2018-07-24 10:00:53 +00:00
2021-06-25 12:03:10 +00:00
if constexpr ( jf . need_filter )
2019-07-31 20:19:52 +00:00
{
/// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones.
for ( size_t i = 0 ; i < existing_columns ; + + i )
block . safeGetByPosition ( i ) . column = block . safeGetByPosition ( i ) . column - > filter ( row_filter , - 1 ) ;
2018-07-24 10:00:53 +00:00
2021-09-06 10:59:18 +00:00
/// Add join key columns from right block if needed using value from left table because of equality
2019-11-01 17:41:07 +00:00
for ( size_t i = 0 ; i < required_right_keys . columns ( ) ; + + i )
2018-07-24 10:00:53 +00:00
{
2019-11-01 17:41:07 +00:00
const auto & right_key = required_right_keys . getByPosition ( i ) ;
2021-06-25 12:03:10 +00:00
// renamed ???
if ( ! block . findByName ( right_key . name ) )
{
const auto & left_name = required_right_keys_sources [ i ] ;
/// asof column is already in block.
2021-09-06 10:59:18 +00:00
if ( jf . is_asof_join & & right_key . name = = table_join - > getOnlyClause ( ) . key_names_right . back ( ) )
2021-06-25 12:03:10 +00:00
continue ;
const auto & col = block . getByName ( left_name ) ;
2022-02-15 11:11:59 +00:00
bool is_nullable = JoinCommon : : isNullable ( right_key . type ) ;
2021-06-25 12:03:10 +00:00
auto right_col_name = getTableJoin ( ) . renamedRightColumnName ( right_key . name ) ;
ColumnWithTypeAndName right_col ( col . column , col . type , right_col_name ) ;
if ( right_col . type - > lowCardinality ( ) ! = right_key . type - > lowCardinality ( ) )
JoinCommon : : changeLowCardinalityInplace ( right_col ) ;
2022-12-19 12:31:58 +00:00
correctNullabilityInplace ( right_col , is_nullable ) ;
2022-02-15 11:11:59 +00:00
block . insert ( std : : move ( right_col ) ) ;
2021-06-25 12:03:10 +00:00
}
2019-01-29 12:38:53 +00:00
}
}
2019-12-03 12:55:20 +00:00
else if ( has_required_right_keys )
2019-01-29 12:38:53 +00:00
{
2019-07-31 20:19:52 +00:00
/// Some trash to represent IColumn::Filter as ColumnUInt8 needed for ColumnNullable::applyNullMap()
auto null_map_filter_ptr = ColumnUInt8 : : create ( ) ;
2019-08-21 02:28:04 +00:00
ColumnUInt8 & null_map_filter = assert_cast < ColumnUInt8 & > ( * null_map_filter_ptr ) ;
2019-07-31 20:19:52 +00:00
null_map_filter . getData ( ) . swap ( row_filter ) ;
const IColumn : : Filter & filter = null_map_filter . getData ( ) ;
2019-01-29 12:38:53 +00:00
2019-10-29 19:39:42 +00:00
/// Add join key columns from right block if needed.
2019-11-01 17:41:07 +00:00
for ( size_t i = 0 ; i < required_right_keys . columns ( ) ; + + i )
2019-01-29 12:38:53 +00:00
{
2019-11-01 17:41:07 +00:00
const auto & right_key = required_right_keys . getByPosition ( i ) ;
2021-06-25 12:03:10 +00:00
auto right_col_name = getTableJoin ( ) . renamedRightColumnName ( right_key . name ) ;
2022-12-07 16:27:01 +00:00
if ( ! block . findByName ( right_col_name ) )
2021-06-25 12:03:10 +00:00
{
const auto & left_name = required_right_keys_sources [ i ] ;
2018-09-28 13:06:09 +00:00
2021-06-25 12:03:10 +00:00
/// asof column is already in block.
2021-09-06 10:59:18 +00:00
if ( jf . is_asof_join & & right_key . name = = table_join - > getOnlyClause ( ) . key_names_right . back ( ) )
2021-06-25 12:03:10 +00:00
continue ;
2020-11-03 11:28:28 +00:00
2021-06-25 12:03:10 +00:00
const auto & col = block . getByName ( left_name ) ;
2022-02-15 11:11:59 +00:00
bool is_nullable = JoinCommon : : isNullable ( right_key . type ) ;
2018-09-28 13:06:09 +00:00
2022-07-07 12:26:34 +00:00
ColumnPtr thin_column = JoinCommon : : filterWithBlanks ( col . column , filter ) ;
2021-05-15 11:39:13 +00:00
2021-06-25 12:03:10 +00:00
ColumnWithTypeAndName right_col ( thin_column , col . type , right_col_name ) ;
if ( right_col . type - > lowCardinality ( ) ! = right_key . type - > lowCardinality ( ) )
JoinCommon : : changeLowCardinalityInplace ( right_col ) ;
2022-12-19 12:31:58 +00:00
correctNullabilityInplace ( right_col , is_nullable , null_map_filter ) ;
2022-02-15 11:11:59 +00:00
block . insert ( std : : move ( right_col ) ) ;
2021-05-15 11:39:13 +00:00
2021-06-25 12:03:10 +00:00
if constexpr ( jf . need_replication )
2022-12-07 16:27:01 +00:00
right_keys_to_replicate . push_back ( block . getPositionByName ( right_col_name ) ) ;
2021-06-25 12:03:10 +00:00
}
2018-07-24 10:00:53 +00:00
}
2019-07-31 20:19:52 +00:00
}
2021-06-25 12:03:10 +00:00
if constexpr ( jf . need_replication )
2019-07-31 20:19:52 +00:00
{
2019-11-06 19:39:52 +00:00
std : : unique_ptr < IColumn : : Offsets > & offsets_to_replicate = added_columns . offsets_to_replicate ;
2018-09-28 13:06:09 +00:00
2019-01-29 12:38:53 +00:00
/// If ALL ... JOIN - we replicate all the columns except the new ones.
2018-09-28 13:06:09 +00:00
for ( size_t i = 0 ; i < existing_columns ; + + i )
block . safeGetByPosition ( i ) . column = block . safeGetByPosition ( i ) . column - > replicate ( * offsets_to_replicate ) ;
2019-07-31 20:19:52 +00:00
/// Replicate additional right keys
for ( size_t pos : right_keys_to_replicate )
block . safeGetByPosition ( pos ) . column = block . safeGetByPosition ( pos ) . column - > replicate ( * offsets_to_replicate ) ;
2018-09-28 13:06:09 +00:00
}
2014-06-18 18:31:35 +00:00
}
2020-04-07 09:48:47 +00:00
void HashJoin : : joinBlockImplCross ( Block & block , ExtraBlockPtr & not_processed ) const
2015-07-23 20:23:24 +00:00
{
2020-04-06 13:39:57 +00:00
size_t max_joined_block_rows = table_join - > maxJoinedBlockRows ( ) ;
size_t start_left_row = 0 ;
size_t start_right_block = 0 ;
if ( not_processed )
{
auto & continuation = static_cast < NotProcessedCrossJoin & > ( * not_processed ) ;
start_left_row = continuation . left_position ;
start_right_block = continuation . right_block ;
not_processed . reset ( ) ;
}
2017-12-15 19:17:15 +00:00
size_t num_existing_columns = block . columns ( ) ;
2015-07-23 20:23:24 +00:00
size_t num_columns_to_add = sample_block_with_columns_to_add . columns ( ) ;
2017-04-01 07:20:54 +00:00
2020-04-06 13:39:57 +00:00
ColumnRawPtrs src_left_columns ;
MutableColumns dst_columns ;
2017-04-01 07:20:54 +00:00
2015-07-23 20:23:24 +00:00
{
2020-04-06 13:39:57 +00:00
src_left_columns . reserve ( num_existing_columns ) ;
dst_columns . reserve ( num_existing_columns + num_columns_to_add ) ;
2017-04-01 07:20:54 +00:00
2020-04-06 13:39:57 +00:00
for ( const ColumnWithTypeAndName & left_column : block )
{
src_left_columns . push_back ( left_column . column . get ( ) ) ;
dst_columns . emplace_back ( src_left_columns . back ( ) - > cloneEmpty ( ) ) ;
}
for ( const ColumnWithTypeAndName & right_column : sample_block_with_columns_to_add )
dst_columns . emplace_back ( right_column . column - > cloneEmpty ( ) ) ;
for ( auto & dst : dst_columns )
dst - > reserve ( max_joined_block_rows ) ;
2015-07-23 20:23:24 +00:00
}
2017-04-01 07:20:54 +00:00
2020-04-06 13:39:57 +00:00
size_t rows_left = block . rows ( ) ;
size_t rows_added = 0 ;
2017-04-01 07:20:54 +00:00
2020-04-06 13:39:57 +00:00
for ( size_t left_row = start_left_row ; left_row < rows_left ; + + left_row )
2015-07-23 20:23:24 +00:00
{
2020-04-06 13:39:57 +00:00
size_t block_number = 0 ;
2019-12-19 15:50:28 +00:00
for ( const Block & block_right : data - > blocks )
2015-07-23 20:23:24 +00:00
{
2020-04-06 13:39:57 +00:00
+ + block_number ;
if ( block_number < start_right_block )
continue ;
2017-01-02 20:42:49 +00:00
size_t rows_right = block_right . rows ( ) ;
2020-04-06 13:39:57 +00:00
rows_added + = rows_right ;
2017-04-01 07:20:54 +00:00
2015-07-23 20:23:24 +00:00
for ( size_t col_num = 0 ; col_num < num_existing_columns ; + + col_num )
2020-04-06 13:39:57 +00:00
dst_columns [ col_num ] - > insertManyFrom ( * src_left_columns [ col_num ] , left_row , rows_right ) ;
2017-04-01 07:20:54 +00:00
2015-07-23 20:23:24 +00:00
for ( size_t col_num = 0 ; col_num < num_columns_to_add ; + + col_num )
{
2020-04-06 13:39:57 +00:00
const IColumn & column_right = * block_right . getByPosition ( col_num ) . column ;
dst_columns [ num_existing_columns + col_num ] - > insertRangeFrom ( column_right , 0 , rows_right ) ;
2015-07-23 20:23:24 +00:00
}
}
2020-04-06 13:39:57 +00:00
start_right_block = 0 ;
if ( rows_added > max_joined_block_rows )
{
not_processed = std : : make_shared < NotProcessedCrossJoin > (
NotProcessedCrossJoin { { block . cloneEmpty ( ) } , left_row , block_number + 1 } ) ;
not_processed - > block . swap ( block ) ;
break ;
}
2015-07-23 20:23:24 +00:00
}
2017-04-01 07:20:54 +00:00
2020-04-06 13:39:57 +00:00
for ( const ColumnWithTypeAndName & src_column : sample_block_with_columns_to_add )
block . insert ( src_column ) ;
2017-12-15 19:17:15 +00:00
block = block . cloneWithColumns ( std : : move ( dst_columns ) ) ;
2015-07-23 20:23:24 +00:00
}
2020-07-11 07:12:42 +00:00
DataTypePtr HashJoin : : joinGetCheckAndGetReturnType ( const DataTypes & data_types , const String & column_name , bool or_null ) const
2020-07-23 07:47:17 +00:00
{
2020-07-11 07:12:42 +00:00
size_t num_keys = data_types . size ( ) ;
if ( right_table_keys . columns ( ) ! = num_keys )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NUMBER_OF_ARGUMENTS_DOESNT_MATCH ,
" Number of arguments for function joinGet{} doesn't match: passed, should be equal to {} " ,
toString ( or_null ? " OrNull " : " " ) , toString ( num_keys ) ) ;
2020-07-11 07:12:42 +00:00
for ( size_t i = 0 ; i < num_keys ; + + i )
{
const auto & left_type_origin = data_types [ i ] ;
const auto & [ c2 , right_type_origin , right_name ] = right_table_keys . safeGetByPosition ( i ) ;
2021-02-24 08:44:51 +00:00
auto left_type = removeNullable ( recursiveRemoveLowCardinality ( left_type_origin ) ) ;
auto right_type = removeNullable ( recursiveRemoveLowCardinality ( right_type_origin ) ) ;
2020-07-11 07:12:42 +00:00
if ( ! left_type - > equals ( * right_type ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : TYPE_MISMATCH , " Type mismatch in joinGet key {}: "
" found type {}, while the needed type is {} " , i , left_type - > getName ( ) , right_type - > getName ( ) ) ;
2020-07-11 07:12:42 +00:00
}
2018-11-30 14:49:35 +00:00
if ( ! sample_block_with_columns_to_add . has ( column_name ) )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NO_SUCH_COLUMN_IN_TABLE , " StorageJoin doesn't contain column {} " , column_name ) ;
2020-07-11 07:12:42 +00:00
2020-04-12 03:18:21 +00:00
auto elem = sample_block_with_columns_to_add . getByName ( column_name ) ;
2022-05-30 13:01:27 +00:00
if ( or_null & & JoinCommon : : canBecomeNullable ( elem . type ) )
2020-04-12 03:25:47 +00:00
elem . type = makeNullable ( elem . type ) ;
2020-04-12 03:18:21 +00:00
return elem . type ;
2018-11-30 14:49:35 +00:00
}
2021-02-25 09:31:22 +00:00
/// TODO: return multiple columns as named tuple
2022-07-29 16:30:50 +00:00
/// TODO: return array of values when strictness == JoinStrictness::All
2021-02-25 09:31:22 +00:00
ColumnWithTypeAndName HashJoin : : joinGet ( const Block & block , const Block & block_with_columns_to_add ) const
2018-11-30 14:49:35 +00:00
{
2022-07-29 16:30:50 +00:00
bool is_valid = ( strictness = = JoinStrictness : : Any | | strictness = = JoinStrictness : : RightAny )
& & kind = = JoinKind : : Left ;
2021-02-25 09:31:22 +00:00
if ( ! is_valid )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : INCOMPATIBLE_TYPE_OF_JOIN , " joinGet only supports StorageJoin of type Left Any " ) ;
2021-09-06 10:59:18 +00:00
const auto & key_names_right = table_join - > getOnlyClause ( ) . key_names_right ;
2021-02-25 09:31:22 +00:00
2021-02-24 16:19:04 +00:00
/// Assemble the key block with correct names.
2020-07-11 07:12:42 +00:00
Block keys ;
for ( size_t i = 0 ; i < block . columns ( ) ; + + i )
{
auto key = block . getByPosition ( i ) ;
key . name = key_names_right [ i ] ;
keys . insert ( std : : move ( key ) ) ;
}
2022-07-29 16:30:50 +00:00
static_assert ( ! MapGetter < JoinKind : : Left , JoinStrictness : : Any > : : flagged ,
2021-02-25 09:42:20 +00:00
" joinGet are not protected from hash table changes between block processing " ) ;
2021-06-25 12:03:10 +00:00
2021-09-06 10:59:18 +00:00
std : : vector < const MapsOne * > maps_vector ;
2021-06-25 12:03:10 +00:00
maps_vector . push_back ( & std : : get < MapsOne > ( data - > maps [ 0 ] ) ) ;
2022-07-29 16:30:50 +00:00
joinBlockImpl < JoinKind : : Left , JoinStrictness : : Any > (
2021-09-06 10:59:18 +00:00
keys , block_with_columns_to_add , maps_vector , true ) ;
2020-07-11 07:12:42 +00:00
return keys . getByPosition ( keys . columns ( ) - 1 ) ;
2018-11-30 14:49:35 +00:00
}
2021-09-13 13:35:17 +00:00
void HashJoin : : checkTypesOfKeys ( const Block & block ) const
{
2021-09-20 19:30:34 +00:00
for ( const auto & onexpr : table_join - > getClauses ( ) )
{
JoinCommon : : checkTypesOfKeys ( block , onexpr . key_names_left , right_table_keys , onexpr . key_names_right ) ;
}
2021-09-13 13:35:17 +00:00
}
2020-04-07 09:48:47 +00:00
void HashJoin : : joinBlock ( Block & block , ExtraBlockPtr & not_processed )
2014-06-18 18:31:35 +00:00
{
2022-12-20 12:50:27 +00:00
if ( ! data )
2022-10-19 10:20:41 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Cannot join after data has been released " ) ;
2021-09-06 10:59:18 +00:00
for ( const auto & onexpr : table_join - > getClauses ( ) )
2021-06-25 12:03:10 +00:00
{
2021-09-06 10:59:18 +00:00
auto cond_column_name = onexpr . condColumnNames ( ) ;
JoinCommon : : checkTypesOfKeys (
block , onexpr . key_names_left , cond_column_name . first ,
right_sample_block , onexpr . key_names_right , cond_column_name . second ) ;
2021-06-25 12:03:10 +00:00
}
2022-07-29 16:30:50 +00:00
if ( kind = = JoinKind : : Cross )
2021-06-25 12:03:10 +00:00
{
joinBlockImplCross ( block , not_processed ) ;
return ;
}
2021-09-09 09:47:08 +00:00
2022-07-29 16:30:50 +00:00
if ( kind = = JoinKind : : Right | | kind = = JoinKind : : Full )
2021-06-25 12:03:10 +00:00
{
materializeBlockInplace ( block ) ;
}
2017-04-01 07:20:54 +00:00
2020-04-08 18:59:52 +00:00
{
2021-09-06 10:59:18 +00:00
std : : vector < const std : : decay_t < decltype ( data - > maps [ 0 ] ) > * > maps_vector ;
for ( size_t i = 0 ; i < table_join - > getClauses ( ) . size ( ) ; + + i )
2021-06-25 12:03:10 +00:00
maps_vector . push_back ( & data - > maps [ i ] ) ;
2021-09-06 10:59:18 +00:00
if ( joinDispatch ( kind , strictness , maps_vector , [ & ] ( auto kind_ , auto strictness_ , auto & maps_vector_ )
2019-01-14 21:40:02 +00:00
{
2021-09-06 10:59:18 +00:00
joinBlockImpl < kind_ , strictness_ > ( block , sample_block_with_columns_to_add , maps_vector_ ) ;
2018-12-30 15:54:45 +00:00
} ) )
2021-06-25 12:03:10 +00:00
{
/// Joined
}
else
2021-09-16 15:06:30 +00:00
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Wrong JOIN combination: {} {} " , strictness , kind ) ;
2019-01-14 21:40:02 +00:00
}
2014-06-12 04:04:47 +00:00
}
2022-12-19 15:19:49 +00:00
HashJoin : : ~ HashJoin ( )
{
if ( ! data )
{
2023-01-23 12:34:36 +00:00
LOG_TRACE ( log , " ({}) Join data has been already released " , fmt : : ptr ( this ) ) ;
2022-12-19 15:19:49 +00:00
return ;
}
2023-01-23 12:34:36 +00:00
LOG_TRACE ( log , " ({}) Join data is being destroyed, {} bytes and {} rows in hash table " , fmt : : ptr ( this ) , getTotalByteCount ( ) , getTotalRowCount ( ) ) ;
2022-12-19 15:19:49 +00:00
}
2015-04-17 08:46:06 +00:00
template < typename Mapped >
2019-12-02 18:07:27 +00:00
struct AdderNonJoined
2015-04-17 08:46:06 +00:00
{
2019-03-26 19:46:03 +00:00
static void add ( const Mapped & mapped , size_t & rows_added , MutableColumns & columns_right )
2015-04-17 08:46:06 +00:00
{
2021-02-04 14:46:36 +00:00
constexpr bool mapped_asof = std : : is_same_v < Mapped , AsofRowRefs > ;
[[maybe_unused]] constexpr bool mapped_one = std : : is_same_v < Mapped , RowRef > ;
2019-11-07 21:32:44 +00:00
2019-12-02 18:07:27 +00:00
if constexpr ( mapped_asof )
2019-07-31 20:19:52 +00:00
{
2019-12-02 18:07:27 +00:00
/// Do nothing
2019-07-31 20:19:52 +00:00
}
2019-12-02 18:07:27 +00:00
else if constexpr ( mapped_one )
2015-04-17 08:46:06 +00:00
{
2019-01-31 18:10:16 +00:00
for ( size_t j = 0 ; j < columns_right . size ( ) ; + + j )
2019-07-31 20:19:52 +00:00
{
2019-12-02 18:07:27 +00:00
const auto & mapped_column = mapped . block - > getByPosition ( j ) . column ;
columns_right [ j ] - > insertFrom ( * mapped_column , mapped . row_num ) ;
2019-07-31 20:19:52 +00:00
}
2018-11-29 15:44:12 +00:00
+ + rows_added ;
2015-04-17 08:46:06 +00:00
}
2019-12-02 18:07:27 +00:00
else
{
for ( auto it = mapped . begin ( ) ; it . ok ( ) ; + + it )
{
for ( size_t j = 0 ; j < columns_right . size ( ) ; + + j )
{
const auto & mapped_column = it - > block - > getByPosition ( j ) . column ;
columns_right [ j ] - > insertFrom ( * mapped_column , it - > row_num ) ;
}
2015-04-17 08:46:06 +00:00
2019-12-02 18:07:27 +00:00
+ + rows_added ;
}
}
2019-03-20 16:58:28 +00:00
}
} ;
2015-04-17 08:46:06 +00:00
2017-04-02 17:37:49 +00:00
/// Stream from not joined earlier rows of the right table.
2021-09-06 10:59:18 +00:00
/// Based on:
/// - map offsetInternal saved in used_flags for single disjuncts
/// - flags in BlockWithFlags for multiple disjuncts
2021-06-25 12:03:10 +00:00
template < bool multiple_disjuncts >
2021-08-17 13:30:01 +00:00
class NotJoinedHash final : public NotJoinedBlocks : : RightColumnsFiller
2015-04-17 08:46:06 +00:00
{
public :
2021-08-09 14:30:37 +00:00
NotJoinedHash ( const HashJoin & parent_ , UInt64 max_block_size_ )
2021-11-08 12:44:13 +00:00
: parent ( parent_ ) , max_block_size ( max_block_size_ ) , current_block_start ( 0 )
2022-12-20 12:50:27 +00:00
{
if ( parent . data = = nullptr )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Cannot join after data has been released " ) ;
}
2015-04-17 08:46:06 +00:00
2021-08-09 14:30:37 +00:00
Block getEmptyBlock ( ) override { return parent . savedBlockSample ( ) . cloneEmpty ( ) ; }
2017-04-01 07:20:54 +00:00
2021-08-06 14:15:11 +00:00
size_t fillColumns ( MutableColumns & columns_right ) override
2019-11-01 17:41:07 +00:00
{
2015-04-17 08:46:06 +00:00
size_t rows_added = 0 ;
2021-11-08 12:44:13 +00:00
if ( unlikely ( parent . data - > type = = HashJoin : : Type : : EMPTY ) )
{
rows_added = fillColumnsFromData ( parent . data - > blocks , columns_right ) ;
}
else
2017-03-28 06:51:22 +00:00
{
2021-11-08 12:44:13 +00:00
auto fill_callback = [ & ] ( auto , auto strictness , auto & map )
{
rows_added = fillColumnsFromMap < strictness > ( map , columns_right ) ;
} ;
2017-04-01 07:20:54 +00:00
2021-11-08 12:44:13 +00:00
if ( ! joinDispatch ( parent . kind , parent . strictness , parent . data - > maps . front ( ) , fill_callback ) )
throw Exception ( ErrorCodes : : LOGICAL_ERROR , " Unknown JOIN strictness '{}' (must be on of: ANY, ALL, ASOF) " , parent.strictness) ;
}
2017-04-01 07:20:54 +00:00
2021-06-25 12:03:10 +00:00
if constexpr ( ! multiple_disjuncts )
{
fillNullsFromBlocks ( columns_right , rows_added ) ;
}
2021-08-06 14:15:11 +00:00
return rows_added ;
}
2021-08-06 11:54:04 +00:00
2021-08-06 14:15:11 +00:00
private :
const HashJoin & parent ;
UInt64 max_block_size ;
2021-08-06 11:54:04 +00:00
2021-11-08 12:44:13 +00:00
size_t current_block_start ;
2021-08-06 14:15:11 +00:00
std : : any position ;
std : : optional < HashJoin : : BlockNullmapList : : const_iterator > nulls_position ;
2021-08-18 13:28:15 +00:00
std : : optional < BlocksList : : const_iterator > used_position ;
2017-04-01 07:20:54 +00:00
2021-11-08 12:44:13 +00:00
size_t fillColumnsFromData ( const BlocksList & blocks , MutableColumns & columns_right )
{
if ( ! position . has_value ( ) )
position = std : : make_any < BlocksList : : const_iterator > ( blocks . begin ( ) ) ;
auto & block_it = std : : any_cast < BlocksList : : const_iterator & > ( position ) ;
auto end = blocks . end ( ) ;
size_t rows_added = 0 ;
for ( ; block_it ! = end ; + + block_it )
{
size_t rows_from_block = std : : min < size_t > ( max_block_size - rows_added , block_it - > rows ( ) - current_block_start ) ;
for ( size_t j = 0 ; j < columns_right . size ( ) ; + + j )
{
const auto & col = block_it - > getByPosition ( j ) . column ;
columns_right [ j ] - > insertRangeFrom ( * col , current_block_start , rows_from_block ) ;
}
rows_added + = rows_from_block ;
if ( rows_added > = max_block_size )
{
/// How many rows have been read
current_block_start + = rows_from_block ;
if ( block_it - > rows ( ) < = current_block_start )
{
/// current block was fully read
+ + block_it ;
current_block_start = 0 ;
}
break ;
}
current_block_start = 0 ;
}
return rows_added ;
}
2022-07-29 16:30:50 +00:00
template < JoinStrictness STRICTNESS , typename Maps >
2019-07-03 19:06:34 +00:00
size_t fillColumnsFromMap ( const Maps & maps , MutableColumns & columns_keys_and_right )
{
2019-12-19 15:50:28 +00:00
switch ( parent . data - > type )
2019-07-03 19:06:34 +00:00
{
# define M(TYPE) \
2020-04-07 09:48:47 +00:00
case HashJoin : : Type : : TYPE : \
2019-07-03 19:06:34 +00:00
return fillColumns < STRICTNESS > ( * maps . TYPE , columns_keys_and_right ) ;
APPLY_FOR_JOIN_VARIANTS ( M )
# undef M
default :
2021-11-08 12:44:13 +00:00
throw Exception ( ErrorCodes : : UNSUPPORTED_JOIN_KEYS , " Unsupported JOIN keys (type: {}) " , parent.data->type) ;
2019-07-03 19:06:34 +00:00
}
2022-10-07 19:20:14 +00:00
UNREACHABLE ( ) ;
2019-07-03 19:06:34 +00:00
}
2022-07-29 16:30:50 +00:00
template < JoinStrictness STRICTNESS , typename Map >
2019-03-26 19:46:03 +00:00
size_t fillColumns ( const Map & map , MutableColumns & columns_keys_and_right )
2015-04-17 08:46:06 +00:00
{
size_t rows_added = 0 ;
2017-04-01 07:20:54 +00:00
2021-06-25 12:03:10 +00:00
if constexpr ( multiple_disjuncts )
{
if ( ! used_position . has_value ( ) )
used_position = parent . data - > blocks . begin ( ) ;
2017-04-01 07:20:54 +00:00
2021-06-25 12:03:10 +00:00
auto end = parent . data - > blocks . end ( ) ;
2017-04-01 07:20:54 +00:00
2021-06-25 12:03:10 +00:00
for ( auto & it = * used_position ; it ! = end & & rows_added < max_block_size ; + + it )
{
2021-08-18 13:28:15 +00:00
const Block & mapped_block = * it ;
2021-06-25 12:03:10 +00:00
2021-08-17 10:27:23 +00:00
for ( size_t row = 0 ; row < mapped_block . rows ( ) ; + + row )
2021-06-25 12:03:10 +00:00
{
2021-08-17 10:27:23 +00:00
if ( ! parent . isUsed ( & mapped_block , row ) )
2021-06-25 12:03:10 +00:00
{
for ( size_t colnum = 0 ; colnum < columns_keys_and_right . size ( ) ; + + colnum )
{
2021-08-17 10:27:23 +00:00
columns_keys_and_right [ colnum ] - > insertFrom ( * mapped_block . getByPosition ( colnum ) . column , row ) ;
2021-06-25 12:03:10 +00:00
}
+ + rows_added ;
}
}
}
}
else
2015-04-17 08:46:06 +00:00
{
2021-06-25 12:03:10 +00:00
using Mapped = typename Map : : mapped_type ;
using Iterator = typename Map : : const_iterator ;
2019-12-02 18:07:27 +00:00
2017-04-01 07:20:54 +00:00
2021-06-25 12:03:10 +00:00
if ( ! position . has_value ( ) )
position = std : : make_any < Iterator > ( map . begin ( ) ) ;
Iterator & it = std : : any_cast < Iterator & > ( position ) ;
auto end = map . end ( ) ;
2017-04-01 07:20:54 +00:00
2021-06-25 12:03:10 +00:00
for ( ; it ! = end ; + + it )
2018-01-05 16:12:35 +00:00
{
2021-06-25 12:03:10 +00:00
const Mapped & mapped = it - > getMapped ( ) ;
size_t off = map . offsetInternal ( it . getPtr ( ) ) ;
if ( parent . isUsed ( off ) )
continue ;
AdderNonJoined < Mapped > : : add ( mapped , rows_added , columns_keys_and_right ) ;
if ( rows_added > = max_block_size )
{
+ + it ;
break ;
}
2018-01-05 16:12:35 +00:00
}
2015-04-17 08:46:06 +00:00
}
2017-04-01 07:20:54 +00:00
2015-04-17 08:46:06 +00:00
return rows_added ;
}
2019-03-29 18:07:22 +00:00
2019-07-03 19:06:34 +00:00
void fillNullsFromBlocks ( MutableColumns & columns_keys_and_right , size_t & rows_added )
{
2019-07-04 12:12:39 +00:00
if ( ! nulls_position . has_value ( ) )
2019-12-19 15:50:28 +00:00
nulls_position = parent . data - > blocks_nullmaps . begin ( ) ;
2019-07-03 19:06:34 +00:00
2019-12-19 15:50:28 +00:00
auto end = parent . data - > blocks_nullmaps . end ( ) ;
2019-07-03 19:06:34 +00:00
2019-07-04 12:12:39 +00:00
for ( auto & it = * nulls_position ; it ! = end & & rows_added < max_block_size ; + + it )
2019-07-03 19:06:34 +00:00
{
2021-11-08 12:36:34 +00:00
const auto * block = it - > first ;
ConstNullMapPtr nullmap = nullptr ;
if ( it - > second )
nullmap = & assert_cast < const ColumnUInt8 & > ( * it - > second ) . getData ( ) ;
2019-07-03 19:06:34 +00:00
2021-11-08 12:36:34 +00:00
for ( size_t row = 0 ; row < block - > rows ( ) ; + + row )
2019-07-03 19:06:34 +00:00
{
2021-11-08 12:36:34 +00:00
if ( nullmap & & ( * nullmap ) [ row ] )
2019-07-03 19:06:34 +00:00
{
for ( size_t col = 0 ; col < columns_keys_and_right . size ( ) ; + + col )
columns_keys_and_right [ col ] - > insertFrom ( * block - > getByPosition ( col ) . column , row ) ;
+ + rows_added ;
}
}
}
}
2015-04-17 08:46:06 +00:00
} ;
2022-10-18 11:43:01 +00:00
IBlocksStreamPtr HashJoin : : getNonJoinedBlocks ( const Block & left_sample_block ,
2021-08-23 11:09:33 +00:00
const Block & result_sample_block ,
UInt64 max_block_size ) const
2015-04-17 08:46:06 +00:00
{
2022-06-15 15:11:24 +00:00
if ( ! JoinCommon : : hasNonJoinedBlocks ( * table_join ) )
2019-12-02 18:07:27 +00:00
return { } ;
2022-06-15 15:11:24 +00:00
2021-09-06 10:59:18 +00:00
bool multiple_disjuncts = ! table_join - > oneDisjunct ( ) ;
2021-06-25 12:03:10 +00:00
if ( multiple_disjuncts )
{
/// ... calculate `left_columns_count` ...
2021-08-23 11:09:33 +00:00
size_t left_columns_count = left_sample_block . columns ( ) ;
2021-06-25 12:03:10 +00:00
auto non_joined = std : : make_unique < NotJoinedHash < true > > ( * this , max_block_size ) ;
2022-12-05 17:54:12 +00:00
return std : : make_unique < NotJoinedBlocks > ( std : : move ( non_joined ) , result_sample_block , left_columns_count , * table_join ) ;
2021-06-25 12:03:10 +00:00
}
else
{
2021-08-23 11:09:33 +00:00
size_t left_columns_count = left_sample_block . columns ( ) ;
assert ( left_columns_count = = result_sample_block . columns ( ) - required_right_keys . columns ( ) - sample_block_with_columns_to_add . columns ( ) ) ;
2021-06-25 12:03:10 +00:00
auto non_joined = std : : make_unique < NotJoinedHash < false > > ( * this , max_block_size ) ;
2022-12-05 17:54:12 +00:00
return std : : make_unique < NotJoinedBlocks > ( std : : move ( non_joined ) , result_sample_block , left_columns_count , * table_join ) ;
2021-06-25 12:03:10 +00:00
}
2015-04-17 08:46:06 +00:00
}
2021-02-04 14:46:36 +00:00
void HashJoin : : reuseJoinedData ( const HashJoin & join )
{
data = join . data ;
2021-04-28 17:32:12 +00:00
from_storage_join = true ;
2021-08-09 15:58:41 +00:00
2021-09-06 10:59:18 +00:00
bool multiple_disjuncts = ! table_join - > oneDisjunct ( ) ;
2021-08-09 15:58:41 +00:00
if ( multiple_disjuncts )
2023-01-23 21:13:58 +00:00
throw Exception ( ErrorCodes : : NOT_IMPLEMENTED , " StorageJoin with ORs is not supported " ) ;
2021-08-09 15:58:41 +00:00
2021-06-25 12:03:10 +00:00
for ( auto & map : data - > maps )
2021-02-04 14:46:36 +00:00
{
2021-06-25 12:03:10 +00:00
joinDispatch ( kind , strictness , map , [ this ] ( auto kind_ , auto strictness_ , auto & map_ )
{
used_flags . reinit < kind_ , strictness_ > ( map_ . getBufferSizeInCells ( data - > type ) + 1 ) ;
} ) ;
}
2021-02-04 14:46:36 +00:00
}
2021-09-09 09:47:08 +00:00
2022-12-20 12:50:27 +00:00
BlocksList HashJoin : : releaseJoinedBlocks ( bool restructure )
2022-09-27 12:33:09 +00:00
{
2023-01-23 12:34:36 +00:00
LOG_TRACE ( log , " ({}) Join data is being released, {} bytes and {} rows in hash table " , fmt : : ptr ( this ) , getTotalByteCount ( ) , getTotalRowCount ( ) ) ;
2022-12-27 10:13:12 +00:00
2022-10-18 12:45:06 +00:00
BlocksList right_blocks = std : : move ( data - > blocks ) ;
2022-12-20 12:50:27 +00:00
if ( ! restructure )
{
data . reset ( ) ;
return right_blocks ;
}
data - > maps . clear ( ) ;
data - > blocks_nullmaps . clear ( ) ;
2022-06-16 12:09:23 +00:00
BlocksList restored_blocks ;
/// names to positions optimization
std : : vector < size_t > positions ;
std : : vector < bool > is_nullable ;
if ( ! right_blocks . empty ( ) )
{
positions . reserve ( right_sample_block . columns ( ) ) ;
const Block & tmp_block = * right_blocks . begin ( ) ;
for ( const auto & sample_column : right_sample_block )
{
positions . emplace_back ( tmp_block . getPositionByName ( sample_column . name ) ) ;
is_nullable . emplace_back ( JoinCommon : : isNullable ( sample_column . type ) ) ;
}
}
for ( Block & saved_block : right_blocks )
{
Block restored_block ;
for ( size_t i = 0 ; i < positions . size ( ) ; + + i )
{
auto & column = saved_block . getByPosition ( positions [ i ] ) ;
2022-12-19 12:31:58 +00:00
correctNullabilityInplace ( column , is_nullable [ i ] ) ;
restored_block . insert ( column ) ;
2022-06-16 12:09:23 +00:00
}
restored_blocks . emplace_back ( std : : move ( restored_block ) ) ;
}
2022-12-20 12:50:27 +00:00
data . reset ( ) ;
2022-06-16 12:09:23 +00:00
return restored_blocks ;
}
2021-09-09 09:47:08 +00:00
const ColumnWithTypeAndName & HashJoin : : rightAsofKeyColumn ( ) const
{
2022-02-15 11:11:59 +00:00
/// It should be nullable when right side is nullable
2021-09-09 09:47:08 +00:00
return savedBlockSample ( ) . getByName ( table_join - > getOnlyClause ( ) . key_names_right . back ( ) ) ;
2021-02-04 14:46:36 +00:00
}
2014-06-12 02:31:30 +00:00
}