2014-06-12 02:31:30 +00:00
|
|
|
|
#pragma once
|
|
|
|
|
|
2019-07-02 14:38:31 +00:00
|
|
|
|
#include <variant>
|
2019-03-20 16:58:28 +00:00
|
|
|
|
#include <optional>
|
2019-07-04 12:12:39 +00:00
|
|
|
|
#include <deque>
|
2021-09-06 10:59:18 +00:00
|
|
|
|
#include <vector>
|
2014-06-12 02:31:30 +00:00
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
2014-06-18 18:31:35 +00:00
|
|
|
|
|
2019-09-09 19:43:37 +00:00
|
|
|
|
#include <Interpreters/IJoin.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
#include <Interpreters/AggregationCommon.h>
|
2019-03-30 21:30:21 +00:00
|
|
|
|
#include <Interpreters/RowRefs.h>
|
2014-06-12 02:31:30 +00:00
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
|
#include <Common/Arena.h>
|
2019-01-24 14:56:04 +00:00
|
|
|
|
#include <Common/ColumnsHashing.h>
|
2017-04-01 09:19:00 +00:00
|
|
|
|
#include <Common/HashTable/HashMap.h>
|
2019-02-28 09:35:38 +00:00
|
|
|
|
#include <Common/HashTable/FixedHashMap.h>
|
2022-06-23 07:59:13 +00:00
|
|
|
|
#include <Storages/TableLockHolder.h>
|
2014-06-12 02:31:30 +00:00
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
|
#include <Columns/ColumnString.h>
|
|
|
|
|
#include <Columns/ColumnFixedString.h>
|
2017-03-28 06:51:22 +00:00
|
|
|
|
|
2021-10-15 20:18:20 +00:00
|
|
|
|
#include <QueryPipeline/SizeLimits.h>
|
2014-06-12 02:31:30 +00:00
|
|
|
|
|
2020-12-10 23:56:57 +00:00
|
|
|
|
#include <Core/Block.h>
|
2014-06-12 02:31:30 +00:00
|
|
|
|
|
2022-01-21 05:36:36 +00:00
|
|
|
|
#include <Storages/IStorage_fwd.h>
|
2022-08-04 15:39:28 +00:00
|
|
|
|
#include <Interpreters/IKeyValueEntity.h>
|
2024-04-28 11:20:59 +00:00
|
|
|
|
#include <Interpreters/TemporaryDataOnDisk.h>
|
2022-01-21 05:36:36 +00:00
|
|
|
|
|
2014-06-12 02:31:30 +00:00
|
|
|
|
namespace DB
|
|
|
|
|
{
|
2019-05-07 18:21:44 +00:00
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
|
class TableJoin;
|
2024-03-07 11:51:54 +00:00
|
|
|
|
class ExpressionActions;
|
2019-09-02 19:58:45 +00:00
|
|
|
|
|
2019-05-07 18:21:44 +00:00
|
|
|
|
namespace JoinStuff
|
|
|
|
|
{
|
2019-05-08 10:53:45 +00:00
|
|
|
|
|
2021-02-04 14:46:36 +00:00
|
|
|
|
/// Flags needed to implement RIGHT and FULL JOINs.
|
|
|
|
|
class JoinUsedFlags
|
2019-05-07 18:21:44 +00:00
|
|
|
|
{
|
2021-08-17 10:27:23 +00:00
|
|
|
|
using RawBlockPtr = const Block *;
|
|
|
|
|
using UsedFlagsForBlock = std::vector<std::atomic_bool>;
|
|
|
|
|
|
|
|
|
|
/// For multiple dijuncts each empty in hashmap stores flags for particular block
|
|
|
|
|
/// For single dicunct we store all flags in `nullptr` entry, index is the offset in FindResult
|
|
|
|
|
std::unordered_map<RawBlockPtr, UsedFlagsForBlock> flags;
|
|
|
|
|
|
2021-02-04 14:46:36 +00:00
|
|
|
|
bool need_flags;
|
2019-05-08 10:53:45 +00:00
|
|
|
|
|
2021-02-04 14:46:36 +00:00
|
|
|
|
public:
|
|
|
|
|
/// Update size for vector with flags.
|
|
|
|
|
/// Calling this method invalidates existing flags.
|
|
|
|
|
/// It can be called several times, but all of them should happen before using this structure.
|
2022-07-29 16:30:50 +00:00
|
|
|
|
template <JoinKind KIND, JoinStrictness STRICTNESS>
|
2021-02-04 14:46:36 +00:00
|
|
|
|
void reinit(size_t size_);
|
2019-11-07 21:32:44 +00:00
|
|
|
|
|
2022-07-29 16:30:50 +00:00
|
|
|
|
template <JoinKind KIND, JoinStrictness STRICTNESS>
|
2021-08-17 10:27:23 +00:00
|
|
|
|
void reinit(const Block * block_ptr);
|
|
|
|
|
|
2021-02-04 14:46:36 +00:00
|
|
|
|
bool getUsedSafe(size_t i) const;
|
2021-08-17 10:27:23 +00:00
|
|
|
|
bool getUsedSafe(const Block * block_ptr, size_t row_idx) const;
|
2019-05-07 18:21:44 +00:00
|
|
|
|
|
2024-03-07 11:51:54 +00:00
|
|
|
|
template <bool use_flags, bool flag_per_row, typename T>
|
2021-06-25 12:03:10 +00:00
|
|
|
|
void setUsed(const T & f);
|
2019-05-08 10:53:45 +00:00
|
|
|
|
|
2024-03-07 11:51:54 +00:00
|
|
|
|
template <bool use_flags, bool flag_per_row>
|
2022-02-19 20:01:47 +00:00
|
|
|
|
void setUsed(const Block * block, size_t row_num, size_t offset);
|
|
|
|
|
|
2024-03-07 11:51:54 +00:00
|
|
|
|
template <bool use_flags, bool flag_per_row, typename T>
|
2021-06-25 12:03:10 +00:00
|
|
|
|
bool getUsed(const T & f);
|
2019-05-07 18:21:44 +00:00
|
|
|
|
|
2024-03-07 11:51:54 +00:00
|
|
|
|
template <bool use_flags, bool flag_per_row, typename T>
|
2021-06-25 12:03:10 +00:00
|
|
|
|
bool setUsedOnce(const T & f);
|
2021-02-04 14:46:36 +00:00
|
|
|
|
};
|
2019-05-07 18:21:44 +00:00
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2017-03-29 11:12:47 +00:00
|
|
|
|
/** Data structure for implementation of JOIN.
|
|
|
|
|
* It is just a hash table: keys -> rows of joined ("right") table.
|
|
|
|
|
* Additionally, CROSS JOIN is supported: instead of hash table, it use just set of blocks without keys.
|
2014-06-19 19:00:58 +00:00
|
|
|
|
*
|
2019-12-02 18:07:27 +00:00
|
|
|
|
* JOIN-s could be of these types:
|
|
|
|
|
* - ALL × LEFT/INNER/RIGHT/FULL
|
|
|
|
|
* - ANY × LEFT/INNER/RIGHT
|
|
|
|
|
* - SEMI/ANTI x LEFT/RIGHT
|
|
|
|
|
* - ASOF x LEFT/INNER
|
|
|
|
|
* - CROSS
|
2014-06-19 19:00:58 +00:00
|
|
|
|
*
|
2019-12-02 18:07:27 +00:00
|
|
|
|
* ALL means usual JOIN, when rows are multiplied by number of matching rows from the "right" table.
|
2020-08-08 01:01:47 +00:00
|
|
|
|
* ANY uses one line per unique key from right table. For LEFT JOIN it would be any row (with needed joined key) from the right table,
|
2019-12-02 18:07:27 +00:00
|
|
|
|
* for RIGHT JOIN it would be any row from the left table and for INNER one it would be any row from right and any row from left.
|
|
|
|
|
* SEMI JOIN filter left table by keys that are present in right table for LEFT JOIN, and filter right table by keys from left table
|
|
|
|
|
* for RIGHT JOIN. In other words SEMI JOIN returns only rows which joining keys present in another table.
|
|
|
|
|
* ANTI JOIN is the same as SEMI JOIN but returns rows with joining keys that are NOT present in another table.
|
|
|
|
|
* SEMI/ANTI JOINs allow to get values from both tables. For filter table it gets any row with joining same key. For ANTI JOIN it returns
|
|
|
|
|
* defaults other table columns.
|
|
|
|
|
* ASOF JOIN is not-equi join. For one key column it finds nearest value to join according to join inequality.
|
|
|
|
|
* It's expected that ANY|SEMI LEFT JOIN is more efficient that ALL one.
|
2014-06-19 19:00:58 +00:00
|
|
|
|
*
|
2017-03-29 11:12:47 +00:00
|
|
|
|
* If INNER is specified - leave only rows that have matching rows from "right" table.
|
|
|
|
|
* If LEFT is specified - in case when there is no matching row in "right" table, fill it with default values instead.
|
|
|
|
|
* If RIGHT is specified - first process as INNER, but track what rows from the right table was joined,
|
|
|
|
|
* and at the end, add rows from right table that was not joined and substitute default values for columns of left table.
|
|
|
|
|
* If FULL is specified - first process as LEFT, but track what rows from the right table was joined,
|
|
|
|
|
* and at the end, add rows from right table that was not joined and substitute default values for columns of left table.
|
2015-04-17 08:46:06 +00:00
|
|
|
|
*
|
2017-03-29 11:12:47 +00:00
|
|
|
|
* Thus, LEFT and RIGHT JOINs are not symmetric in terms of implementation.
|
2014-06-19 19:00:58 +00:00
|
|
|
|
*
|
2017-03-29 11:12:47 +00:00
|
|
|
|
* All JOINs (except CROSS) are done by equality condition on keys (equijoin).
|
|
|
|
|
* Non-equality and other conditions are not supported.
|
2014-06-19 19:00:58 +00:00
|
|
|
|
*
|
2017-03-29 11:12:47 +00:00
|
|
|
|
* Implementation:
|
2014-06-19 19:00:58 +00:00
|
|
|
|
*
|
2017-03-29 11:12:47 +00:00
|
|
|
|
* 1. Build hash table in memory from "right" table.
|
|
|
|
|
* This hash table is in form of keys -> row in case of ANY or keys -> [rows...] in case of ALL.
|
|
|
|
|
* This is done in insertFromBlock method.
|
2014-06-19 19:00:58 +00:00
|
|
|
|
*
|
2017-03-29 11:12:47 +00:00
|
|
|
|
* 2. Process "left" table and join corresponding rows from "right" table by lookups in the map.
|
|
|
|
|
* This is done in joinBlock methods.
|
2014-06-19 19:00:58 +00:00
|
|
|
|
*
|
2017-03-29 11:12:47 +00:00
|
|
|
|
* In case of ANY LEFT JOIN - form new columns with found values or default values.
|
|
|
|
|
* This is the most simple. Number of rows in left table does not change.
|
2014-06-19 19:00:58 +00:00
|
|
|
|
*
|
2017-03-29 11:12:47 +00:00
|
|
|
|
* In case of ANY INNER JOIN - form new columns with found values,
|
|
|
|
|
* and also build a filter - in what rows nothing was found.
|
|
|
|
|
* Then filter columns of "left" table.
|
2014-06-19 19:00:58 +00:00
|
|
|
|
*
|
2017-03-29 11:12:47 +00:00
|
|
|
|
* In case of ALL ... JOIN - form new columns with all found rows,
|
|
|
|
|
* and also fill 'offsets' array, describing how many times we need to replicate values of "left" table.
|
|
|
|
|
* Then replicate columns of "left" table.
|
2017-03-30 14:09:24 +00:00
|
|
|
|
*
|
|
|
|
|
* How Nullable keys are processed:
|
|
|
|
|
*
|
|
|
|
|
* NULLs never join to anything, even to each other.
|
|
|
|
|
* During building of map, we just skip keys with NULL value of any component.
|
|
|
|
|
* During joining, we simply treat rows with any NULLs in key as non joined.
|
2017-04-04 06:52:39 +00:00
|
|
|
|
*
|
|
|
|
|
* Default values for outer joins (LEFT, RIGHT, FULL):
|
|
|
|
|
*
|
|
|
|
|
* Behaviour is controlled by 'join_use_nulls' settings.
|
|
|
|
|
* If it is false, we substitute (global) default value for the data type, for non-joined rows
|
|
|
|
|
* (zero, empty string, etc. and NULL for Nullable data types).
|
|
|
|
|
* If it is true, we always generate Nullable column and substitute NULLs for non-joined rows,
|
|
|
|
|
* as in standard SQL.
|
2014-06-12 02:31:30 +00:00
|
|
|
|
*/
|
2020-04-07 09:48:47 +00:00
|
|
|
|
class HashJoin : public IJoin
|
2014-06-12 02:31:30 +00:00
|
|
|
|
{
|
|
|
|
|
public:
|
2023-07-10 08:44:01 +00:00
|
|
|
|
HashJoin(
|
2023-11-28 10:28:11 +00:00
|
|
|
|
std::shared_ptr<TableJoin> table_join_, const Block & right_sample_block,
|
2024-04-24 12:05:13 +00:00
|
|
|
|
bool any_take_last_row_ = false, size_t reserve_num_ = 0, const String & instance_id_ = "");
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
2022-12-19 15:19:49 +00:00
|
|
|
|
~HashJoin() override;
|
|
|
|
|
|
2023-08-03 15:55:15 +00:00
|
|
|
|
std::string getName() const override { return "HashJoin"; }
|
2024-04-24 12:05:13 +00:00
|
|
|
|
|
2021-03-25 18:11:54 +00:00
|
|
|
|
const TableJoin & getTableJoin() const override { return *table_join; }
|
|
|
|
|
|
2024-04-24 12:05:13 +00:00
|
|
|
|
bool isCloneSupported() const override
|
|
|
|
|
{
|
|
|
|
|
return true;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
std::shared_ptr<IJoin> clone(const std::shared_ptr<TableJoin> & table_join_,
|
|
|
|
|
const Block &,
|
|
|
|
|
const Block & right_sample_block_) const override
|
|
|
|
|
{
|
|
|
|
|
return std::make_shared<HashJoin>(table_join_, right_sample_block_, any_take_last_row, reserve_num, instance_id);
|
|
|
|
|
}
|
|
|
|
|
|
2017-03-29 11:12:47 +00:00
|
|
|
|
/** Add block of data from right hand of JOIN to the map.
|
|
|
|
|
* Returns false, if some limit was exceeded and you should not insert more data.
|
2014-06-12 04:04:47 +00:00
|
|
|
|
*/
|
2023-07-05 17:03:18 +00:00
|
|
|
|
bool addBlockToJoin(const Block & source_block_, bool check_limits) override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
2021-09-13 13:35:17 +00:00
|
|
|
|
void checkTypesOfKeys(const Block & block) const override;
|
|
|
|
|
|
2023-07-05 17:03:18 +00:00
|
|
|
|
/** Join data from the map (that was previously built by calls to addBlockToJoin) to the block with data from "left" table.
|
2017-03-29 11:12:47 +00:00
|
|
|
|
* Could be called from different threads in parallel.
|
2014-06-12 04:04:47 +00:00
|
|
|
|
*/
|
2020-01-15 20:33:29 +00:00
|
|
|
|
void joinBlock(Block & block, ExtraBlockPtr & not_processed) override;
|
2018-11-30 14:49:35 +00:00
|
|
|
|
|
2020-07-11 07:12:42 +00:00
|
|
|
|
/// Check joinGet arguments and infer the return type.
|
|
|
|
|
DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const;
|
2018-11-30 14:49:35 +00:00
|
|
|
|
|
2020-07-11 07:12:42 +00:00
|
|
|
|
/// Used by joinGet function that turns StorageJoin into a dictionary.
|
|
|
|
|
ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
2022-07-08 13:11:27 +00:00
|
|
|
|
bool isFilled() const override { return from_storage_join; }
|
2021-04-28 17:32:12 +00:00
|
|
|
|
|
2022-07-06 15:11:23 +00:00
|
|
|
|
JoinPipelineType pipelineType() const override
|
2022-03-30 10:07:09 +00:00
|
|
|
|
{
|
2022-07-08 13:11:27 +00:00
|
|
|
|
/// No need to process anything in the right stream if hash table was already filled
|
|
|
|
|
if (from_storage_join)
|
2022-04-01 18:16:38 +00:00
|
|
|
|
return JoinPipelineType::FilledRight;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
2022-07-05 16:21:35 +00:00
|
|
|
|
/// Default pipeline processes right stream at first and then left.
|
2022-04-01 18:16:38 +00:00
|
|
|
|
return JoinPipelineType::FillRightFirst;
|
2022-03-30 10:07:09 +00:00
|
|
|
|
}
|
2021-04-28 17:32:12 +00:00
|
|
|
|
|
2017-03-29 11:12:47 +00:00
|
|
|
|
/** For RIGHT and FULL JOINs.
|
|
|
|
|
* A stream that will contain default values from left table, joined with rows from right table, that was not joined before.
|
|
|
|
|
* Use only after all calls to joinBlock was done.
|
2017-04-04 06:52:39 +00:00
|
|
|
|
* left_sample_block is passed without account of 'use_nulls' setting (columns will be converted to Nullable inside).
|
2015-04-17 08:46:06 +00:00
|
|
|
|
*/
|
2022-10-18 11:43:01 +00:00
|
|
|
|
IBlocksStreamPtr getNonJoinedBlocks(
|
2021-08-23 11:09:33 +00:00
|
|
|
|
const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const override;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
2017-03-29 11:12:47 +00:00
|
|
|
|
/// Number of keys in all built JOIN maps.
|
2019-10-11 17:27:54 +00:00
|
|
|
|
size_t getTotalRowCount() const final;
|
2017-03-29 11:12:47 +00:00
|
|
|
|
/// Sum size in bytes of all buffers, used for JOIN maps and for all memory pools.
|
2020-02-17 17:08:31 +00:00
|
|
|
|
size_t getTotalByteCount() const final;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
2021-01-04 23:49:31 +00:00
|
|
|
|
bool alwaysReturnsEmptySet() const final;
|
2019-10-11 17:27:54 +00:00
|
|
|
|
|
2022-07-29 16:30:50 +00:00
|
|
|
|
JoinKind getKind() const { return kind; }
|
|
|
|
|
JoinStrictness getStrictness() const { return strictness; }
|
2020-08-03 23:11:39 +00:00
|
|
|
|
const std::optional<TypeIndex> & getAsofType() const { return asof_type; }
|
2022-07-29 16:30:50 +00:00
|
|
|
|
ASOFJoinInequality getAsofInequality() const { return asof_inequality; }
|
2019-05-07 18:21:44 +00:00
|
|
|
|
bool anyTakeLastRow() const { return any_take_last_row; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
2021-09-09 09:47:08 +00:00
|
|
|
|
const ColumnWithTypeAndName & rightAsofKeyColumn() const;
|
2020-08-03 23:11:39 +00:00
|
|
|
|
|
2017-03-29 11:12:47 +00:00
|
|
|
|
/// Different types of keys for maps.
|
2017-03-28 06:51:22 +00:00
|
|
|
|
#define APPLY_FOR_JOIN_VARIANTS(M) \
|
2017-11-12 00:46:22 +00:00
|
|
|
|
M(key8) \
|
|
|
|
|
M(key16) \
|
|
|
|
|
M(key32) \
|
|
|
|
|
M(key64) \
|
|
|
|
|
M(key_string) \
|
|
|
|
|
M(key_fixed_string) \
|
|
|
|
|
M(keys128) \
|
|
|
|
|
M(keys256) \
|
2017-03-28 06:51:22 +00:00
|
|
|
|
M(hashed)
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
2023-07-10 08:44:01 +00:00
|
|
|
|
/// Only for maps using hash table.
|
|
|
|
|
#define APPLY_FOR_HASH_JOIN_VARIANTS(M) \
|
|
|
|
|
M(key32) \
|
|
|
|
|
M(key64) \
|
|
|
|
|
M(key_string) \
|
|
|
|
|
M(key_fixed_string) \
|
|
|
|
|
M(keys128) \
|
|
|
|
|
M(keys256) \
|
|
|
|
|
M(hashed)
|
2018-11-30 14:49:35 +00:00
|
|
|
|
|
|
|
|
|
/// Used for reading from StorageJoin and applying joinGet function
|
|
|
|
|
#define APPLY_FOR_JOIN_VARIANTS_LIMITED(M) \
|
|
|
|
|
M(key8) \
|
|
|
|
|
M(key16) \
|
|
|
|
|
M(key32) \
|
|
|
|
|
M(key64) \
|
|
|
|
|
M(key_string) \
|
|
|
|
|
M(key_fixed_string)
|
|
|
|
|
|
2024-05-08 23:08:33 +00:00
|
|
|
|
enum class Type : uint8_t
|
2017-03-28 06:51:22 +00:00
|
|
|
|
{
|
|
|
|
|
EMPTY,
|
|
|
|
|
CROSS,
|
|
|
|
|
#define M(NAME) NAME,
|
|
|
|
|
APPLY_FOR_JOIN_VARIANTS(M)
|
|
|
|
|
#undef M
|
|
|
|
|
};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
2017-03-29 11:12:47 +00:00
|
|
|
|
/** Different data structures, that are used to perform JOIN.
|
2014-06-18 19:14:29 +00:00
|
|
|
|
*/
|
2015-04-17 08:46:06 +00:00
|
|
|
|
template <typename Mapped>
|
|
|
|
|
struct MapsTemplate
|
2014-06-18 20:08:31 +00:00
|
|
|
|
{
|
2024-03-14 12:16:33 +00:00
|
|
|
|
/// NOLINTBEGIN(bugprone-macro-parentheses)
|
2021-11-08 12:44:13 +00:00
|
|
|
|
using MappedType = Mapped;
|
2021-01-27 00:54:57 +00:00
|
|
|
|
std::unique_ptr<FixedHashMap<UInt8, Mapped>> key8;
|
|
|
|
|
std::unique_ptr<FixedHashMap<UInt16, Mapped>> key16;
|
|
|
|
|
std::unique_ptr<HashMap<UInt32, Mapped, HashCRC32<UInt32>>> key32;
|
|
|
|
|
std::unique_ptr<HashMap<UInt64, Mapped, HashCRC32<UInt64>>> key64;
|
|
|
|
|
std::unique_ptr<HashMapWithSavedHash<StringRef, Mapped>> key_string;
|
|
|
|
|
std::unique_ptr<HashMapWithSavedHash<StringRef, Mapped>> key_fixed_string;
|
|
|
|
|
std::unique_ptr<HashMap<UInt128, Mapped, UInt128HashCRC32>> keys128;
|
|
|
|
|
std::unique_ptr<HashMap<UInt256, Mapped, UInt256HashCRC32>> keys256;
|
|
|
|
|
std::unique_ptr<HashMap<UInt128, Mapped, UInt128TrivialHash>> hashed;
|
2019-03-19 16:53:36 +00:00
|
|
|
|
|
|
|
|
|
void create(Type which)
|
|
|
|
|
{
|
|
|
|
|
switch (which)
|
|
|
|
|
{
|
|
|
|
|
case Type::EMPTY: break;
|
|
|
|
|
case Type::CROSS: break;
|
|
|
|
|
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
case Type::NAME: NAME = std::make_unique<typename decltype(NAME)::element_type>(); break;
|
|
|
|
|
APPLY_FOR_JOIN_VARIANTS(M)
|
|
|
|
|
#undef M
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2023-07-10 08:44:01 +00:00
|
|
|
|
void reserve(Type which, size_t num)
|
|
|
|
|
{
|
|
|
|
|
switch (which)
|
|
|
|
|
{
|
|
|
|
|
case Type::EMPTY: break;
|
|
|
|
|
case Type::CROSS: break;
|
|
|
|
|
case Type::key8: break;
|
|
|
|
|
case Type::key16: break;
|
|
|
|
|
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
case Type::NAME: NAME->reserve(num); break;
|
|
|
|
|
APPLY_FOR_HASH_JOIN_VARIANTS(M)
|
|
|
|
|
#undef M
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2019-03-19 16:53:36 +00:00
|
|
|
|
size_t getTotalRowCount(Type which) const
|
|
|
|
|
{
|
|
|
|
|
switch (which)
|
|
|
|
|
{
|
|
|
|
|
case Type::EMPTY: return 0;
|
|
|
|
|
case Type::CROSS: return 0;
|
|
|
|
|
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
case Type::NAME: return NAME ? NAME->size() : 0;
|
|
|
|
|
APPLY_FOR_JOIN_VARIANTS(M)
|
|
|
|
|
#undef M
|
|
|
|
|
}
|
|
|
|
|
|
2022-10-07 19:20:14 +00:00
|
|
|
|
UNREACHABLE();
|
2019-03-19 16:53:36 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
size_t getTotalByteCountImpl(Type which) const
|
|
|
|
|
{
|
|
|
|
|
switch (which)
|
|
|
|
|
{
|
|
|
|
|
case Type::EMPTY: return 0;
|
|
|
|
|
case Type::CROSS: return 0;
|
|
|
|
|
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
case Type::NAME: return NAME ? NAME->getBufferSizeInBytes() : 0;
|
|
|
|
|
APPLY_FOR_JOIN_VARIANTS(M)
|
|
|
|
|
#undef M
|
|
|
|
|
}
|
|
|
|
|
|
2022-10-07 19:20:14 +00:00
|
|
|
|
UNREACHABLE();
|
2019-03-19 16:53:36 +00:00
|
|
|
|
}
|
2021-02-04 14:46:36 +00:00
|
|
|
|
|
|
|
|
|
size_t getBufferSizeInCells(Type which) const
|
|
|
|
|
{
|
|
|
|
|
switch (which)
|
|
|
|
|
{
|
|
|
|
|
case Type::EMPTY: return 0;
|
|
|
|
|
case Type::CROSS: return 0;
|
|
|
|
|
|
|
|
|
|
#define M(NAME) \
|
|
|
|
|
case Type::NAME: return NAME ? NAME->getBufferSizeInCells() : 0;
|
|
|
|
|
APPLY_FOR_JOIN_VARIANTS(M)
|
|
|
|
|
#undef M
|
|
|
|
|
}
|
|
|
|
|
|
2022-10-07 19:20:14 +00:00
|
|
|
|
UNREACHABLE();
|
2021-02-04 14:46:36 +00:00
|
|
|
|
}
|
2024-03-14 12:16:33 +00:00
|
|
|
|
/// NOLINTEND(bugprone-macro-parentheses)
|
2014-06-18 20:08:31 +00:00
|
|
|
|
};
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
2021-02-04 14:46:36 +00:00
|
|
|
|
using MapsOne = MapsTemplate<RowRef>;
|
|
|
|
|
using MapsAll = MapsTemplate<RowRefList>;
|
|
|
|
|
using MapsAsof = MapsTemplate<AsofRowRefs>;
|
2018-12-30 15:54:45 +00:00
|
|
|
|
|
2021-02-04 14:46:36 +00:00
|
|
|
|
using MapsVariant = std::variant<MapsOne, MapsAll, MapsAsof>;
|
2021-06-25 12:03:10 +00:00
|
|
|
|
|
2021-08-17 10:27:23 +00:00
|
|
|
|
using RawBlockPtr = const Block *;
|
|
|
|
|
using BlockNullmapList = std::deque<std::pair<RawBlockPtr, ColumnPtr>>;
|
|
|
|
|
|
2019-12-19 15:50:28 +00:00
|
|
|
|
struct RightTableData
|
|
|
|
|
{
|
|
|
|
|
Type type = Type::EMPTY;
|
|
|
|
|
bool empty = true;
|
|
|
|
|
|
2021-06-25 12:03:10 +00:00
|
|
|
|
std::vector<MapsVariant> maps;
|
2019-12-19 15:50:28 +00:00
|
|
|
|
Block sample_block; /// Block as it would appear in the BlockList
|
2021-09-06 10:59:18 +00:00
|
|
|
|
BlocksList blocks; /// Blocks of "right" table.
|
2019-12-19 15:50:28 +00:00
|
|
|
|
BlockNullmapList blocks_nullmaps; /// Nullmaps for blocks of "right" table (if needed)
|
|
|
|
|
|
|
|
|
|
/// Additional data - strings for string keys and continuation elements of single-linked lists of references to rows.
|
|
|
|
|
Arena pool;
|
2022-10-19 10:20:41 +00:00
|
|
|
|
|
2022-12-19 15:19:49 +00:00
|
|
|
|
size_t blocks_allocated_size = 0;
|
|
|
|
|
size_t blocks_nullmaps_allocated_size = 0;
|
2019-12-19 15:50:28 +00:00
|
|
|
|
};
|
|
|
|
|
|
2021-08-17 10:27:23 +00:00
|
|
|
|
using RightTableDataPtr = std::shared_ptr<RightTableData>;
|
2021-06-25 12:03:10 +00:00
|
|
|
|
|
2021-02-25 09:31:22 +00:00
|
|
|
|
/// We keep correspondence between used_flags and hash table internal buffer.
|
|
|
|
|
/// Hash table cannot be modified during HashJoin lifetime and must be protected with lock.
|
2022-06-23 07:59:13 +00:00
|
|
|
|
void setLock(TableLockHolder rwlock_holder)
|
2019-12-19 15:50:28 +00:00
|
|
|
|
{
|
2021-09-29 17:30:07 +00:00
|
|
|
|
storage_join_lock = rwlock_holder;
|
2019-12-19 15:50:28 +00:00
|
|
|
|
}
|
2014-06-18 20:08:31 +00:00
|
|
|
|
|
2021-02-04 14:46:36 +00:00
|
|
|
|
void reuseJoinedData(const HashJoin & join);
|
2014-06-18 20:08:31 +00:00
|
|
|
|
|
2021-08-17 10:27:23 +00:00
|
|
|
|
RightTableDataPtr getJoinedData() const { return data; }
|
2022-12-20 12:50:27 +00:00
|
|
|
|
BlocksList releaseJoinedBlocks(bool restructure = false);
|
|
|
|
|
|
2023-01-23 18:09:26 +00:00
|
|
|
|
/// Modify right block (update structure according to sample block) to save it in block list
|
2022-12-20 12:50:27 +00:00
|
|
|
|
static Block prepareRightBlock(const Block & block, const Block & saved_block_sample_);
|
|
|
|
|
Block prepareRightBlock(const Block & block) const;
|
|
|
|
|
|
|
|
|
|
const Block & savedBlockSample() const { return data->sample_block; }
|
2020-02-17 17:08:31 +00:00
|
|
|
|
|
2021-02-04 14:46:36 +00:00
|
|
|
|
bool isUsed(size_t off) const { return used_flags.getUsedSafe(off); }
|
2021-08-17 10:27:23 +00:00
|
|
|
|
bool isUsed(const Block * block_ptr, size_t row_idx) const { return used_flags.getUsedSafe(block_ptr, row_idx); }
|
2021-02-04 14:46:36 +00:00
|
|
|
|
|
2022-10-04 08:20:13 +00:00
|
|
|
|
void debugKeys() const;
|
|
|
|
|
|
2023-09-14 12:03:30 +00:00
|
|
|
|
void shrinkStoredBlocksToFit(size_t & total_bytes_in_join);
|
|
|
|
|
|
2024-01-08 10:20:39 +00:00
|
|
|
|
void setMaxJoinedBlockRows(size_t value) { max_joined_block_rows = value; }
|
|
|
|
|
|
2014-06-18 19:14:29 +00:00
|
|
|
|
private:
|
2024-01-10 12:30:23 +00:00
|
|
|
|
friend class NotJoinedHash;
|
2021-06-25 12:03:10 +00:00
|
|
|
|
|
2020-01-31 14:51:09 +00:00
|
|
|
|
friend class JoinSource;
|
2015-04-17 08:46:06 +00:00
|
|
|
|
|
2020-04-07 09:48:47 +00:00
|
|
|
|
std::shared_ptr<TableJoin> table_join;
|
2022-10-06 12:22:44 +00:00
|
|
|
|
const JoinKind kind;
|
|
|
|
|
const JoinStrictness strictness;
|
2014-06-18 19:14:29 +00:00
|
|
|
|
|
2021-04-28 17:32:12 +00:00
|
|
|
|
/// This join was created from StorageJoin and it is already filled.
|
|
|
|
|
bool from_storage_join = false;
|
|
|
|
|
|
2024-04-24 12:05:13 +00:00
|
|
|
|
const bool any_take_last_row; /// Overwrite existing values when encountering the same key again
|
|
|
|
|
const size_t reserve_num;
|
|
|
|
|
const String instance_id;
|
2020-06-01 09:38:46 +00:00
|
|
|
|
std::optional<TypeIndex> asof_type;
|
2022-10-06 12:22:44 +00:00
|
|
|
|
const ASOFJoinInequality asof_inequality;
|
2015-03-02 01:10:58 +00:00
|
|
|
|
|
2019-12-19 15:50:28 +00:00
|
|
|
|
/// Right table data. StorageJoin shares it between many Join objects.
|
2021-02-08 11:38:31 +00:00
|
|
|
|
/// Flags that indicate that particular row already used in join.
|
|
|
|
|
/// Flag is stored for every record in hash map.
|
|
|
|
|
/// Number of this flags equals to hashtable buffer size (plus one for zero value).
|
2021-02-25 09:31:22 +00:00
|
|
|
|
/// Changes in hash table broke correspondence,
|
|
|
|
|
/// so we must guarantee constantness of hash table during HashJoin lifetime (using method setLock)
|
2021-02-04 14:46:36 +00:00
|
|
|
|
mutable JoinStuff::JoinUsedFlags used_flags;
|
2021-06-25 12:03:10 +00:00
|
|
|
|
RightTableDataPtr data;
|
2021-09-06 10:59:18 +00:00
|
|
|
|
std::vector<Sizes> key_sizes;
|
2014-06-12 02:31:30 +00:00
|
|
|
|
|
2024-04-28 11:20:59 +00:00
|
|
|
|
/// Needed to do external cross join
|
|
|
|
|
TemporaryDataOnDiskPtr tmp_data;
|
|
|
|
|
TemporaryFileStream* tmp_stream{nullptr};
|
|
|
|
|
|
2020-07-10 18:10:06 +00:00
|
|
|
|
/// Block with columns from the right-side table.
|
|
|
|
|
Block right_sample_block;
|
2017-04-27 17:48:30 +00:00
|
|
|
|
/// Block with columns from the right-side table except key columns.
|
2015-06-09 23:50:22 +00:00
|
|
|
|
Block sample_block_with_columns_to_add;
|
2019-09-11 15:57:09 +00:00
|
|
|
|
/// Block with key columns in the same order they appear in the right-side table (duplicates appear once).
|
|
|
|
|
Block right_table_keys;
|
2019-11-01 17:41:07 +00:00
|
|
|
|
/// Block with key columns right-side table keys that are needed in result (would be attached after joined columns).
|
|
|
|
|
Block required_right_keys;
|
|
|
|
|
/// Left table column names that are sources for required_right_keys columns
|
|
|
|
|
std::vector<String> required_right_keys_sources;
|
2015-05-26 00:37:48 +00:00
|
|
|
|
|
2024-01-08 10:20:39 +00:00
|
|
|
|
/// Maximum number of rows in result block. If it is 0, then no limits.
|
|
|
|
|
size_t max_joined_block_rows = 0;
|
|
|
|
|
|
2023-09-14 01:50:35 +00:00
|
|
|
|
/// When tracked memory consumption is more than a threshold, we will shrink to fit stored blocks.
|
2023-09-13 09:59:02 +00:00
|
|
|
|
bool shrink_blocks = false;
|
2023-09-14 12:03:30 +00:00
|
|
|
|
Int64 memory_usage_before_adding_blocks = 0;
|
2023-09-13 09:59:02 +00:00
|
|
|
|
|
2023-11-28 10:28:11 +00:00
|
|
|
|
/// Identifier to distinguish different HashJoin instances in logs
|
|
|
|
|
/// Several instances can be created, for example, in GraceHashJoin to handle different buckets
|
|
|
|
|
String instance_log_id;
|
|
|
|
|
|
2024-01-23 17:04:50 +00:00
|
|
|
|
LoggerPtr log;
|
2014-07-06 19:48:39 +00:00
|
|
|
|
|
2021-02-25 09:31:22 +00:00
|
|
|
|
/// Should be set via setLock to protect hash table from modification from StorageJoin
|
2023-07-05 17:03:18 +00:00
|
|
|
|
/// If set HashJoin instance is not available for modification (addBlockToJoin)
|
2022-06-23 07:59:13 +00:00
|
|
|
|
TableLockHolder storage_join_lock = nullptr;
|
2021-02-20 15:00:59 +00:00
|
|
|
|
|
2024-04-24 12:05:13 +00:00
|
|
|
|
void dataMapInit(MapsVariant & map);
|
2019-12-19 15:50:28 +00:00
|
|
|
|
|
2020-02-18 12:41:23 +00:00
|
|
|
|
void initRightBlockStructure(Block & saved_block_sample);
|
2019-04-01 10:35:37 +00:00
|
|
|
|
|
2022-07-29 16:30:50 +00:00
|
|
|
|
template <JoinKind KIND, JoinStrictness STRICTNESS, typename Maps>
|
2023-11-20 14:56:05 +00:00
|
|
|
|
Block joinBlockImpl(
|
2019-11-07 12:42:39 +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 = false) const;
|
2014-06-18 18:31:35 +00:00
|
|
|
|
|
2020-04-06 13:39:57 +00:00
|
|
|
|
void joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const;
|
2019-11-07 12:42:39 +00:00
|
|
|
|
|
2022-07-29 16:30:50 +00:00
|
|
|
|
static Type chooseMethod(JoinKind kind, const ColumnRawPtrs & key_columns, Sizes & key_sizes);
|
2020-12-30 13:52:37 +00:00
|
|
|
|
|
2021-01-04 23:49:31 +00:00
|
|
|
|
bool empty() const;
|
2024-02-29 11:53:23 +00:00
|
|
|
|
|
2024-03-07 11:51:54 +00:00
|
|
|
|
void validateAdditionalFilterExpression(std::shared_ptr<ExpressionActions> additional_filter_expression);
|
|
|
|
|
bool needUsedFlagsForPerRightTableRow(std::shared_ptr<TableJoin> table_join_) const;
|
2014-06-12 02:31:30 +00:00
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
}
|