ClickHouse/dbms/src/Interpreters/MergeJoin.h

132 lines
4.0 KiB
C++
Raw Normal View History

2019-09-09 19:43:37 +00:00
#pragma once
#include <memory>
2019-09-10 14:51:28 +00:00
#include <shared_mutex>
2019-09-09 19:43:37 +00:00
#include <Common/LRUCache.h>
#include <Common/filesystemHelpers.h>
2019-09-09 19:43:37 +00:00
#include <Core/Block.h>
2019-09-12 18:06:25 +00:00
#include <Core/SortDescription.h>
2019-09-09 19:43:37 +00:00
#include <Interpreters/IJoin.h>
#include <DataStreams/SizeLimits.h>
2019-09-09 19:43:37 +00:00
namespace DB
{
class AnalyzedJoin;
2019-09-13 16:17:37 +00:00
class MergeJoinCursor;
struct MergeJoinEqualRange;
2019-09-09 19:43:37 +00:00
struct MiniLSM
{
using SortedFiles = std::vector<std::unique_ptr<TemporaryFile>>;
const String & path;
const Block & sample_block;
const SortDescription & sort_description;
const size_t rows_in_block;
const size_t max_size;
std::vector<SortedFiles> sorted_files;
MiniLSM(const String & path_, const Block & sample_block_, const SortDescription & description,
size_t rows_in_block_, size_t max_size_ = 16)
: path(path_)
, sample_block(sample_block_)
, sort_description(description)
, rows_in_block(rows_in_block_)
, max_size(max_size_)
{}
void insert(const BlocksList & blocks);
void merge(std::function<void(const Block &)> callback = [](const Block &){});
};
2019-09-09 19:43:37 +00:00
class MergeJoin : public IJoin
{
public:
MergeJoin(std::shared_ptr<AnalyzedJoin> table_join_, const Block & right_sample_block);
2019-09-09 19:43:37 +00:00
2019-09-10 14:51:28 +00:00
bool addJoinedBlock(const Block & block) override;
void joinBlock(Block &) override;
2019-09-19 14:53:03 +00:00
void joinTotals(Block &) const override;
2019-09-12 18:06:25 +00:00
void setTotals(const Block &) override;
2019-09-23 14:37:42 +00:00
bool hasTotals() const override { return totals; }
2019-09-10 14:51:28 +00:00
size_t getTotalRowCount() const override { return right_blocks_row_count; }
2019-09-09 19:43:37 +00:00
private:
/// There're two size limits for right-hand table: max_rows_in_join, max_bytes_in_join.
/// max_bytes is prefered. If it isn't set we aproximate it as (max_rows * bytes/row).
struct BlockByteWeight
{
size_t operator()(const Block & block) const { return block.bytes(); }
};
using Cache = LRUCache<size_t, Block, std::hash<size_t>, BlockByteWeight>;
2019-09-10 14:51:28 +00:00
mutable std::shared_mutex rwlock;
std::shared_ptr<AnalyzedJoin> table_join;
SizeLimits size_limits;
2019-09-12 18:06:25 +00:00
SortDescription left_sort_description;
2019-09-13 16:17:37 +00:00
SortDescription right_sort_description;
SortDescription left_merge_description;
SortDescription right_merge_description;
Block right_sample_block;
2019-09-11 16:19:33 +00:00
Block right_table_keys;
2019-09-12 14:09:05 +00:00
Block right_columns_to_add;
2019-09-09 19:43:37 +00:00
BlocksList right_blocks;
Blocks min_max_right_blocks;
std::unique_ptr<Cache> cached_right_blocks;
std::vector<std::shared_ptr<Block>> loaded_right_blocks;
std::unique_ptr<MiniLSM> lsm;
MiniLSM::SortedFiles flushed_right_blocks;
2019-09-12 18:06:25 +00:00
Block totals;
2019-09-10 14:51:28 +00:00
size_t right_blocks_row_count = 0;
size_t right_blocks_bytes = 0;
bool is_in_memory = true;
const bool nullable_right_side;
const bool is_all;
const bool is_inner;
const bool is_left;
2019-09-23 19:36:47 +00:00
const bool skip_not_intersected;
const size_t max_rows_in_right_block;
2019-09-09 19:43:37 +00:00
void changeLeftColumns(Block & block, MutableColumns && columns);
void addRightColumns(Block & block, MutableColumns && columns);
2019-09-13 16:17:37 +00:00
2019-09-12 18:06:25 +00:00
void mergeRightBlocks();
template <bool in_memory>
size_t rightBlocksCount();
template <bool in_memory>
void joinSortedBlock(Block & block);
template <bool in_memory>
std::shared_ptr<Block> loadRightBlock(size_t pos);
void leftJoin(MergeJoinCursor & left_cursor, const Block & left_block, const Block & right_block,
MutableColumns & left_columns, MutableColumns & right_columns, size_t & left_key_tail);
2019-09-13 17:23:32 +00:00
void innerJoin(MergeJoinCursor & left_cursor, const Block & left_block, const Block & right_block,
MutableColumns & left_columns, MutableColumns & right_columns, size_t & left_key_tail);
bool saveRightBlock(Block && block);
void flushRightBlocks();
void mergeInMemoryRightBlocks();
void mergeFlushedRightBlocks();
void clearRightBlocksList()
{
right_blocks.clear();
right_blocks_row_count = 0;
right_blocks_bytes = 0;
}
void countBlockSize(const Block & block)
{
right_blocks_row_count += block.rows();
right_blocks_bytes += block.bytes();
}
2019-09-09 19:43:37 +00:00
};
}