ClickHouse/src/Interpreters/FullSortingMergeJoin.h

92 lines
3.4 KiB
C++
Raw Normal View History

2022-04-08 10:04:52 +00:00
#pragma once
2022-03-30 10:07:09 +00:00
#include <Interpreters/IJoin.h>
#include <Interpreters/TableJoin.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeLowCardinality.h>
2022-03-30 10:07:09 +00:00
#include <Poco/Logger.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TYPE_MISMATCH;
extern const int NOT_IMPLEMENTED;
}
2022-03-30 10:07:09 +00:00
/// Dummy class, actual joining is done by MergeTransform
class FullSortingMergeJoin : public IJoin
{
public:
2022-04-05 10:12:42 +00:00
explicit FullSortingMergeJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_sample_block_)
2022-03-30 10:07:09 +00:00
: table_join(table_join_)
, right_sample_block(right_sample_block_)
2022-03-30 10:07:09 +00:00
{
LOG_TRACE(&Poco::Logger::get("FullSortingMergeJoin"), "Will use full sorting merge join");
}
2022-04-05 10:12:42 +00:00
const TableJoin & getTableJoin() const override { return *table_join; }
2022-03-30 10:07:09 +00:00
bool addJoinedBlock(const Block & /* block */, bool /* check_limits */) override { __builtin_unreachable(); }
void checkTypesOfKeys(const Block & left_block) const override
2022-03-30 10:07:09 +00:00
{
if (table_join->getClauses().size() != 1)
throw Exception("FullSortingMergeJoin supports only one join key", ErrorCodes::NOT_IMPLEMENTED);
2022-05-03 12:05:22 +00:00
/// Key column can change nullability and it's not handled on type conversion stage, so algorithm should be aware of it
2022-05-02 16:27:43 +00:00
if (table_join->hasUsing() && table_join->joinUseNulls())
throw DB::Exception(ErrorCodes::NOT_IMPLEMENTED, "FullSortingMergeJoin doesn't support USING with join_use_nulls");
const auto & onexpr = table_join->getOnlyClause();
for (size_t i = 0; i < onexpr.key_names_left.size(); ++i)
{
DataTypePtr left_type = left_block.getByName(onexpr.key_names_left[i]).type;
DataTypePtr right_type = right_sample_block.getByName(onexpr.key_names_right[i]).type;
2022-05-02 16:27:43 +00:00
bool type_equals
= table_join->hasUsing() ? left_type->equals(*right_type) : removeNullable(left_type)->equals(*removeNullable(right_type));
if (!type_equals)
{
throw DB::Exception(
2022-05-02 15:54:42 +00:00
ErrorCodes::TYPE_MISMATCH,
"Type mismatch of columns to JOIN by: {} :: {} at left, {} :: {} at right",
onexpr.key_names_left[i], left_type->getName(),
onexpr.key_names_right[i], right_type->getName());
}
}
2022-03-30 10:07:09 +00:00
}
/// Used just to get result header
void joinBlock(Block & block, std::shared_ptr<ExtraBlock> & /* not_processed */) override
{
for (const auto & col : right_sample_block)
block.insert(col);
block = materializeBlock(block).cloneEmpty();
2022-03-30 10:07:09 +00:00
}
void setTotals(const Block & block) override { totals = block; }
const Block & getTotals() const override { return totals; }
2022-03-30 10:07:09 +00:00
size_t getTotalRowCount() const override { __builtin_unreachable(); }
size_t getTotalByteCount() const override { __builtin_unreachable(); }
bool alwaysReturnsEmptySet() const override { __builtin_unreachable(); }
std::shared_ptr<NotJoinedBlocks>
getNonJoinedBlocks(const Block & /* left_sample_block */, const Block & /* result_sample_block */, UInt64 /* max_block_size */) const override
{
__builtin_unreachable();
}
virtual JoinPipelineType pipelineType() const override { return JoinPipelineType::YShaped; }
private:
2022-04-05 10:12:42 +00:00
std::shared_ptr<TableJoin> table_join;
Block right_sample_block;
Block totals;
2022-03-30 10:07:09 +00:00
};
}