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>
|
2017-07-28 17:34:02 +00:00
|
|
|
|
#include <shared_mutex>
|
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
|
|
|
|
|
2017-04-01 09:19:00 +00:00
|
|
|
|
#include <Interpreters/AggregationCommon.h>
|
2019-03-30 21:30:21 +00:00
|
|
|
|
#include <Interpreters/RowRefs.h>
|
2019-03-22 12:08:30 +00:00
|
|
|
|
#include <Core/SettingsCommon.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>
|
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
|
|
|
|
|
2018-03-11 00:15:26 +00:00
|
|
|
|
#include <DataStreams/SizeLimits.h>
|
2019-05-17 14:34:25 +00:00
|
|
|
|
#include <DataStreams/IBlockStream_fwd.h>
|
2014-06-12 02:31:30 +00:00
|
|
|
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
|
{
|
2019-05-07 18:21:44 +00:00
|
|
|
|
|
|
|
|
|
namespace JoinStuff
|
|
|
|
|
{
|
2019-05-08 10:53:45 +00:00
|
|
|
|
|
|
|
|
|
/// Base class with optional flag attached that's needed to implement RIGHT and FULL JOINs.
|
|
|
|
|
template <typename T, bool with_used>
|
2019-05-07 18:21:44 +00:00
|
|
|
|
struct WithFlags;
|
|
|
|
|
|
2019-05-08 10:53:45 +00:00
|
|
|
|
template <typename T>
|
|
|
|
|
struct WithFlags<T, true> : T
|
2019-05-07 18:21:44 +00:00
|
|
|
|
{
|
2019-05-08 10:53:45 +00:00
|
|
|
|
using Base = T;
|
|
|
|
|
using T::T;
|
|
|
|
|
|
2019-05-07 18:21:44 +00:00
|
|
|
|
mutable std::atomic<bool> used {};
|
|
|
|
|
void setUsed() const { used.store(true, std::memory_order_relaxed); } /// Could be set simultaneously from different threads.
|
|
|
|
|
bool getUsed() const { return used; }
|
|
|
|
|
};
|
|
|
|
|
|
2019-05-08 10:53:45 +00:00
|
|
|
|
template <typename T>
|
|
|
|
|
struct WithFlags<T, false> : T
|
2019-05-07 18:21:44 +00:00
|
|
|
|
{
|
2019-05-08 10:53:45 +00:00
|
|
|
|
using Base = T;
|
|
|
|
|
using T::T;
|
|
|
|
|
|
2019-05-07 18:21:44 +00:00
|
|
|
|
void setUsed() const {}
|
|
|
|
|
bool getUsed() const { return true; }
|
|
|
|
|
};
|
|
|
|
|
|
2019-05-08 10:53:45 +00:00
|
|
|
|
using MappedAny = WithFlags<RowRef, false>;
|
|
|
|
|
using MappedAll = WithFlags<RowRefList, false>;
|
|
|
|
|
using MappedAnyFull = WithFlags<RowRef, true>;
|
|
|
|
|
using MappedAllFull = WithFlags<RowRefList, true>;
|
|
|
|
|
using MappedAsof = WithFlags<AsofRowRefs, false>;
|
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
|
|
|
|
*
|
2017-03-29 11:12:47 +00:00
|
|
|
|
* JOIN-s could be of nine types: ANY/ALL × LEFT/INNER/RIGHT/FULL, and also CROSS.
|
2014-06-19 19:00:58 +00:00
|
|
|
|
*
|
2017-03-29 11:12:47 +00:00
|
|
|
|
* If ANY is specified - then select only one row from the "right" table, (first encountered row), even if there was more matching rows.
|
|
|
|
|
* If ALL is specified - usual JOIN, when rows are multiplied by number of matching rows from the "right" table.
|
|
|
|
|
* ANY is more efficient.
|
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
|
|
|
|
*/
|
|
|
|
|
class Join
|
|
|
|
|
{
|
|
|
|
|
public:
|
2018-11-30 14:49:35 +00:00
|
|
|
|
Join(const Names & key_names_right_, bool use_nulls_, const SizeLimits & limits,
|
2019-01-24 17:12:05 +00:00
|
|
|
|
ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_, bool any_take_last_row_ = false);
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
|
|
bool empty() { return type == Type::EMPTY; }
|
|
|
|
|
|
|
|
|
|
/** Set information about structure of right hand of JOIN (joined data).
|
|
|
|
|
* You must call this method before subsequent calls to insertFromBlock.
|
|
|
|
|
*/
|
|
|
|
|
void setSampleBlock(const Block & block);
|
|
|
|
|
|
|
|
|
|
/** 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.
|
|
|
|
|
*/
|
|
|
|
|
bool insertFromBlock(const Block & block);
|
|
|
|
|
|
|
|
|
|
/** Join data from the map (that was previously built by calls to insertFromBlock) to the block with data from "left" table.
|
|
|
|
|
* Could be called from different threads in parallel.
|
|
|
|
|
*/
|
2019-01-30 15:51:39 +00:00
|
|
|
|
void joinBlock(Block & block, const Names & key_names_left, const NamesAndTypesList & columns_added_by_join) const;
|
2018-11-30 14:49:35 +00:00
|
|
|
|
|
|
|
|
|
/// Infer the return type for joinGet function
|
|
|
|
|
DataTypePtr joinGetReturnType(const String & column_name) const;
|
|
|
|
|
|
|
|
|
|
/// Used by joinGet function that turns StorageJoin into a dictionary
|
|
|
|
|
void joinGet(Block & block, const String & column_name) const;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
|
|
/** Keep "totals" (separate part of dataset, see WITH TOTALS) to use later.
|
|
|
|
|
*/
|
|
|
|
|
void setTotals(const Block & block) { totals = block; }
|
2018-06-03 20:39:06 +00:00
|
|
|
|
bool hasTotals() const { return totals; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
|
|
void joinTotals(Block & block) const;
|
|
|
|
|
|
|
|
|
|
/** 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).
|
2017-04-01 07:20:54 +00:00
|
|
|
|
*/
|
2019-01-31 18:10:16 +00:00
|
|
|
|
BlockInputStreamPtr createStreamWithNonJoinedRows(const Block & left_sample_block, const Names & key_names_left,
|
2019-02-10 16:55:12 +00:00
|
|
|
|
const NamesAndTypesList & columns_added_by_join, UInt64 max_block_size) const;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
|
|
/// Number of keys in all built JOIN maps.
|
|
|
|
|
size_t getTotalRowCount() const;
|
|
|
|
|
/// Sum size in bytes of all buffers, used for JOIN maps and for all memory pools.
|
|
|
|
|
size_t getTotalByteCount() const;
|
|
|
|
|
|
|
|
|
|
ASTTableJoin::Kind getKind() const { return kind; }
|
2019-04-01 16:44:15 +00:00
|
|
|
|
AsofRowRefs::Type getAsofType() const { return *asof_type; }
|
2019-05-07 18:21:44 +00:00
|
|
|
|
bool anyTakeLastRow() const { return any_take_last_row; }
|
2017-04-01 07:20:54 +00:00
|
|
|
|
|
|
|
|
|
/// Different types of keys for maps.
|
|
|
|
|
#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-04-01 07:20:54 +00:00
|
|
|
|
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)
|
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
enum class Type
|
|
|
|
|
{
|
|
|
|
|
EMPTY,
|
|
|
|
|
CROSS,
|
|
|
|
|
#define M(NAME) NAME,
|
|
|
|
|
APPLY_FOR_JOIN_VARIANTS(M)
|
|
|
|
|
#undef M
|
|
|
|
|
};
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
/** Different data structures, that are used to perform JOIN.
|
|
|
|
|
*/
|
|
|
|
|
template <typename Mapped>
|
|
|
|
|
struct MapsTemplate
|
|
|
|
|
{
|
2019-02-28 09:35:38 +00:00
|
|
|
|
std::unique_ptr<FixedHashMap<UInt8, Mapped>> key8;
|
|
|
|
|
std::unique_ptr<FixedHashMap<UInt16, Mapped>> key16;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
std::unique_ptr<HashMap<UInt32, Mapped, HashCRC32<UInt32>>> key32;
|
|
|
|
|
std::unique_ptr<HashMap<UInt64, Mapped, HashCRC32<UInt64>>> key64;
|
2017-11-12 00:46:22 +00:00
|
|
|
|
std::unique_ptr<HashMapWithSavedHash<StringRef, Mapped>> key_string;
|
|
|
|
|
std::unique_ptr<HashMapWithSavedHash<StringRef, Mapped>> key_fixed_string;
|
2017-04-01 07:20:54 +00:00
|
|
|
|
std::unique_ptr<HashMap<UInt128, Mapped, UInt128HashCRC32>> keys128;
|
|
|
|
|
std::unique_ptr<HashMap<UInt256, Mapped, UInt256HashCRC32>> keys256;
|
2017-11-12 00:46:22 +00:00
|
|
|
|
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
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
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
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
__builtin_unreachable();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
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
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
__builtin_unreachable();
|
|
|
|
|
}
|
2017-04-01 07:20:54 +00:00
|
|
|
|
};
|
|
|
|
|
|
2019-05-07 18:21:44 +00:00
|
|
|
|
using MapsAny = MapsTemplate<JoinStuff::MappedAny>;
|
|
|
|
|
using MapsAll = MapsTemplate<JoinStuff::MappedAll>;
|
|
|
|
|
using MapsAnyFull = MapsTemplate<JoinStuff::MappedAnyFull>;
|
|
|
|
|
using MapsAllFull = MapsTemplate<JoinStuff::MappedAllFull>;
|
|
|
|
|
using MapsAsof = MapsTemplate<JoinStuff::MappedAsof>;
|
2018-12-30 15:54:45 +00:00
|
|
|
|
|
2019-07-02 14:38:31 +00:00
|
|
|
|
using MapsVariant = std::variant<MapsAny, MapsAll, MapsAnyFull, MapsAllFull, MapsAsof>;
|
2014-06-18 20:08:31 +00:00
|
|
|
|
|
2014-06-18 19:14:29 +00:00
|
|
|
|
private:
|
2017-04-01 07:20:54 +00:00
|
|
|
|
friend class NonJoinedBlockInputStream;
|
2018-11-30 14:49:35 +00:00
|
|
|
|
friend class JoinBlockInputStream;
|
2015-04-17 08:46:06 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
ASTTableJoin::Kind kind;
|
|
|
|
|
ASTTableJoin::Strictness strictness;
|
2014-06-18 19:14:29 +00:00
|
|
|
|
|
2017-04-27 17:48:30 +00:00
|
|
|
|
/// Names of key columns (columns for equi-JOIN) in "right" table (in the order they appear in USING clause).
|
2017-04-01 07:20:54 +00:00
|
|
|
|
const Names key_names_right;
|
2014-06-18 19:14:29 +00:00
|
|
|
|
|
2017-04-04 06:52:39 +00:00
|
|
|
|
/// Substitute NULLs for non-JOINed rows.
|
|
|
|
|
bool use_nulls;
|
|
|
|
|
|
2018-12-30 15:54:45 +00:00
|
|
|
|
/// Overwrite existing values when encountering the same key again
|
2019-01-24 17:12:05 +00:00
|
|
|
|
bool any_take_last_row;
|
2018-12-30 15:54:45 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
/** Blocks of "right" table.
|
|
|
|
|
*/
|
|
|
|
|
BlocksList blocks;
|
2014-06-12 02:31:30 +00:00
|
|
|
|
|
2019-07-02 14:38:31 +00:00
|
|
|
|
MapsVariant maps;
|
2014-06-12 02:31:30 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
/// Additional data - strings for string keys and continuation elements of single-linked lists of references to rows.
|
|
|
|
|
Arena pool;
|
2014-06-12 02:31:30 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
Type type = Type::EMPTY;
|
2019-04-01 16:44:15 +00:00
|
|
|
|
std::optional<AsofRowRefs::Type> asof_type;
|
2015-03-02 01:10:58 +00:00
|
|
|
|
|
2017-12-13 01:27:53 +00:00
|
|
|
|
static Type chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes);
|
2014-07-06 19:48:39 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
Sizes key_sizes;
|
2014-06-12 02:31:30 +00:00
|
|
|
|
|
2017-04-27 17:48:30 +00:00
|
|
|
|
/// Block with columns from the right-side table except key columns.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
Block sample_block_with_columns_to_add;
|
2017-04-27 17:48:30 +00:00
|
|
|
|
/// Block with key columns in the same order they appear in the right-side table.
|
2017-04-01 07:20:54 +00:00
|
|
|
|
Block sample_block_with_keys;
|
2015-05-26 00:37:48 +00:00
|
|
|
|
|
2019-03-31 23:09:00 +00:00
|
|
|
|
/// Block as it would appear in the BlockList
|
|
|
|
|
Block blocklist_sample;
|
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
Poco::Logger * log;
|
2014-07-06 19:48:39 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
/// Limits for maximum map size.
|
2018-03-11 00:15:26 +00:00
|
|
|
|
SizeLimits limits;
|
2014-06-12 02:31:30 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
Block totals;
|
2015-04-16 09:55:24 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
/** Protect state for concurrent use in insertFromBlock and joinBlock.
|
|
|
|
|
* Note that these methods could be called simultaneously only while use of StorageJoin,
|
|
|
|
|
* and StorageJoin only calls these two methods.
|
|
|
|
|
* That's why another methods are not guarded.
|
|
|
|
|
*/
|
2017-07-28 17:34:02 +00:00
|
|
|
|
mutable std::shared_mutex rwlock;
|
2015-01-27 21:24:24 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
void init(Type type_);
|
2014-06-18 20:08:31 +00:00
|
|
|
|
|
2019-04-01 10:35:37 +00:00
|
|
|
|
/** Take an inserted block and discard everything that does not need to be stored
|
|
|
|
|
* Example, remove the keys as they come from the LHS block, but do keep the ASOF timestamps
|
|
|
|
|
*/
|
|
|
|
|
void prepareBlockListStructure(Block & stored_block);
|
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
/// Throw an exception if blocks have different types of key columns.
|
2018-11-30 14:49:35 +00:00
|
|
|
|
void checkTypesOfKeys(const Block & block_left, const Names & key_names_left, const Block & block_right) const;
|
2017-03-28 06:51:22 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename Maps>
|
2018-11-30 14:49:35 +00:00
|
|
|
|
void joinBlockImpl(
|
|
|
|
|
Block & block,
|
|
|
|
|
const Names & key_names_left,
|
2019-01-31 18:10:16 +00:00
|
|
|
|
const NamesAndTypesList & columns_added_by_join,
|
2018-11-30 14:49:35 +00:00
|
|
|
|
const Block & block_with_columns_to_add,
|
|
|
|
|
const Maps & maps) const;
|
2014-06-18 18:31:35 +00:00
|
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
|
void joinBlockImplCross(Block & block) const;
|
2018-11-30 14:49:35 +00:00
|
|
|
|
|
|
|
|
|
template <typename Maps>
|
|
|
|
|
void joinGetImpl(Block & block, const String & column_name, const Maps & maps) const;
|
2014-06-12 02:31:30 +00:00
|
|
|
|
};
|
|
|
|
|
|
2016-05-28 10:35:44 +00:00
|
|
|
|
using JoinPtr = std::shared_ptr<Join>;
|
|
|
|
|
using Joins = std::vector<JoinPtr>;
|
2014-06-12 02:31:30 +00:00
|
|
|
|
|
|
|
|
|
}
|