#pragma once #include #include #include /** Used in implementation of Join to process different data structures. */ namespace DB { /// HashJoin::MapsOne is more efficient, it only store one row for each key in the map. It is recommended to use it whenever possible. /// When only need to match only one row from right table, use HashJoin::MapsOne. For example, LEFT ANY/SEMI/ANTI. /// /// HashJoin::MapsAll will store all rows for each key in the map. It is used when need to match multiple rows from right table. /// For example, LEFT ALL, INNER ALL, RIGHT ALL/ANY. /// /// prefer_use_maps_all is true when there is mixed inequal condition in the join condition. For example, `t1.a = t2.a AND t1.b > t2.b`. /// In this case, we need to use HashJoin::MapsAll to store all rows for each key in the map. We will select all matched rows from the map /// and filter them by `t1.b > t2.b`. /// /// flagged indicates whether we need to store flags for each row whether it has been used in the join. See JoinUsedFlags.h. template struct MapGetter; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; /// Only SEMI LEFT and SEMI RIGHT are valid. INNER and FULL are here for templates instantiation. template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; /// Only ANTI LEFT and ANTI RIGHT are valid. INNER and FULL are here for templates instantiation. template <> struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; template <> struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; }; template struct MapGetter { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; }; template struct MapGetter { using Map = HashJoin::MapsAsof; static constexpr bool flagged = false; }; static constexpr std::array STRICTNESSES = { JoinStrictness::RightAny, JoinStrictness::Any, JoinStrictness::All, JoinStrictness::Asof, JoinStrictness::Semi, JoinStrictness::Anti, }; static constexpr std::array KINDS = { JoinKind::Left, JoinKind::Inner, JoinKind::Full, JoinKind::Right }; /// Init specified join map inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin::MapsVariant & maps, bool prefer_use_maps_all = false) { 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]) { if (prefer_use_maps_all) maps = typename MapGetter::Map(); else maps = typename MapGetter::Map(); return true; } return false; }); } /// Call function on specified join map template inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant & maps, bool prefer_use_maps_all, 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]) { if (prefer_use_maps_all) func( std::integral_constant(), std::integral_constant(), std::get::Map>(maps)); else func( std::integral_constant(), std::integral_constant(), std::get::Map>(maps)); return true; } return false; }); } /// Call function on specified join map template inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, std::vector & mapsv, bool prefer_use_maps_all, 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]) { if (prefer_use_maps_all) { using MapType = typename MapGetter::Map; std::vector v; v.reserve(mapsv.size()); for (const auto & el : mapsv) v.push_back(&std::get(*el)); func( std::integral_constant(), std::integral_constant(), v /*std::get::Map>(maps)*/); } else { using MapType = typename MapGetter::Map; std::vector v; v.reserve(mapsv.size()); for (const auto & el : mapsv) v.push_back(&std::get(*el)); func( std::integral_constant(), std::integral_constant(), v /*std::get::Map>(maps)*/); } return true; } return false; }); } }