ClickHouse/src/Interpreters/HashJoin.h

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

500 lines
19 KiB
C++
Raw Normal View History

2014-06-12 02:31:30 +00:00
#pragma once
#include <variant>
#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
#include <Parsers/ASTTablesInSelectQuery.h>
2019-09-09 19:43:37 +00:00
#include <Interpreters/IJoin.h>
#include <Interpreters/AggregationCommon.h>
2019-03-30 21:30:21 +00:00
#include <Interpreters/RowRefs.h>
2014-06-12 02:31:30 +00:00
#include <Common/Arena.h>
2019-01-24 14:56:04 +00:00
#include <Common/ColumnsHashing.h>
#include <Common/HashTable/HashMap.h>
A Proper lookup table that uses HashTable's API This is the first step of allowing heterogeneous cells in hash tables. performance test results are ``` 1. HashMap<UInt16, UInt8, TrivialHash, HashTableFixedGrower<16>>; 2. NewLookupMap<UInt16, UInt8> ResolutionWidth 30000 1 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................223550276.46 ResolutionWidth 30000 2 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................248772721.24 Best: 2 - 24877272124 ResolutionWidth 100000 1 ..........................................................................................................................................................................................................................................................238498413.99 ResolutionWidth 100000 2 ..........................................................................................................................................................................................................................................................261808889.98 Best: 2 - 26180888998 ResolutionWidth 300000 1 ...................................................................................239307348.81 ResolutionWidth 300000 2 ...................................................................................257592761.30 Best: 2 - 25759276130 ResolutionWidth 1000000 1 .........................240144759.26 ResolutionWidth 1000000 2 .........................257093531.91 Best: 2 - 25709353191 ResolutionWidth 5000000 1 .....241573260.35 ResolutionWidth 5000000 2 .....259314162.79 Best: 2 - 25931416279 ResolutionDepth 30000 1 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................217108119.84 ResolutionDepth 30000 2 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................249459504.41 Best: 2 - 24945950441 ResolutionDepth 100000 1 ..........................................................................................................................................................................................................................................................229065162.17 ResolutionDepth 100000 2 ..........................................................................................................................................................................................................................................................253769105.64 Best: 2 - 25376910564 ResolutionDepth 300000 1 ...................................................................................233079225.18 ResolutionDepth 300000 2 ...................................................................................256316273.78 Best: 2 - 25631627378 ResolutionDepth 1000000 1 .........................234184633.51 ResolutionDepth 1000000 2 .........................261100491.57 Best: 2 - 26110049157 ResolutionDepth 5000000 1 .....233118795.66 ResolutionDepth 5000000 2 .....252436160.41 Best: 2 - 25243616041 ```
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
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
2021-10-15 20:18:20 +00:00
#include <QueryPipeline/SizeLimits.h>
2014-06-12 02:31:30 +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>
#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
{
class TableJoin;
class ExpressionActions;
namespace JoinStuff
{
2019-05-08 10:53:45 +00:00
/// Flags needed to implement RIGHT and FULL JOINs.
class JoinUsedFlags
{
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;
bool need_flags;
2019-05-08 10:53:45 +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>
void reinit(size_t size_);
2022-07-29 16:30:50 +00:00
template <JoinKind KIND, JoinStrictness STRICTNESS>
void reinit(const Block * block_ptr);
bool getUsedSafe(size_t i) const;
bool getUsedSafe(const Block * block_ptr, size_t row_idx) const;
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
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);
template <bool use_flags, bool flag_per_row, typename T>
2021-06-25 12:03:10 +00:00
bool getUsed(const T & f);
template <bool use_flags, bool flag_per_row, typename T>
2021-06-25 12:03:10 +00:00
bool setUsedOnce(const T & f);
};
}
/** 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
*
* 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.
*
* Thus, LEFT and RIGHT JOINs are not symmetric in terms of implementation.
2014-06-19 19:00:58 +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
*
* Implementation:
2014-06-19 19:00:58 +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
*
* 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
*
* 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
*
* 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
*
* 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.
*
* 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.
*
* 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 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,
bool any_take_last_row_ = false, size_t reserve_num_ = 0, const String & instance_id_ = "");
~HashJoin() override;
2023-08-03 15:55:15 +00:00
std::string getName() const override { return "HashJoin"; }
const TableJoin & getTableJoin() const override { return *table_join; }
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);
}
/** 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.
*/
2023-07-05 17:03:18 +00:00
bool addBlockToJoin(const Block & source_block_, bool check_limits) override;
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.
* Could be called from different threads in parallel.
*/
2020-01-15 20:33:29 +00:00
void joinBlock(Block & block, ExtraBlockPtr & not_processed) override;
/// Check joinGet arguments and infer the return type.
DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const;
/// Used by joinGet function that turns StorageJoin into a dictionary.
ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const;
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;
/// 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
/** 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.
* left_sample_block is passed without account of 'use_nulls' setting (columns will be converted to Nullable inside).
*/
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;
/// Number of keys in all built JOIN maps.
2019-10-11 17:27:54 +00:00
size_t getTotalRowCount() const final;
/// Sum size in bytes of all buffers, used for JOIN maps and for all memory pools.
size_t getTotalByteCount() const final;
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; }
const std::optional<TypeIndex> & getAsofType() const { return asof_type; }
2022-07-29 16:30:50 +00:00
ASOFJoinInequality getAsofInequality() const { return asof_inequality; }
bool anyTakeLastRow() const { return any_take_last_row; }
2021-09-09 09:47:08 +00:00
const ColumnWithTypeAndName & rightAsofKeyColumn() const;
/// 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) \
M(hashed)
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)
/// 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
{
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
{
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
}
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
}
UNREACHABLE();
2019-03-19 16:53: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
}
UNREACHABLE();
}
2024-03-14 12:16:33 +00:00
/// NOLINTEND(bugprone-macro-parentheses)
};
using MapsOne = MapsTemplate<RowRef>;
using MapsAll = MapsTemplate<RowRefList>;
using MapsAsof = MapsTemplate<AsofRowRefs>;
using MapsVariant = std::variant<MapsOne, MapsAll, MapsAsof>;
2021-06-25 12:03:10 +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
size_t blocks_allocated_size = 0;
size_t blocks_nullmaps_allocated_size = 0;
2019-12-19 15:50:28 +00:00
};
using RightTableDataPtr = std::shared_ptr<RightTableData>;
2021-06-25 12:03:10 +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
{
storage_join_lock = rwlock_holder;
2019-12-19 15:50:28 +00:00
}
void reuseJoinedData(const HashJoin & join);
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; }
bool isUsed(size_t off) const { return used_flags.getUsedSafe(off); }
bool isUsed(const Block * block_ptr, size_t row_idx) const { return used_flags.getUsedSafe(block_ptr, row_idx); }
2022-10-04 08:20:13 +00:00
void debugKeys() const;
void shrinkStoredBlocksToFit(size_t & total_bytes_in_join);
void setMaxJoinedBlockRows(size_t value) { max_joined_block_rows = value; }
private:
friend class NotJoinedHash;
2021-06-25 12:03:10 +00:00
friend class JoinSource;
std::shared_ptr<TableJoin> table_join;
const JoinKind kind;
const JoinStrictness strictness;
2021-04-28 17:32:12 +00:00
/// This join was created from StorageJoin and it is already filled.
bool from_storage_join = false;
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;
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.
/// 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).
/// Changes in hash table broke correspondence,
/// so we must guarantee constantness of hash table during HashJoin lifetime (using method setLock)
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};
/// Block with columns from the right-side table.
Block right_sample_block;
/// Block with columns from the right-side table except key columns.
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;
/// 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;
/// 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.
bool shrink_blocks = false;
Int64 memory_usage_before_adding_blocks = 0;
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;
/// 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;
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>
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_,
bool is_join_get = false) const;
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;
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
};
}