提交 20714502 编写于 作者: C chertus

Join.h refactoring: extract joinDispatch into own file

上级 106a2d0c
#include <array>
#include <common/constexpr_helpers.h>
#include <common/logger_useful.h>
#include <Columns/ColumnConst.h>
......@@ -10,6 +8,7 @@
#include <DataTypes/DataTypeNullable.h>
#include <Interpreters/Join.h>
#include <Interpreters/joinDispatch.h>
#include <Interpreters/NullableUtils.h>
#include <DataStreams/IBlockInputStream.h>
......@@ -32,7 +31,6 @@ namespace ErrorCodes
extern const int ILLEGAL_COLUMN;
}
static std::unordered_map<String, DataTypePtr> requiredRightKeys(const Names & key_names, const NamesAndTypesList & columns_added_by_join)
{
NameSet right_keys;
......@@ -226,8 +224,8 @@ void Join::init(Type type_)
if (kind == ASTTableJoin::Kind::Cross)
return;
dispatch(MapInitTag());
dispatch([&](auto, auto, auto & map) { map.create(type); });
joinDispatchInit(kind, strictness, maps);
joinDispatch(kind, strictness, maps, [&](auto, auto, auto & map) { map.create(type); });
}
size_t Join::getTotalRowCount() const
......@@ -241,7 +239,7 @@ size_t Join::getTotalRowCount() const
}
else
{
dispatch([&](auto, auto, auto & map) { res += map.getTotalRowCount(type); });
joinDispatch(kind, strictness, maps, [&](auto, auto, auto & map) { res += map.getTotalRowCount(type); });
}
return res;
......@@ -258,7 +256,7 @@ size_t Join::getTotalByteCount() const
}
else
{
dispatch([&](auto, auto, auto & map) { res += map.getTotalByteCountImpl(type); });
joinDispatch(kind, strictness, maps, [&](auto, auto, auto & map) { res += map.getTotalByteCountImpl(type); });
res += pool.size();
}
......@@ -561,7 +559,7 @@ bool Join::insertFromBlock(const Block & block)
if (kind != ASTTableJoin::Kind::Cross)
{
dispatch([&](auto, auto strictness_, auto & map)
joinDispatch(kind, strictness, maps, [&](auto, auto strictness_, auto & map)
{
insertFromBlockImpl<strictness_>(*this, type, map, rows, key_columns, key_sizes, stored_block, null_map, pool);
});
......@@ -1082,7 +1080,7 @@ void Join::joinBlock(Block & block, const Names & key_names_left, const NamesAnd
checkTypesOfKeys(block, key_names_left, sample_block_with_keys);
if (dispatch([&](auto kind_, auto strictness_, auto & map)
if (joinDispatch(kind, strictness, maps, [&](auto kind_, auto strictness_, auto & map)
{
joinBlockImpl<kind_, strictness_>(block, key_names_left, columns_added_by_join, sample_block_with_columns_to_add, map);
}))
......@@ -1239,9 +1237,8 @@ protected:
return Block();
Block block;
if (parent.dispatch([&](auto, auto strictness, auto & map) { block = createBlock<strictness>(map); }))
;
else
if (!joinDispatch(parent.kind, parent.strictness, parent.maps,
[&](auto, auto strictness, auto & map) { block = createBlock<strictness>(map); }))
throw Exception("Logical error: unknown JOIN strictness (must be on of: ANY, ALL, ASOF)", ErrorCodes::LOGICAL_ERROR);
return block;
}
......
#pragma once
#include <variant>
#include <optional>
#include <shared_mutex>
......@@ -19,8 +20,6 @@
#include <DataStreams/SizeLimits.h>
#include <DataStreams/IBlockStream_fwd.h>
#include <variant>
#include <common/constexpr_helpers.h>
namespace DB
......@@ -271,80 +270,7 @@ public:
using MapsAllFull = MapsTemplate<JoinStuff::MappedAllFull>;
using MapsAsof = MapsTemplate<JoinStuff::MappedAsof>;
template <ASTTableJoin::Kind KIND>
struct KindTrait
{
// Affects the Adder trait so that when the right part is empty, adding a default value on the left
static constexpr bool fill_left = static_in_v<KIND, ASTTableJoin::Kind::Left, ASTTableJoin::Kind::Full>;
// Affects the Map trait so that a `used` flag is attached to map slots in order to
// generate default values on the right when the left part is empty
static constexpr bool fill_right = static_in_v<KIND, ASTTableJoin::Kind::Right, ASTTableJoin::Kind::Full>;
};
template <bool fill_right, typename ASTTableJoin::Strictness>
struct MapGetterImpl;
template <ASTTableJoin::Kind kind, ASTTableJoin::Strictness strictness>
using Map = typename MapGetterImpl<KindTrait<kind>::fill_right, strictness>::Map;
static constexpr std::array<ASTTableJoin::Strictness, 3> STRICTNESSES
= {ASTTableJoin::Strictness::Any, ASTTableJoin::Strictness::All, ASTTableJoin::Strictness::Asof};
static constexpr std::array<ASTTableJoin::Kind, 4> KINDS
= {ASTTableJoin::Kind::Left, ASTTableJoin::Kind::Inner, ASTTableJoin::Kind::Full, ASTTableJoin::Kind::Right};
struct MapInitTag {};
template <typename Func>
bool dispatch(Func && func)
{
if (any_take_last_row)
{
return static_for<0, KINDS.size()>([&](auto i)
{
if (kind == KINDS[i] && strictness == ASTTableJoin::Strictness::Any)
{
if constexpr (std::is_same_v<Func, MapInitTag>)
maps = Map<KINDS[i], ASTTableJoin::Strictness::Any>();
else
func(
std::integral_constant<ASTTableJoin::Kind, KINDS[i]>(),
std::integral_constant<ASTTableJoin::Strictness, ASTTableJoin::Strictness::Any>(),
std::get<Map<KINDS[i], ASTTableJoin::Strictness::Any>>(maps));
return true;
}
return false;
});
}
else
{
return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij)
{
// NOTE: Avoid using nested static loop as GCC and CLANG have bugs in different ways
// See https://stackoverflow.com/questions/44386415/gcc-and-clang-disagree-about-c17-constexpr-lambda-captures
constexpr auto i = ij / STRICTNESSES.size();
constexpr auto j = ij % STRICTNESSES.size();
if (kind == KINDS[i] && strictness == STRICTNESSES[j])
{
if constexpr (std::is_same_v<Func, MapInitTag>)
maps = Map<KINDS[i], STRICTNESSES[j]>();
else
func(
std::integral_constant<ASTTableJoin::Kind, KINDS[i]>(),
std::integral_constant<ASTTableJoin::Strictness, STRICTNESSES[j]>(),
std::get<Map<KINDS[i], STRICTNESSES[j]>>(maps));
return true;
}
return false;
});
}
}
template <typename Func>
bool dispatch(Func && func) const
{
return const_cast<Join &>(*this).dispatch(std::forward<Func>(func));
}
using MapsVariant = std::variant<MapsAny, MapsAll, MapsAnyFull, MapsAllFull, MapsAsof>;
private:
friend class NonJoinedBlockInputStream;
......@@ -366,12 +292,11 @@ private:
*/
BlocksList blocks;
std::variant<MapsAny, MapsAll, MapsAnyFull, MapsAllFull, MapsAsof> maps;
MapsVariant maps;
/// Additional data - strings for string keys and continuation elements of single-linked lists of references to rows.
Arena pool;
private:
Type type = Type::EMPTY;
std::optional<AsofRowRefs::Type> asof_type;
......@@ -428,34 +353,4 @@ private:
using JoinPtr = std::shared_ptr<Join>;
using Joins = std::vector<JoinPtr>;
template <>
struct Join::MapGetterImpl<false, ASTTableJoin::Strictness::Any>
{
using Map = MapsAny;
};
template <>
struct Join::MapGetterImpl<true, ASTTableJoin::Strictness::Any>
{
using Map = MapsAnyFull;
};
template <>
struct Join::MapGetterImpl<false, ASTTableJoin::Strictness::All>
{
using Map = MapsAll;
};
template <>
struct Join::MapGetterImpl<true, ASTTableJoin::Strictness::All>
{
using Map = MapsAllFull;
};
template <bool fill_right>
struct Join::MapGetterImpl<fill_right, ASTTableJoin::Strictness::Asof>
{
using Map = MapsAsof;
};
}
#pragma once
#include <array>
#include <common/constexpr_helpers.h>
#include <Interpreters/Join.h>
namespace DB
{
template <bool fill_right, typename ASTTableJoin::Strictness>
struct MapGetterImpl;
template <>
struct MapGetterImpl<false, ASTTableJoin::Strictness::Any>
{
using Map = Join::MapsAny;
};
template <>
struct MapGetterImpl<true, ASTTableJoin::Strictness::Any>
{
using Map = Join::MapsAnyFull;
};
template <>
struct MapGetterImpl<false, ASTTableJoin::Strictness::All>
{
using Map = Join::MapsAll;
};
template <>
struct MapGetterImpl<true, ASTTableJoin::Strictness::All>
{
using Map = Join::MapsAllFull;
};
template <bool fill_right>
struct MapGetterImpl<fill_right, ASTTableJoin::Strictness::Asof>
{
using Map = Join::MapsAsof;
};
template <ASTTableJoin::Kind KIND>
struct KindTrait
{
// Affects the Adder trait so that when the right part is empty, adding a default value on the left
static constexpr bool fill_left = static_in_v<KIND, ASTTableJoin::Kind::Left, ASTTableJoin::Kind::Full>;
// Affects the Map trait so that a `used` flag is attached to map slots in order to
// generate default values on the right when the left part is empty
static constexpr bool fill_right = static_in_v<KIND, ASTTableJoin::Kind::Right, ASTTableJoin::Kind::Full>;
};
template <ASTTableJoin::Kind kind, ASTTableJoin::Strictness strictness>
using Map = typename MapGetterImpl<KindTrait<kind>::fill_right, strictness>::Map;
static constexpr std::array<ASTTableJoin::Strictness, 3> STRICTNESSES = {
ASTTableJoin::Strictness::Any,
ASTTableJoin::Strictness::All,
ASTTableJoin::Strictness::Asof
};
static constexpr std::array<ASTTableJoin::Kind, 4> KINDS = {
ASTTableJoin::Kind::Left,
ASTTableJoin::Kind::Inner,
ASTTableJoin::Kind::Full,
ASTTableJoin::Kind::Right
};
/// Init specified join map
inline bool joinDispatchInit(ASTTableJoin::Kind kind, ASTTableJoin::Strictness strictness, Join::MapsVariant & maps)
{
return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij)
{
constexpr auto i = ij / STRICTNESSES.size();
constexpr auto j = ij % STRICTNESSES.size();
if (kind == KINDS[i] && strictness == STRICTNESSES[j])
{
maps = Map<KINDS[i], STRICTNESSES[j]>();
return true;
}
return false;
});
}
/// Call function on specified join map
template <typename MapsVariant, typename Func>
inline bool joinDispatch(ASTTableJoin::Kind kind, ASTTableJoin::Strictness strictness, MapsVariant & maps, Func && func)
{
return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij)
{
// NOTE: Avoid using nested static loop as GCC and CLANG have bugs in different ways
// See https://stackoverflow.com/questions/44386415/gcc-and-clang-disagree-about-c17-constexpr-lambda-captures
constexpr auto i = ij / STRICTNESSES.size();
constexpr auto j = ij % STRICTNESSES.size();
if (kind == KINDS[i] && strictness == STRICTNESSES[j])
{
func(
std::integral_constant<ASTTableJoin::Kind, KINDS[i]>(),
std::integral_constant<ASTTableJoin::Strictness, STRICTNESSES[j]>(),
std::get<Map<KINDS[i], STRICTNESSES[j]>>(maps));
return true;
}
return false;
});
}
}
......@@ -7,6 +7,7 @@
#include <Core/ColumnNumbers.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataTypes/NestedUtils.h>
#include <Interpreters/joinDispatch.h>
#include <Poco/String.h> /// toLower
#include <Poco/File.h>
......@@ -24,7 +25,6 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
StorageJoin::StorageJoin(
const String & path_,
const String & name_,
......@@ -230,9 +230,8 @@ protected:
return Block();
Block block;
if (parent.dispatch([&](auto, auto strictness, auto & map) { block = createBlock<strictness>(map); }))
;
else
if (!joinDispatch(parent.kind, parent.strictness, parent.maps,
[&](auto, auto strictness, auto & map) { block = createBlock<strictness>(map); }))
throw Exception("Logical error: unknown JOIN strictness (must be ANY or ALL)", ErrorCodes::LOGICAL_ERROR);
return block;
}
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册