#pragma once #include #include #include #include #include namespace DB { namespace ErrorCodes { extern const int TYPE_MISMATCH; extern const int NOT_IMPLEMENTED; } /// Dummy class, actual joining is done by MergeTransform class FullSortingMergeJoin : public IJoin { public: explicit FullSortingMergeJoin(std::shared_ptr table_join_, const Block & right_sample_block_) : table_join(table_join_) , right_sample_block(right_sample_block_) { LOG_TRACE(&Poco::Logger::get("FullSortingMergeJoin"), "Will use full sorting merge join"); } const TableJoin & getTableJoin() const override { return *table_join; } bool addJoinedBlock(const Block & /* block */, bool /* check_limits */) override { __builtin_unreachable(); } void checkTypesOfKeys(const Block & left_block) const override { if (table_join->getClauses().size() != 1) throw Exception("FullSortingMergeJoin supports only one join key", ErrorCodes::NOT_IMPLEMENTED); /// Key column can change nullability and it's not handled on type conversion stage, so algorithm should be aware of it 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; bool type_equals = table_join->hasUsing() ? left_type->equals(*right_type) : removeNullable(left_type)->equals(*removeNullable(right_type)); if (!type_equals) { throw DB::Exception( 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()); } } } /// Used just to get result header void joinBlock(Block & block, std::shared_ptr & /* not_processed */) override { for (const auto & col : right_sample_block) block.insert(col); block = materializeBlock(block).cloneEmpty(); } void setTotals(const Block & block) override { totals = block; } const Block & getTotals() const override { return totals; } size_t getTotalRowCount() const override { __builtin_unreachable(); } size_t getTotalByteCount() const override { __builtin_unreachable(); } bool alwaysReturnsEmptySet() const override { __builtin_unreachable(); } std::shared_ptr 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: std::shared_ptr table_join; Block right_sample_block; Block totals; }; }