mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-20 14:42:02 +00:00
get/setTotas to IJoin
This commit is contained in:
parent
b3c581a1f9
commit
87794f9b5b
@ -45,17 +45,6 @@ public:
|
|||||||
/// Could be called from different threads in parallel.
|
/// Could be called from different threads in parallel.
|
||||||
virtual void joinBlock(Block & block, std::shared_ptr<ExtraBlock> &) override;
|
virtual void joinBlock(Block & block, std::shared_ptr<ExtraBlock> &) override;
|
||||||
|
|
||||||
virtual void setTotals(const Block &) override
|
|
||||||
{
|
|
||||||
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "not implemented");
|
|
||||||
}
|
|
||||||
|
|
||||||
virtual const Block & getTotals() const override
|
|
||||||
{
|
|
||||||
static const Block totals;
|
|
||||||
return totals;
|
|
||||||
}
|
|
||||||
|
|
||||||
virtual size_t getTotalRowCount() const override
|
virtual size_t getTotalRowCount() const override
|
||||||
{
|
{
|
||||||
return 0;
|
return 0;
|
||||||
|
@ -170,11 +170,6 @@ public:
|
|||||||
/// Used by joinGet function that turns StorageJoin into a dictionary.
|
/// Used by joinGet function that turns StorageJoin into a dictionary.
|
||||||
ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const;
|
ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const;
|
||||||
|
|
||||||
/** Keep "totals" (separate part of dataset, see WITH TOTALS) to use later.
|
|
||||||
*/
|
|
||||||
void setTotals(const Block & block) override { totals = block; }
|
|
||||||
const Block & getTotals() const override { return totals; }
|
|
||||||
|
|
||||||
bool isFilled() const override { return from_storage_join || data->type == Type::DICT; }
|
bool isFilled() const override { return from_storage_join || data->type == Type::DICT; }
|
||||||
|
|
||||||
/** For RIGHT and FULL JOINs.
|
/** For RIGHT and FULL JOINs.
|
||||||
@ -393,8 +388,6 @@ private:
|
|||||||
|
|
||||||
Poco::Logger * log;
|
Poco::Logger * log;
|
||||||
|
|
||||||
Block totals;
|
|
||||||
|
|
||||||
/// Should be set via setLock to protect hash table from modification from StorageJoin
|
/// Should be set via setLock to protect hash table from modification from StorageJoin
|
||||||
/// If set HashJoin instance is not available for modification (addJoinedBlock)
|
/// If set HashJoin instance is not available for modification (addJoinedBlock)
|
||||||
TableLockHolder storage_join_lock = nullptr;
|
TableLockHolder storage_join_lock = nullptr;
|
||||||
|
@ -4,12 +4,12 @@
|
|||||||
#include <vector>
|
#include <vector>
|
||||||
|
|
||||||
#include <Core/Names.h>
|
#include <Core/Names.h>
|
||||||
|
#include <Core/Block.h>
|
||||||
#include <Columns/IColumn.h>
|
#include <Columns/IColumn.h>
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
class Block;
|
|
||||||
struct ExtraBlock;
|
struct ExtraBlock;
|
||||||
using ExtraBlockPtr = std::shared_ptr<ExtraBlock>;
|
using ExtraBlockPtr = std::shared_ptr<ExtraBlock>;
|
||||||
|
|
||||||
@ -33,9 +33,11 @@ public:
|
|||||||
/// Could be called from different threads in parallel.
|
/// Could be called from different threads in parallel.
|
||||||
virtual void joinBlock(Block & block, std::shared_ptr<ExtraBlock> & not_processed) = 0;
|
virtual void joinBlock(Block & block, std::shared_ptr<ExtraBlock> & not_processed) = 0;
|
||||||
|
|
||||||
/// Set/Get totals for right table
|
/** Set/Get totals for right table
|
||||||
virtual void setTotals(const Block & block) = 0;
|
* Keep "totals" (separate part of dataset, see WITH TOTALS) to use later.
|
||||||
virtual const Block & getTotals() const = 0;
|
*/
|
||||||
|
virtual void setTotals(const Block & block) { totals = block; }
|
||||||
|
virtual const Block & getTotals() const { return totals; }
|
||||||
|
|
||||||
virtual size_t getTotalRowCount() const = 0;
|
virtual size_t getTotalRowCount() const = 0;
|
||||||
virtual size_t getTotalByteCount() const = 0;
|
virtual size_t getTotalByteCount() const = 0;
|
||||||
@ -50,6 +52,9 @@ public:
|
|||||||
|
|
||||||
virtual std::shared_ptr<NotJoinedBlocks>
|
virtual std::shared_ptr<NotJoinedBlocks>
|
||||||
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const = 0;
|
getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const = 0;
|
||||||
|
|
||||||
|
private:
|
||||||
|
Block totals;
|
||||||
};
|
};
|
||||||
|
|
||||||
using JoinPtr = std::shared_ptr<IJoin>;
|
using JoinPtr = std::shared_ptr<IJoin>;
|
||||||
|
@ -563,7 +563,7 @@ MergeJoin::MergeJoin(std::shared_ptr<TableJoin> table_join_, const Block & right
|
|||||||
/// Has to be called even if totals are empty
|
/// Has to be called even if totals are empty
|
||||||
void MergeJoin::setTotals(const Block & totals_block)
|
void MergeJoin::setTotals(const Block & totals_block)
|
||||||
{
|
{
|
||||||
totals = totals_block;
|
IJoin::setTotals(totals_block);
|
||||||
mergeRightBlocks();
|
mergeRightBlocks();
|
||||||
|
|
||||||
if (is_right || is_full)
|
if (is_right || is_full)
|
||||||
|
@ -29,7 +29,6 @@ public:
|
|||||||
void joinBlock(Block &, ExtraBlockPtr & not_processed) override;
|
void joinBlock(Block &, ExtraBlockPtr & not_processed) override;
|
||||||
|
|
||||||
void setTotals(const Block &) override;
|
void setTotals(const Block &) override;
|
||||||
const Block & getTotals() const override { return totals; }
|
|
||||||
|
|
||||||
size_t getTotalRowCount() const override { return right_blocks.row_count; }
|
size_t getTotalRowCount() const override { return right_blocks.row_count; }
|
||||||
size_t getTotalByteCount() const override { return right_blocks.bytes; }
|
size_t getTotalByteCount() const override { return right_blocks.bytes; }
|
||||||
@ -100,7 +99,6 @@ private:
|
|||||||
std::unique_ptr<SortedBlocksWriter> disk_writer;
|
std::unique_ptr<SortedBlocksWriter> disk_writer;
|
||||||
/// Set of files with sorted blocks
|
/// Set of files with sorted blocks
|
||||||
SortedBlocksWriter::SortedFiles flushed_right_blocks;
|
SortedBlocksWriter::SortedFiles flushed_right_blocks;
|
||||||
Block totals;
|
|
||||||
std::atomic<bool> is_in_memory{true};
|
std::atomic<bool> is_in_memory{true};
|
||||||
const bool is_any_join;
|
const bool is_any_join;
|
||||||
const bool is_all_join;
|
const bool is_all_join;
|
||||||
|
Loading…
Reference in New Issue
Block a user