ClickHouse/src/Interpreters/IJoin.h

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

130 lines
3.7 KiB
C++
Raw Normal View History

2019-09-09 19:43:37 +00:00
#pragma once
#include <memory>
2019-09-10 18:39:10 +00:00
#include <vector>
#include <Core/Names.h>
2022-03-21 13:12:17 +00:00
#include <Core/Block.h>
#include <Columns/IColumn.h>
2022-03-30 10:07:09 +00:00
#include <Common/Exception.h>
2019-09-09 19:43:37 +00:00
namespace DB
{
2020-01-15 20:33:29 +00:00
struct ExtraBlock;
using ExtraBlockPtr = std::shared_ptr<ExtraBlock>;
2019-09-09 19:43:37 +00:00
class TableJoin;
class NotJoinedBlocks;
2022-10-04 08:20:13 +00:00
class IBlocksStream;
2022-10-18 11:43:01 +00:00
using IBlocksStreamPtr = std::shared_ptr<IBlocksStream>;
2022-10-04 08:20:13 +00:00
class IJoin;
using JoinPtr = std::shared_ptr<IJoin>;
2022-03-30 10:07:09 +00:00
enum class JoinPipelineType
{
/*
* Right stream processed first, then when join data structures are ready, the left stream is processed using it.
2022-03-30 10:07:09 +00:00
* The pipeline is not sorted.
*/
2022-04-01 18:16:38 +00:00
FillRightFirst,
2022-03-30 10:07:09 +00:00
/*
* Only the left stream is processed. Right is already filled.
2022-03-30 10:07:09 +00:00
*/
2022-04-01 18:16:38 +00:00
FilledRight,
2022-03-30 10:07:09 +00:00
/*
* The pipeline is created from the left and right streams processed with merging transform.
* Left and right streams have the same priority and are processed simultaneously.
* The pipelines are sorted.
2022-03-30 10:07:09 +00:00
*/
YShaped,
};
2019-09-09 19:43:37 +00:00
class IJoin
{
public:
virtual ~IJoin() = default;
virtual const TableJoin & getTableJoin() const = 0;
2019-09-09 19:43:37 +00:00
/// Add block of data from right hand of JOIN.
/// @returns false, if some limit was exceeded and you should not insert more data.
virtual bool addJoinedBlock(const Block & block, bool check_limits = true) = 0; /// NOLINT
2019-09-09 19:43:37 +00:00
2022-11-29 11:46:11 +00:00
/* Some initialization may be required before joinBlock() call.
* It's better to done in in constructor, but left block exact structure is not known at that moment.
* TODO: pass correct left block sample to the constructor.
*/
virtual void initialize(const Block & /* left_sample_block */) {}
virtual void checkTypesOfKeys(const Block & block) const = 0;
2019-09-09 19:43:37 +00:00
/// Join the block with data from left hand of JOIN to the right hand data (that was previously built by calls to addJoinedBlock).
/// Could be called from different threads in parallel.
2020-01-15 20:33:29 +00:00
virtual void joinBlock(Block & block, std::shared_ptr<ExtraBlock> & not_processed) = 0;
2019-09-09 19:43:37 +00:00
2022-03-21 13:12:17 +00:00
/** Set/Get totals for right table
* Keep "totals" (separate part of dataset, see WITH TOTALS) to use later.
*/
virtual void setTotals(const Block & block) { totals = block; }
virtual const Block & getTotals() const { return totals; }
2019-09-09 19:43:37 +00:00
2022-07-04 17:10:34 +00:00
/// Number of rows/bytes stored in memory
2019-09-10 14:51:28 +00:00
virtual size_t getTotalRowCount() const = 0;
virtual size_t getTotalByteCount() const = 0;
2022-07-04 17:10:34 +00:00
/// Returns true if no data to join with.
virtual bool alwaysReturnsEmptySet() const = 0;
2021-04-29 09:08:49 +00:00
/// StorageJoin/Dictionary is already filled. No need to call addJoinedBlock.
/// Different query plan is used for such joins.
2022-04-01 18:16:38 +00:00
virtual bool isFilled() const { return pipelineType() == JoinPipelineType::FilledRight; }
virtual JoinPipelineType pipelineType() const { return JoinPipelineType::FillRightFirst; }
2021-04-28 17:32:12 +00:00
// That can run FillingRightJoinSideTransform parallelly
virtual bool supportParallelJoin() const { return false; }
2022-10-27 15:25:50 +00:00
virtual bool supportTotals() const { return true; }
2022-06-16 12:09:23 +00:00
/// Peek next stream of delayed joined blocks.
2022-10-18 11:43:01 +00:00
virtual IBlocksStreamPtr getDelayedBlocks() { return nullptr; }
virtual bool hasDelayedBlocks() const { return false; }
2022-10-04 08:20:13 +00:00
2022-10-18 11:43:01 +00:00
virtual IBlocksStreamPtr
2022-10-04 08:20:13 +00:00
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const = 0;
2022-03-21 13:12:17 +00:00
private:
Block totals;
2019-09-09 19:43:37 +00:00
};
2022-10-04 08:20:13 +00:00
class IBlocksStream
{
public:
/// Returns empty block on EOF
2022-10-05 12:40:32 +00:00
Block next()
{
if (finished)
return {};
if (Block res = nextImpl())
return res;
finished = true;
return {};
}
2022-03-30 10:07:09 +00:00
2022-10-04 08:20:13 +00:00
virtual ~IBlocksStream() = default;
2022-10-05 12:40:32 +00:00
bool isFinished() const { return finished; }
protected:
virtual Block nextImpl() = 0;
2022-10-27 14:23:55 +00:00
std::atomic_bool finished{false};
2022-10-05 12:40:32 +00:00
2022-10-04 08:20:13 +00:00
};
2019-09-09 19:43:37 +00:00
2019-09-11 18:03:21 +00:00
}