mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge pull request #64281 from bigo-sg/extend_inequal_join
`Any/Semi/Anti` join support mixed join conditions
This commit is contained in:
commit
e6e06af7bd
@ -186,7 +186,7 @@ Otherwise, you'll get `INVALID_JOIN_ON_EXPRESSION`.
|
||||
|
||||
:::
|
||||
|
||||
Clickhouse currently supports `ALL INNER/LEFT/RIGHT/FULL JOIN` with inequality conditions in addition to equality conditions. The inequality conditions are supported only for `hash` and `grace_hash` join algorithms. The inequality conditions are not supported with `join_use_nulls`.
|
||||
Clickhouse currently supports `ALL/ANY/SEMI/ANTI INNER/LEFT/RIGHT/FULL JOIN` with inequality conditions in addition to equality conditions. The inequality conditions are supported only for `hash` and `grace_hash` join algorithms. The inequality conditions are not supported with `join_use_nulls`.
|
||||
|
||||
**Example**
|
||||
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Interpreters/HashJoin/HashJoinMethods.h>
|
||||
#include <Interpreters/HashJoin/HashJoinMethodsImpl.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -291,12 +291,13 @@ void HashJoin::dataMapInit(MapsVariant & map)
|
||||
{
|
||||
if (kind == JoinKind::Cross)
|
||||
return;
|
||||
joinDispatchInit(kind, strictness, map);
|
||||
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.create(data->type); });
|
||||
auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr;
|
||||
joinDispatchInit(kind, strictness, map, prefer_use_maps_all);
|
||||
joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { map_.create(data->type); });
|
||||
|
||||
if (reserve_num)
|
||||
{
|
||||
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.reserve(data->type, reserve_num); });
|
||||
joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { map_.reserve(data->type, reserve_num); });
|
||||
}
|
||||
|
||||
if (!data)
|
||||
@ -327,9 +328,10 @@ size_t HashJoin::getTotalRowCount() const
|
||||
}
|
||||
else
|
||||
{
|
||||
auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr;
|
||||
for (const auto & map : data->maps)
|
||||
{
|
||||
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { res += map_.getTotalRowCount(data->type); });
|
||||
joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { res += map_.getTotalRowCount(data->type); });
|
||||
}
|
||||
}
|
||||
|
||||
@ -367,9 +369,10 @@ size_t HashJoin::getTotalByteCount() const
|
||||
|
||||
if (data->type != Type::CROSS)
|
||||
{
|
||||
auto prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr;
|
||||
for (const auto & map : data->maps)
|
||||
{
|
||||
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { res += map_.getTotalByteCountImpl(data->type); });
|
||||
joinDispatch(kind, strictness, map, prefer_use_maps_all, [&](auto, auto, auto & map_) { res += map_.getTotalByteCountImpl(data->type); });
|
||||
}
|
||||
}
|
||||
return res;
|
||||
@ -520,6 +523,8 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
|
||||
return true;
|
||||
}
|
||||
|
||||
bool prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr;
|
||||
|
||||
size_t total_rows = 0;
|
||||
size_t total_bytes = 0;
|
||||
{
|
||||
@ -592,7 +597,7 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
|
||||
bool is_inserted = false;
|
||||
if (kind != JoinKind::Cross)
|
||||
{
|
||||
joinDispatch(kind, strictness, data->maps[onexpr_idx], [&](auto kind_, auto strictness_, auto & map)
|
||||
joinDispatch(kind, strictness, data->maps[onexpr_idx], prefer_use_maps_all, [&](auto kind_, auto strictness_, auto & map)
|
||||
{
|
||||
size_t size = HashJoinMethods<kind_, strictness_, std::decay_t<decltype(map)>>::insertFromBlockImpl(
|
||||
*this,
|
||||
@ -608,10 +613,10 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
|
||||
is_inserted);
|
||||
|
||||
if (flag_per_row)
|
||||
used_flags->reinit<kind_, strictness_>(stored_block);
|
||||
used_flags->reinit<kind_, strictness_, std::is_same_v<std::decay_t<decltype(map)>, MapsAll>>(stored_block);
|
||||
else if (is_inserted)
|
||||
/// Number of buckets + 1 value from zero storage
|
||||
used_flags->reinit<kind_, strictness_>(size + 1);
|
||||
used_flags->reinit<kind_, strictness_, std::is_same_v<std::decay_t<decltype(map)>, MapsAll>>(size + 1);
|
||||
});
|
||||
}
|
||||
|
||||
@ -873,7 +878,7 @@ ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block
|
||||
keys.insert(std::move(key));
|
||||
}
|
||||
|
||||
static_assert(!MapGetter<JoinKind::Left, JoinStrictness::Any>::flagged,
|
||||
static_assert(!MapGetter<JoinKind::Left, JoinStrictness::Any, false>::flagged,
|
||||
"joinGet are not protected from hash table changes between block processing");
|
||||
|
||||
std::vector<const MapsOne *> maps_vector;
|
||||
@ -914,16 +919,34 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed)
|
||||
materializeBlockInplace(block);
|
||||
}
|
||||
|
||||
bool prefer_use_maps_all = table_join->getMixedJoinExpression() != nullptr;
|
||||
{
|
||||
std::vector<const std::decay_t<decltype(data->maps[0])> * > maps_vector;
|
||||
for (size_t i = 0; i < table_join->getClauses().size(); ++i)
|
||||
maps_vector.push_back(&data->maps[i]);
|
||||
|
||||
if (joinDispatch(kind, strictness, maps_vector, [&](auto kind_, auto strictness_, auto & maps_vector_)
|
||||
if (joinDispatch(kind, strictness, maps_vector, prefer_use_maps_all, [&](auto kind_, auto strictness_, auto & maps_vector_)
|
||||
{
|
||||
using MapType = typename MapGetter<kind_, strictness_>::Map;
|
||||
Block remaining_block = HashJoinMethods<kind_, strictness_, MapType>::joinBlockImpl(
|
||||
*this, block, sample_block_with_columns_to_add, maps_vector_);
|
||||
Block remaining_block;
|
||||
if constexpr (std::is_same_v<std::decay_t<decltype(maps_vector_)>, std::vector<const MapsAll *>>)
|
||||
{
|
||||
remaining_block = HashJoinMethods<kind_, strictness_, MapsAll>::joinBlockImpl(
|
||||
*this, block, sample_block_with_columns_to_add, maps_vector_);
|
||||
}
|
||||
else if constexpr (std::is_same_v<std::decay_t<decltype(maps_vector_)>, std::vector<const MapsOne *>>)
|
||||
{
|
||||
remaining_block = HashJoinMethods<kind_, strictness_, MapsOne>::joinBlockImpl(
|
||||
*this, block, sample_block_with_columns_to_add, maps_vector_);
|
||||
}
|
||||
else if constexpr (std::is_same_v<std::decay_t<decltype(maps_vector_)>, std::vector<const MapsAsof *>>)
|
||||
{
|
||||
remaining_block = HashJoinMethods<kind_, strictness_, MapsAsof>::joinBlockImpl(
|
||||
*this, block, sample_block_with_columns_to_add, maps_vector_);
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown maps type");
|
||||
}
|
||||
if (remaining_block.rows())
|
||||
not_processed = std::make_shared<ExtraBlock>(ExtraBlock{std::move(remaining_block)});
|
||||
else
|
||||
@ -1023,7 +1046,8 @@ public:
|
||||
rows_added = fillColumnsFromMap(map, columns_right);
|
||||
};
|
||||
|
||||
if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps.front(), fill_callback))
|
||||
bool prefer_use_maps_all = parent.table_join->getMixedJoinExpression() != nullptr;
|
||||
if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps.front(), prefer_use_maps_all, fill_callback))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness '{}' (must be on of: ANY, ALL, ASOF)", parent.strictness);
|
||||
}
|
||||
|
||||
@ -1220,11 +1244,12 @@ void HashJoin::reuseJoinedData(const HashJoin & join)
|
||||
if (flag_per_row)
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "StorageJoin with ORs is not supported");
|
||||
|
||||
bool prefer_use_maps_all = join.table_join->getMixedJoinExpression() != nullptr;
|
||||
for (auto & map : data->maps)
|
||||
{
|
||||
joinDispatch(kind, strictness, map, [this](auto kind_, auto strictness_, auto & map_)
|
||||
joinDispatch(kind, strictness, map, prefer_use_maps_all, [this](auto kind_, auto strictness_, auto & map_)
|
||||
{
|
||||
used_flags->reinit<kind_, strictness_>(map_.getBufferSizeInCells(data->type) + 1);
|
||||
used_flags->reinit<kind_, strictness_, std::is_same_v<std::decay_t<decltype(map_)>, MapsAll>>(map_.getBufferSizeInCells(data->type) + 1);
|
||||
});
|
||||
}
|
||||
}
|
||||
@ -1304,7 +1329,9 @@ void HashJoin::validateAdditionalFilterExpression(ExpressionActionsPtr additiona
|
||||
additional_filter_expression->dumpActions());
|
||||
}
|
||||
|
||||
bool is_supported = (strictness == JoinStrictness::All) && (isInnerOrLeft(kind) || isRightOrFull(kind));
|
||||
bool is_supported = ((strictness == JoinStrictness::All) && (isInnerOrLeft(kind) || isRightOrFull(kind)))
|
||||
|| ((strictness == JoinStrictness::Semi || strictness == JoinStrictness::Any || strictness == JoinStrictness::Anti)
|
||||
&& (isLeft(kind) || isRight(kind))) || (strictness == JoinStrictness::Any && (isInner(kind)));
|
||||
if (!is_supported)
|
||||
{
|
||||
throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION,
|
||||
|
@ -12,15 +12,8 @@
|
||||
#include <Poco/Logger.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNSUPPORTED_JOIN_KEYS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
/// Inserting an element into a hash table of the form `key -> reference to a string`, which will then be used by JOIN.
|
||||
template <typename HashMap, typename KeyGetter>
|
||||
struct Inserter
|
||||
@ -64,7 +57,6 @@ struct Inserter
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// MapsTemplate is one of MapsOne, MapsAll and MapsAsof
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
class HashJoinMethods
|
||||
@ -81,27 +73,7 @@ public:
|
||||
ConstNullMapPtr null_map,
|
||||
UInt8ColumnDataPtr join_mask,
|
||||
Arena & pool,
|
||||
bool & is_inserted)
|
||||
{
|
||||
switch (type)
|
||||
{
|
||||
case HashJoin::Type::EMPTY:
|
||||
[[fallthrough]];
|
||||
case HashJoin::Type::CROSS:
|
||||
/// Do nothing. We will only save block, and it is enough
|
||||
is_inserted = true;
|
||||
return 0;
|
||||
|
||||
#define M(TYPE) \
|
||||
case HashJoin::Type::TYPE: \
|
||||
return insertFromBlockImplTypeCase<typename KeyGetterForType<HashJoin::Type::TYPE, std::remove_reference_t<decltype(*maps.TYPE)>>::Type>(\
|
||||
join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); \
|
||||
break;
|
||||
|
||||
APPLY_FOR_JOIN_VARIANTS(M)
|
||||
#undef M
|
||||
}
|
||||
}
|
||||
bool & is_inserted);
|
||||
|
||||
using MapsTemplateVector = std::vector<const MapsTemplate *>;
|
||||
|
||||
@ -110,280 +82,36 @@ public:
|
||||
Block & block,
|
||||
const Block & block_with_columns_to_add,
|
||||
const MapsTemplateVector & maps_,
|
||||
bool is_join_get = false)
|
||||
{
|
||||
constexpr JoinFeatures<KIND, STRICTNESS> join_features;
|
||||
|
||||
std::vector<JoinOnKeyColumns> join_on_keys;
|
||||
const auto & onexprs = join.table_join->getClauses();
|
||||
for (size_t i = 0; i < onexprs.size(); ++i)
|
||||
{
|
||||
const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right;
|
||||
join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]);
|
||||
}
|
||||
size_t existing_columns = block.columns();
|
||||
|
||||
/** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized.
|
||||
* Because if they are constants, then in the "not joined" rows, they may have different values
|
||||
* - default values, which can differ from the values of these constants.
|
||||
*/
|
||||
if constexpr (join_features.right || join_features.full)
|
||||
{
|
||||
materializeBlockInplace(block);
|
||||
}
|
||||
|
||||
/** For LEFT/INNER JOIN, the saved blocks do not contain keys.
|
||||
* For FULL/RIGHT JOIN, the saved blocks contain keys;
|
||||
* but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped.
|
||||
* For ASOF, the last column is used as the ASOF column
|
||||
*/
|
||||
AddedColumns<!join_features.is_any_join> added_columns(
|
||||
block,
|
||||
block_with_columns_to_add,
|
||||
join.savedBlockSample(),
|
||||
join,
|
||||
std::move(join_on_keys),
|
||||
join.table_join->getMixedJoinExpression(),
|
||||
join_features.is_asof_join,
|
||||
is_join_get);
|
||||
|
||||
bool has_required_right_keys = (join.required_right_keys.columns() != 0);
|
||||
added_columns.need_filter = join_features.need_filter || has_required_right_keys;
|
||||
added_columns.max_joined_block_rows = join.max_joined_block_rows;
|
||||
if (!added_columns.max_joined_block_rows)
|
||||
added_columns.max_joined_block_rows = std::numeric_limits<size_t>::max();
|
||||
else
|
||||
added_columns.reserve(join_features.need_replication);
|
||||
|
||||
size_t num_joined = switchJoinRightColumns(maps_, added_columns, join.data->type, *join.used_flags);
|
||||
/// Do not hold memory for join_on_keys anymore
|
||||
added_columns.join_on_keys.clear();
|
||||
Block remaining_block = sliceBlock(block, num_joined);
|
||||
|
||||
added_columns.buildOutput();
|
||||
for (size_t i = 0; i < added_columns.size(); ++i)
|
||||
block.insert(added_columns.moveColumn(i));
|
||||
|
||||
std::vector<size_t> right_keys_to_replicate [[maybe_unused]];
|
||||
|
||||
if constexpr (join_features.need_filter)
|
||||
{
|
||||
/// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones.
|
||||
for (size_t i = 0; i < existing_columns; ++i)
|
||||
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(added_columns.filter, -1);
|
||||
|
||||
/// Add join key columns from right block if needed using value from left table because of equality
|
||||
for (size_t i = 0; i < join.required_right_keys.columns(); ++i)
|
||||
{
|
||||
const auto & right_key = join.required_right_keys.getByPosition(i);
|
||||
/// asof column is already in block.
|
||||
if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back())
|
||||
continue;
|
||||
|
||||
const auto & left_column = block.getByName(join.required_right_keys_sources[i]);
|
||||
const auto & right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name);
|
||||
auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column);
|
||||
block.insert(std::move(right_col));
|
||||
}
|
||||
}
|
||||
else if (has_required_right_keys)
|
||||
{
|
||||
/// Add join key columns from right block if needed.
|
||||
for (size_t i = 0; i < join.required_right_keys.columns(); ++i)
|
||||
{
|
||||
const auto & right_key = join.required_right_keys.getByPosition(i);
|
||||
auto right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name);
|
||||
/// asof column is already in block.
|
||||
if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back())
|
||||
continue;
|
||||
|
||||
const auto & left_column = block.getByName(join.required_right_keys_sources[i]);
|
||||
auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &added_columns.filter);
|
||||
block.insert(std::move(right_col));
|
||||
|
||||
if constexpr (join_features.need_replication)
|
||||
right_keys_to_replicate.push_back(block.getPositionByName(right_col_name));
|
||||
}
|
||||
}
|
||||
|
||||
if constexpr (join_features.need_replication)
|
||||
{
|
||||
std::unique_ptr<IColumn::Offsets> & offsets_to_replicate = added_columns.offsets_to_replicate;
|
||||
|
||||
/// If ALL ... JOIN - we replicate all the columns except the new ones.
|
||||
for (size_t i = 0; i < existing_columns; ++i)
|
||||
{
|
||||
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate);
|
||||
}
|
||||
|
||||
/// Replicate additional right keys
|
||||
for (size_t pos : right_keys_to_replicate)
|
||||
{
|
||||
block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate);
|
||||
}
|
||||
}
|
||||
|
||||
return remaining_block;
|
||||
}
|
||||
|
||||
bool is_join_get = false);
|
||||
private:
|
||||
template <typename KeyGetter, bool is_asof_join>
|
||||
static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes)
|
||||
{
|
||||
if constexpr (is_asof_join)
|
||||
{
|
||||
auto key_column_copy = key_columns;
|
||||
auto key_size_copy = key_sizes;
|
||||
key_column_copy.pop_back();
|
||||
key_size_copy.pop_back();
|
||||
return KeyGetter(key_column_copy, key_size_copy, nullptr);
|
||||
}
|
||||
else
|
||||
return KeyGetter(key_columns, key_sizes, nullptr);
|
||||
}
|
||||
static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes);
|
||||
|
||||
template <typename KeyGetter, typename HashMap>
|
||||
static size_t NO_INLINE insertFromBlockImplTypeCase(
|
||||
static size_t insertFromBlockImplTypeCase(
|
||||
HashJoin & join, HashMap & map, size_t rows, const ColumnRawPtrs & key_columns,
|
||||
const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted)
|
||||
{
|
||||
[[maybe_unused]] constexpr bool mapped_one = std::is_same_v<typename HashMap::mapped_type, RowRef>;
|
||||
constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof;
|
||||
|
||||
const IColumn * asof_column [[maybe_unused]] = nullptr;
|
||||
if constexpr (is_asof_join)
|
||||
asof_column = key_columns.back();
|
||||
|
||||
auto key_getter = createKeyGetter<KeyGetter, is_asof_join>(key_columns, key_sizes);
|
||||
|
||||
/// For ALL and ASOF join always insert values
|
||||
is_inserted = !mapped_one || is_asof_join;
|
||||
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
if (null_map && (*null_map)[i])
|
||||
{
|
||||
/// nulls are not inserted into hash table,
|
||||
/// keep them for RIGHT and FULL joins
|
||||
is_inserted = true;
|
||||
continue;
|
||||
}
|
||||
|
||||
/// Check condition for right table from ON section
|
||||
if (join_mask && !(*join_mask)[i])
|
||||
continue;
|
||||
|
||||
if constexpr (is_asof_join)
|
||||
Inserter<HashMap, KeyGetter>::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column);
|
||||
else if constexpr (mapped_one)
|
||||
is_inserted |= Inserter<HashMap, KeyGetter>::insertOne(join, map, key_getter, stored_block, i, pool);
|
||||
else
|
||||
Inserter<HashMap, KeyGetter>::insertAll(join, map, key_getter, stored_block, i, pool);
|
||||
}
|
||||
return map.getBufferSizeInCells();
|
||||
}
|
||||
const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted);
|
||||
|
||||
template <typename AddedColumns>
|
||||
static size_t switchJoinRightColumns(
|
||||
const std::vector<const MapsTemplate *> & mapv,
|
||||
AddedColumns & added_columns,
|
||||
HashJoin::Type type,
|
||||
JoinStuff::JoinUsedFlags & used_flags)
|
||||
{
|
||||
constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof;
|
||||
switch (type)
|
||||
{
|
||||
case HashJoin::Type::EMPTY: {
|
||||
if constexpr (!is_asof_join)
|
||||
{
|
||||
using KeyGetter = KeyGetterEmpty<typename MapsTemplate::MappedType>;
|
||||
std::vector<KeyGetter> key_getter_vector;
|
||||
key_getter_vector.emplace_back();
|
||||
|
||||
using MapTypeVal = typename KeyGetter::MappedType;
|
||||
std::vector<const MapTypeVal *> a_map_type_vector;
|
||||
a_map_type_vector.emplace_back();
|
||||
return joinRightColumnsSwitchNullability<KeyGetter>(
|
||||
std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags);
|
||||
}
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys. Type: {}", type);
|
||||
}
|
||||
#define M(TYPE) \
|
||||
case HashJoin::Type::TYPE: \
|
||||
{ \
|
||||
using MapTypeVal = const typename std::remove_reference_t<decltype(MapsTemplate::TYPE)>::element_type; \
|
||||
using KeyGetter = typename KeyGetterForType<HashJoin::Type::TYPE, MapTypeVal>::Type; \
|
||||
std::vector<const MapTypeVal *> a_map_type_vector(mapv.size()); \
|
||||
std::vector<KeyGetter> key_getter_vector; \
|
||||
for (size_t d = 0; d < added_columns.join_on_keys.size(); ++d) \
|
||||
{ \
|
||||
const auto & join_on_key = added_columns.join_on_keys[d]; \
|
||||
a_map_type_vector[d] = mapv[d]->TYPE.get(); \
|
||||
key_getter_vector.push_back(std::move(createKeyGetter<KeyGetter, is_asof_join>(join_on_key.key_columns, join_on_key.key_sizes))); \
|
||||
} \
|
||||
return joinRightColumnsSwitchNullability<KeyGetter>( \
|
||||
std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags); \
|
||||
}
|
||||
APPLY_FOR_JOIN_VARIANTS(M)
|
||||
#undef M
|
||||
|
||||
default:
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", type);
|
||||
}
|
||||
}
|
||||
JoinStuff::JoinUsedFlags & used_flags);
|
||||
|
||||
template <typename KeyGetter, typename Map, typename AddedColumns>
|
||||
static size_t joinRightColumnsSwitchNullability(
|
||||
std::vector<KeyGetter> && key_getter_vector,
|
||||
const std::vector<const Map *> & mapv,
|
||||
AddedColumns & added_columns,
|
||||
JoinStuff::JoinUsedFlags & used_flags)
|
||||
{
|
||||
if (added_columns.need_filter)
|
||||
{
|
||||
return joinRightColumnsSwitchMultipleDisjuncts<KeyGetter, Map, true>(
|
||||
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
|
||||
}
|
||||
else
|
||||
{
|
||||
return joinRightColumnsSwitchMultipleDisjuncts<KeyGetter, Map, false>(
|
||||
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
|
||||
}
|
||||
}
|
||||
JoinStuff::JoinUsedFlags & used_flags);
|
||||
|
||||
template <typename KeyGetter, typename Map, bool need_filter, typename AddedColumns>
|
||||
static size_t joinRightColumnsSwitchMultipleDisjuncts(
|
||||
std::vector<KeyGetter> && key_getter_vector,
|
||||
const std::vector<const Map *> & mapv,
|
||||
AddedColumns & added_columns,
|
||||
JoinStuff::JoinUsedFlags & used_flags)
|
||||
{
|
||||
constexpr JoinFeatures<KIND, STRICTNESS> join_features;
|
||||
if constexpr (join_features.is_all_join)
|
||||
{
|
||||
if (added_columns.additional_filter_expression)
|
||||
{
|
||||
bool mark_per_row_used = join_features.right || join_features.full || mapv.size() > 1;
|
||||
return joinRightColumnsWithAddtitionalFilter<KeyGetter, Map, join_features.need_replication>(
|
||||
std::forward<std::vector<KeyGetter>>(key_getter_vector),
|
||||
mapv,
|
||||
added_columns,
|
||||
used_flags,
|
||||
need_filter,
|
||||
join_features.need_flags,
|
||||
join_features.add_missing,
|
||||
mark_per_row_used);
|
||||
}
|
||||
}
|
||||
|
||||
if (added_columns.additional_filter_expression)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Additional filter expression is not supported for this JOIN");
|
||||
|
||||
return mapv.size() > 1 ? joinRightColumns<KeyGetter, Map, need_filter, true>(
|
||||
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags)
|
||||
: joinRightColumns<KeyGetter, Map, need_filter, false>(
|
||||
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
|
||||
}
|
||||
JoinStuff::JoinUsedFlags & used_flags);
|
||||
|
||||
/// Joins right table columns which indexes are present in right_indexes using specified map.
|
||||
/// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS).
|
||||
@ -392,464 +120,30 @@ private:
|
||||
std::vector<KeyGetter> && key_getter_vector,
|
||||
const std::vector<const Map *> & mapv,
|
||||
AddedColumns & added_columns,
|
||||
JoinStuff::JoinUsedFlags & used_flags)
|
||||
{
|
||||
constexpr JoinFeatures<KIND, STRICTNESS> join_features;
|
||||
|
||||
size_t rows = added_columns.rows_to_add;
|
||||
if constexpr (need_filter)
|
||||
added_columns.filter = IColumn::Filter(rows, 0);
|
||||
|
||||
Arena pool;
|
||||
|
||||
if constexpr (join_features.need_replication)
|
||||
added_columns.offsets_to_replicate = std::make_unique<IColumn::Offsets>(rows);
|
||||
|
||||
IColumn::Offset current_offset = 0;
|
||||
size_t max_joined_block_rows = added_columns.max_joined_block_rows;
|
||||
size_t i = 0;
|
||||
for (; i < rows; ++i)
|
||||
{
|
||||
if constexpr (join_features.need_replication)
|
||||
{
|
||||
if (unlikely(current_offset >= max_joined_block_rows))
|
||||
{
|
||||
added_columns.offsets_to_replicate->resize_assume_reserved(i);
|
||||
added_columns.filter.resize_assume_reserved(i);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
bool right_row_found = false;
|
||||
|
||||
KnownRowsHolder<flag_per_row> known_rows;
|
||||
for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx)
|
||||
{
|
||||
const auto & join_keys = added_columns.join_on_keys[onexpr_idx];
|
||||
if (join_keys.null_map && (*join_keys.null_map)[i])
|
||||
continue;
|
||||
|
||||
bool row_acceptable = !join_keys.isRowFiltered(i);
|
||||
using FindResult = typename KeyGetter::FindResult;
|
||||
auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult();
|
||||
|
||||
if (find_result.isFound())
|
||||
{
|
||||
right_row_found = true;
|
||||
auto & mapped = find_result.getMapped();
|
||||
if constexpr (join_features.is_asof_join)
|
||||
{
|
||||
const IColumn & left_asof_key = added_columns.leftAsofKey();
|
||||
|
||||
auto row_ref = mapped->findAsof(left_asof_key, i);
|
||||
if (row_ref.block)
|
||||
{
|
||||
setUsed<need_filter>(added_columns.filter, i);
|
||||
if constexpr (flag_per_row)
|
||||
used_flags.template setUsed<join_features.need_flags, flag_per_row>(row_ref.block, row_ref.row_num, 0);
|
||||
else
|
||||
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
|
||||
|
||||
added_columns.appendFromBlock(*row_ref.block, row_ref.row_num, join_features.add_missing);
|
||||
}
|
||||
else
|
||||
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, current_offset);
|
||||
}
|
||||
else if constexpr (join_features.is_all_join)
|
||||
{
|
||||
setUsed<need_filter>(added_columns.filter, i);
|
||||
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
|
||||
auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr;
|
||||
addFoundRowAll<Map, join_features.add_missing>(mapped, added_columns, current_offset, known_rows, used_flags_opt);
|
||||
}
|
||||
else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right)
|
||||
{
|
||||
/// Use first appeared left key + it needs left columns replication
|
||||
bool used_once = used_flags.template setUsedOnce<join_features.need_flags, flag_per_row>(find_result);
|
||||
if (used_once)
|
||||
{
|
||||
auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr;
|
||||
setUsed<need_filter>(added_columns.filter, i);
|
||||
addFoundRowAll<Map, join_features.add_missing>(
|
||||
mapped, added_columns, current_offset, known_rows, used_flags_opt);
|
||||
}
|
||||
}
|
||||
else if constexpr (join_features.is_any_join && KIND == JoinKind::Inner)
|
||||
{
|
||||
bool used_once = used_flags.template setUsedOnce<join_features.need_flags, flag_per_row>(find_result);
|
||||
|
||||
/// Use first appeared left key only
|
||||
if (used_once)
|
||||
{
|
||||
setUsed<need_filter>(added_columns.filter, i);
|
||||
added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing);
|
||||
}
|
||||
|
||||
break;
|
||||
}
|
||||
else if constexpr (join_features.is_any_join && join_features.full)
|
||||
{
|
||||
/// TODO
|
||||
}
|
||||
else if constexpr (join_features.is_anti_join)
|
||||
{
|
||||
if constexpr (join_features.right && join_features.need_flags)
|
||||
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
|
||||
}
|
||||
else /// ANY LEFT, SEMI LEFT, old ANY (RightAny)
|
||||
{
|
||||
setUsed<need_filter>(added_columns.filter, i);
|
||||
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
|
||||
added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing);
|
||||
|
||||
if (join_features.is_any_or_semi_join)
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!right_row_found)
|
||||
{
|
||||
if constexpr (join_features.is_anti_join && join_features.left)
|
||||
setUsed<need_filter>(added_columns.filter, i);
|
||||
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, current_offset);
|
||||
}
|
||||
|
||||
if constexpr (join_features.need_replication)
|
||||
{
|
||||
(*added_columns.offsets_to_replicate)[i] = current_offset;
|
||||
}
|
||||
}
|
||||
|
||||
added_columns.applyLazyDefaults();
|
||||
return i;
|
||||
}
|
||||
JoinStuff::JoinUsedFlags & used_flags);
|
||||
|
||||
template <bool need_filter>
|
||||
static void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]])
|
||||
{
|
||||
if constexpr (need_filter)
|
||||
filter[pos] = 1;
|
||||
}
|
||||
static void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]]);
|
||||
|
||||
template <typename AddedColumns>
|
||||
static ColumnPtr buildAdditionalFilter(
|
||||
size_t left_start_row,
|
||||
const std::vector<RowRef> & selected_rows,
|
||||
const std::vector<size_t> & row_replicate_offset,
|
||||
AddedColumns & added_columns)
|
||||
{
|
||||
ColumnPtr result_column;
|
||||
do
|
||||
{
|
||||
if (selected_rows.empty())
|
||||
{
|
||||
result_column = ColumnUInt8::create();
|
||||
break;
|
||||
}
|
||||
const Block & sample_right_block = *selected_rows.begin()->block;
|
||||
if (!sample_right_block || !added_columns.additional_filter_expression)
|
||||
{
|
||||
auto filter = ColumnUInt8::create();
|
||||
filter->insertMany(1, selected_rows.size());
|
||||
result_column = std::move(filter);
|
||||
break;
|
||||
}
|
||||
|
||||
auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes();
|
||||
if (required_cols.empty())
|
||||
{
|
||||
Block block;
|
||||
added_columns.additional_filter_expression->execute(block);
|
||||
result_column = block.getByPosition(0).column->cloneResized(selected_rows.size());
|
||||
break;
|
||||
}
|
||||
NameSet required_column_names;
|
||||
for (auto & col : required_cols)
|
||||
required_column_names.insert(col.name);
|
||||
|
||||
Block executed_block;
|
||||
size_t right_col_pos = 0;
|
||||
for (const auto & col : sample_right_block.getColumnsWithTypeAndName())
|
||||
{
|
||||
if (required_column_names.contains(col.name))
|
||||
{
|
||||
auto new_col = col.column->cloneEmpty();
|
||||
for (const auto & selected_row : selected_rows)
|
||||
{
|
||||
const auto & src_col = selected_row.block->getByPosition(right_col_pos);
|
||||
new_col->insertFrom(*src_col.column, selected_row.row_num);
|
||||
}
|
||||
executed_block.insert({std::move(new_col), col.type, col.name});
|
||||
}
|
||||
right_col_pos += 1;
|
||||
}
|
||||
if (!executed_block)
|
||||
{
|
||||
result_column = ColumnUInt8::create();
|
||||
break;
|
||||
}
|
||||
|
||||
for (const auto & col_name : required_column_names)
|
||||
{
|
||||
const auto * src_col = added_columns.left_block.findByName(col_name);
|
||||
if (!src_col)
|
||||
continue;
|
||||
auto new_col = src_col->column->cloneEmpty();
|
||||
size_t prev_left_offset = 0;
|
||||
for (size_t i = 1; i < row_replicate_offset.size(); ++i)
|
||||
{
|
||||
const size_t & left_offset = row_replicate_offset[i];
|
||||
size_t rows = left_offset - prev_left_offset;
|
||||
if (rows)
|
||||
new_col->insertManyFrom(*src_col->column, left_start_row + i - 1, rows);
|
||||
prev_left_offset = left_offset;
|
||||
}
|
||||
executed_block.insert({std::move(new_col), src_col->type, col_name});
|
||||
}
|
||||
if (!executed_block)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"required columns: [{}], but not found any in left/right table. right table: {}, left table: {}",
|
||||
required_cols.toString(),
|
||||
sample_right_block.dumpNames(),
|
||||
added_columns.left_block.dumpNames());
|
||||
}
|
||||
|
||||
for (const auto & col : executed_block.getColumnsWithTypeAndName())
|
||||
if (!col.column || !col.type)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal nullptr column in input block: {}", executed_block.dumpStructure());
|
||||
|
||||
added_columns.additional_filter_expression->execute(executed_block);
|
||||
result_column = executed_block.getByPosition(0).column->convertToFullColumnIfConst();
|
||||
executed_block.clear();
|
||||
} while (false);
|
||||
|
||||
result_column = result_column->convertToFullIfNeeded();
|
||||
if (result_column->isNullable())
|
||||
{
|
||||
/// Convert Nullable(UInt8) to UInt8 ensuring that nulls are zeros
|
||||
/// Trying to avoid copying data, since we are the only owner of the column.
|
||||
ColumnPtr mask_column = assert_cast<const ColumnNullable &>(*result_column).getNullMapColumnPtr();
|
||||
|
||||
MutableColumnPtr mutable_column;
|
||||
{
|
||||
ColumnPtr nested_column = assert_cast<const ColumnNullable &>(*result_column).getNestedColumnPtr();
|
||||
result_column.reset();
|
||||
mutable_column = IColumn::mutate(std::move(nested_column));
|
||||
}
|
||||
|
||||
auto & column_data = assert_cast<ColumnUInt8 &>(*mutable_column).getData();
|
||||
const auto & mask_column_data = assert_cast<const ColumnUInt8 &>(*mask_column).getData();
|
||||
for (size_t i = 0; i < column_data.size(); ++i)
|
||||
{
|
||||
if (mask_column_data[i])
|
||||
column_data[i] = 0;
|
||||
}
|
||||
return mutable_column;
|
||||
}
|
||||
return result_column;
|
||||
}
|
||||
AddedColumns & added_columns);
|
||||
|
||||
/// First to collect all matched rows refs by join keys, then filter out rows which are not true in additional filter expression.
|
||||
template <typename KeyGetter, typename Map, bool need_replication, typename AddedColumns>
|
||||
template <typename KeyGetter, typename Map, typename AddedColumns>
|
||||
static size_t joinRightColumnsWithAddtitionalFilter(
|
||||
std::vector<KeyGetter> && key_getter_vector,
|
||||
const std::vector<const Map *> & mapv,
|
||||
AddedColumns & added_columns,
|
||||
JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]],
|
||||
bool need_filter [[maybe_unused]],
|
||||
bool need_flags [[maybe_unused]],
|
||||
bool add_missing [[maybe_unused]],
|
||||
bool flag_per_row [[maybe_unused]])
|
||||
{
|
||||
size_t left_block_rows = added_columns.rows_to_add;
|
||||
if (need_filter)
|
||||
added_columns.filter = IColumn::Filter(left_block_rows, 0);
|
||||
|
||||
std::unique_ptr<Arena> pool;
|
||||
|
||||
if constexpr (need_replication)
|
||||
added_columns.offsets_to_replicate = std::make_unique<IColumn::Offsets>(left_block_rows);
|
||||
|
||||
std::vector<size_t> row_replicate_offset;
|
||||
row_replicate_offset.reserve(left_block_rows);
|
||||
|
||||
using FindResult = typename KeyGetter::FindResult;
|
||||
size_t max_joined_block_rows = added_columns.max_joined_block_rows;
|
||||
size_t left_row_iter = 0;
|
||||
PreSelectedRows selected_rows;
|
||||
selected_rows.reserve(left_block_rows);
|
||||
std::vector<FindResult> find_results;
|
||||
find_results.reserve(left_block_rows);
|
||||
bool exceeded_max_block_rows = false;
|
||||
IColumn::Offset total_added_rows = 0;
|
||||
IColumn::Offset current_added_rows = 0;
|
||||
|
||||
auto collect_keys_matched_rows_refs = [&]()
|
||||
{
|
||||
pool = std::make_unique<Arena>();
|
||||
find_results.clear();
|
||||
row_replicate_offset.clear();
|
||||
row_replicate_offset.push_back(0);
|
||||
current_added_rows = 0;
|
||||
selected_rows.clear();
|
||||
for (; left_row_iter < left_block_rows; ++left_row_iter)
|
||||
{
|
||||
if constexpr (need_replication)
|
||||
{
|
||||
if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows))
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
KnownRowsHolder<true> all_flag_known_rows;
|
||||
KnownRowsHolder<false> single_flag_know_rows;
|
||||
for (size_t join_clause_idx = 0; join_clause_idx < added_columns.join_on_keys.size(); ++join_clause_idx)
|
||||
{
|
||||
const auto & join_keys = added_columns.join_on_keys[join_clause_idx];
|
||||
if (join_keys.null_map && (*join_keys.null_map)[left_row_iter])
|
||||
continue;
|
||||
|
||||
bool row_acceptable = !join_keys.isRowFiltered(left_row_iter);
|
||||
auto find_result = row_acceptable
|
||||
? key_getter_vector[join_clause_idx].findKey(*(mapv[join_clause_idx]), left_row_iter, *pool)
|
||||
: FindResult();
|
||||
|
||||
if (find_result.isFound())
|
||||
{
|
||||
auto & mapped = find_result.getMapped();
|
||||
find_results.push_back(find_result);
|
||||
if (flag_per_row)
|
||||
addFoundRowAll<Map, false, true>(mapped, selected_rows, current_added_rows, all_flag_known_rows, nullptr);
|
||||
else
|
||||
addFoundRowAll<Map, false, false>(mapped, selected_rows, current_added_rows, single_flag_know_rows, nullptr);
|
||||
}
|
||||
}
|
||||
row_replicate_offset.push_back(current_added_rows);
|
||||
}
|
||||
};
|
||||
|
||||
auto copy_final_matched_rows = [&](size_t left_start_row, ColumnPtr filter_col)
|
||||
{
|
||||
const PaddedPODArray<UInt8> & filter_flags = assert_cast<const ColumnUInt8 &>(*filter_col).getData();
|
||||
|
||||
size_t prev_replicated_row = 0;
|
||||
auto selected_right_row_it = selected_rows.begin();
|
||||
size_t find_result_index = 0;
|
||||
for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i)
|
||||
{
|
||||
bool any_matched = false;
|
||||
/// For all right join, flag_per_row is true, we need mark used flags for each row.
|
||||
if (flag_per_row)
|
||||
{
|
||||
for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row)
|
||||
{
|
||||
if (filter_flags[replicated_row])
|
||||
{
|
||||
any_matched = true;
|
||||
added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing);
|
||||
total_added_rows += 1;
|
||||
if (need_flags)
|
||||
used_flags.template setUsed<true, true>(selected_right_row_it->block, selected_right_row_it->row_num, 0);
|
||||
}
|
||||
++selected_right_row_it;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row)
|
||||
{
|
||||
if (filter_flags[replicated_row])
|
||||
{
|
||||
any_matched = true;
|
||||
added_columns.appendFromBlock(*selected_right_row_it->block, selected_right_row_it->row_num, add_missing);
|
||||
total_added_rows += 1;
|
||||
}
|
||||
++selected_right_row_it;
|
||||
}
|
||||
}
|
||||
if (!any_matched)
|
||||
{
|
||||
if (add_missing)
|
||||
addNotFoundRow<true, need_replication>(added_columns, total_added_rows);
|
||||
else
|
||||
addNotFoundRow<false, need_replication>(added_columns, total_added_rows);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!flag_per_row && need_flags)
|
||||
used_flags.template setUsed<true, false>(find_results[find_result_index]);
|
||||
if (need_filter)
|
||||
setUsed<true>(added_columns.filter, left_start_row + i - 1);
|
||||
if (add_missing)
|
||||
added_columns.applyLazyDefaults();
|
||||
}
|
||||
find_result_index += (prev_replicated_row != row_replicate_offset[i]);
|
||||
|
||||
if constexpr (need_replication)
|
||||
{
|
||||
(*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows;
|
||||
}
|
||||
prev_replicated_row = row_replicate_offset[i];
|
||||
}
|
||||
};
|
||||
|
||||
while (left_row_iter < left_block_rows && !exceeded_max_block_rows)
|
||||
{
|
||||
auto left_start_row = left_row_iter;
|
||||
collect_keys_matched_rows_refs();
|
||||
if (selected_rows.size() != current_added_rows || row_replicate_offset.size() != left_row_iter - left_start_row + 1)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Sizes are mismatched. selected_rows.size:{}, current_added_rows:{}, row_replicate_offset.size:{}, left_row_iter: {}, "
|
||||
"left_start_row: {}",
|
||||
selected_rows.size(),
|
||||
current_added_rows,
|
||||
row_replicate_offset.size(),
|
||||
left_row_iter,
|
||||
left_start_row);
|
||||
}
|
||||
auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns);
|
||||
copy_final_matched_rows(left_start_row, filter_col);
|
||||
|
||||
if constexpr (need_replication)
|
||||
{
|
||||
// Add a check for current_added_rows to avoid run the filter expression on too small size batch.
|
||||
if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024)
|
||||
exceeded_max_block_rows = true;
|
||||
}
|
||||
}
|
||||
|
||||
if constexpr (need_replication)
|
||||
{
|
||||
added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter);
|
||||
added_columns.filter.resize_assume_reserved(left_row_iter);
|
||||
}
|
||||
added_columns.applyLazyDefaults();
|
||||
return left_row_iter;
|
||||
}
|
||||
bool flag_per_row [[maybe_unused]]);
|
||||
|
||||
/// Cut first num_rows rows from block in place and returns block with remaining rows
|
||||
static Block sliceBlock(Block & block, size_t num_rows)
|
||||
{
|
||||
size_t total_rows = block.rows();
|
||||
if (num_rows >= total_rows)
|
||||
return {};
|
||||
size_t remaining_rows = total_rows - num_rows;
|
||||
Block remaining_block = block.cloneEmpty();
|
||||
for (size_t i = 0; i < block.columns(); ++i)
|
||||
{
|
||||
auto & col = block.getByPosition(i);
|
||||
remaining_block.getByPosition(i).column = col.column->cut(num_rows, remaining_rows);
|
||||
col.column = col.column->cut(0, num_rows);
|
||||
}
|
||||
return remaining_block;
|
||||
}
|
||||
static Block sliceBlock(Block & block, size_t num_rows);
|
||||
|
||||
/** Since we do not store right key columns,
|
||||
* this function is used to copy left key columns to right key columns.
|
||||
@ -864,70 +158,22 @@ private:
|
||||
const DataTypePtr & right_key_type,
|
||||
const String & renamed_right_column,
|
||||
const ColumnWithTypeAndName & left_column,
|
||||
const IColumn::Filter * null_map_filter = nullptr)
|
||||
{
|
||||
ColumnWithTypeAndName right_column = left_column;
|
||||
right_column.name = renamed_right_column;
|
||||
const IColumn::Filter * null_map_filter = nullptr);
|
||||
|
||||
if (null_map_filter)
|
||||
right_column.column = JoinCommon::filterWithBlanks(right_column.column, *null_map_filter);
|
||||
static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable);
|
||||
|
||||
bool should_be_nullable = isNullableOrLowCardinalityNullable(right_key_type);
|
||||
if (null_map_filter)
|
||||
correctNullabilityInplace(right_column, should_be_nullable, *null_map_filter);
|
||||
else
|
||||
correctNullabilityInplace(right_column, should_be_nullable);
|
||||
|
||||
if (!right_column.type->equals(*right_key_type))
|
||||
{
|
||||
right_column.column = castColumnAccurate(right_column, right_key_type);
|
||||
right_column.type = right_key_type;
|
||||
}
|
||||
|
||||
right_column.column = right_column.column->convertToFullColumnIfConst();
|
||||
return right_column;
|
||||
}
|
||||
|
||||
static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable)
|
||||
{
|
||||
if (nullable)
|
||||
{
|
||||
JoinCommon::convertColumnToNullable(column);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We have to replace values masked by NULLs with defaults.
|
||||
if (column.column)
|
||||
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(&*column.column))
|
||||
column.column = JoinCommon::filterWithBlanks(column.column, nullable_column->getNullMapColumn().getData(), true);
|
||||
|
||||
JoinCommon::removeColumnNullability(column);
|
||||
}
|
||||
}
|
||||
|
||||
static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map)
|
||||
{
|
||||
if (nullable)
|
||||
{
|
||||
JoinCommon::convertColumnToNullable(column);
|
||||
if (column.type->isNullable() && !negative_null_map.empty())
|
||||
{
|
||||
MutableColumnPtr mutable_column = IColumn::mutate(std::move(column.column));
|
||||
assert_cast<ColumnNullable &>(*mutable_column).applyNegatedNullMap(negative_null_map);
|
||||
column.column = std::move(mutable_column);
|
||||
}
|
||||
}
|
||||
else
|
||||
JoinCommon::removeColumnNullability(column);
|
||||
}
|
||||
static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map);
|
||||
};
|
||||
|
||||
/// Instantiate template class ahead in different .cpp files to avoid `too large translation unit`.
|
||||
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::RightAny, HashJoin::MapsOne>;
|
||||
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::Any, HashJoin::MapsOne>;
|
||||
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::Any, HashJoin::MapsAll>;
|
||||
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::All, HashJoin::MapsAll>;
|
||||
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::Semi, HashJoin::MapsOne>;
|
||||
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::Semi, HashJoin::MapsAll>;
|
||||
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::Anti, HashJoin::MapsOne>;
|
||||
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::Anti, HashJoin::MapsAll>;
|
||||
extern template class HashJoinMethods<JoinKind::Left, JoinStrictness::Asof, HashJoin::MapsAsof>;
|
||||
|
||||
extern template class HashJoinMethods<JoinKind::Right, JoinStrictness::RightAny, HashJoin::MapsOne>;
|
||||
@ -939,6 +185,7 @@ extern template class HashJoinMethods<JoinKind::Right, JoinStrictness::Asof, Has
|
||||
|
||||
extern template class HashJoinMethods<JoinKind::Inner, JoinStrictness::RightAny, HashJoin::MapsOne>;
|
||||
extern template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Any, HashJoin::MapsOne>;
|
||||
extern template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Any, HashJoin::MapsAll>;
|
||||
extern template class HashJoinMethods<JoinKind::Inner, JoinStrictness::All, HashJoin::MapsAll>;
|
||||
extern template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Semi, HashJoin::MapsOne>;
|
||||
extern template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Anti, HashJoin::MapsOne>;
|
||||
|
936
src/Interpreters/HashJoin/HashJoinMethodsImpl.h
Normal file
936
src/Interpreters/HashJoin/HashJoinMethodsImpl.h
Normal file
@ -0,0 +1,936 @@
|
||||
#pragma once
|
||||
#include <Interpreters/HashJoin/HashJoinMethods.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNSUPPORTED_JOIN_KEYS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::insertFromBlockImpl(
|
||||
HashJoin & join,
|
||||
HashJoin::Type type,
|
||||
MapsTemplate & maps,
|
||||
size_t rows,
|
||||
const ColumnRawPtrs & key_columns,
|
||||
const Sizes & key_sizes,
|
||||
Block * stored_block,
|
||||
ConstNullMapPtr null_map,
|
||||
UInt8ColumnDataPtr join_mask,
|
||||
Arena & pool,
|
||||
bool & is_inserted)
|
||||
{
|
||||
switch (type)
|
||||
{
|
||||
case HashJoin::Type::EMPTY:
|
||||
[[fallthrough]];
|
||||
case HashJoin::Type::CROSS:
|
||||
/// Do nothing. We will only save block, and it is enough
|
||||
is_inserted = true;
|
||||
return 0;
|
||||
|
||||
#define M(TYPE) \
|
||||
case HashJoin::Type::TYPE: \
|
||||
return insertFromBlockImplTypeCase< \
|
||||
typename KeyGetterForType<HashJoin::Type::TYPE, std::remove_reference_t<decltype(*maps.TYPE)>>::Type>( \
|
||||
join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); \
|
||||
break;
|
||||
|
||||
APPLY_FOR_JOIN_VARIANTS(M)
|
||||
#undef M
|
||||
}
|
||||
}
|
||||
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinBlockImpl(
|
||||
const HashJoin & join, Block & block, const Block & block_with_columns_to_add, const MapsTemplateVector & maps_, bool is_join_get)
|
||||
{
|
||||
constexpr JoinFeatures<KIND, STRICTNESS, MapsTemplate> join_features;
|
||||
|
||||
std::vector<JoinOnKeyColumns> join_on_keys;
|
||||
const auto & onexprs = join.table_join->getClauses();
|
||||
for (size_t i = 0; i < onexprs.size(); ++i)
|
||||
{
|
||||
const auto & key_names = !is_join_get ? onexprs[i].key_names_left : onexprs[i].key_names_right;
|
||||
join_on_keys.emplace_back(block, key_names, onexprs[i].condColumnNames().first, join.key_sizes[i]);
|
||||
}
|
||||
size_t existing_columns = block.columns();
|
||||
|
||||
/** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized.
|
||||
* Because if they are constants, then in the "not joined" rows, they may have different values
|
||||
* - default values, which can differ from the values of these constants.
|
||||
*/
|
||||
if constexpr (join_features.right || join_features.full)
|
||||
{
|
||||
materializeBlockInplace(block);
|
||||
}
|
||||
|
||||
/** For LEFT/INNER JOIN, the saved blocks do not contain keys.
|
||||
* For FULL/RIGHT JOIN, the saved blocks contain keys;
|
||||
* but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped.
|
||||
* For ASOF, the last column is used as the ASOF column
|
||||
*/
|
||||
AddedColumns<!join_features.is_any_join> added_columns(
|
||||
block,
|
||||
block_with_columns_to_add,
|
||||
join.savedBlockSample(),
|
||||
join,
|
||||
std::move(join_on_keys),
|
||||
join.table_join->getMixedJoinExpression(),
|
||||
join_features.is_asof_join,
|
||||
is_join_get);
|
||||
|
||||
bool has_required_right_keys = (join.required_right_keys.columns() != 0);
|
||||
added_columns.need_filter = join_features.need_filter || has_required_right_keys;
|
||||
added_columns.max_joined_block_rows = join.max_joined_block_rows;
|
||||
if (!added_columns.max_joined_block_rows)
|
||||
added_columns.max_joined_block_rows = std::numeric_limits<size_t>::max();
|
||||
else
|
||||
added_columns.reserve(join_features.need_replication);
|
||||
|
||||
size_t num_joined = switchJoinRightColumns(maps_, added_columns, join.data->type, *join.used_flags);
|
||||
/// Do not hold memory for join_on_keys anymore
|
||||
added_columns.join_on_keys.clear();
|
||||
Block remaining_block = sliceBlock(block, num_joined);
|
||||
|
||||
added_columns.buildOutput();
|
||||
for (size_t i = 0; i < added_columns.size(); ++i)
|
||||
block.insert(added_columns.moveColumn(i));
|
||||
|
||||
std::vector<size_t> right_keys_to_replicate [[maybe_unused]];
|
||||
|
||||
if constexpr (join_features.need_filter)
|
||||
{
|
||||
/// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones.
|
||||
for (size_t i = 0; i < existing_columns; ++i)
|
||||
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(added_columns.filter, -1);
|
||||
|
||||
/// Add join key columns from right block if needed using value from left table because of equality
|
||||
for (size_t i = 0; i < join.required_right_keys.columns(); ++i)
|
||||
{
|
||||
const auto & right_key = join.required_right_keys.getByPosition(i);
|
||||
/// asof column is already in block.
|
||||
if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back())
|
||||
continue;
|
||||
|
||||
const auto & left_column = block.getByName(join.required_right_keys_sources[i]);
|
||||
const auto & right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name);
|
||||
auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column);
|
||||
block.insert(std::move(right_col));
|
||||
}
|
||||
}
|
||||
else if (has_required_right_keys)
|
||||
{
|
||||
/// Add join key columns from right block if needed.
|
||||
for (size_t i = 0; i < join.required_right_keys.columns(); ++i)
|
||||
{
|
||||
const auto & right_key = join.required_right_keys.getByPosition(i);
|
||||
auto right_col_name = join.getTableJoin().renamedRightColumnName(right_key.name);
|
||||
/// asof column is already in block.
|
||||
if (join_features.is_asof_join && right_key.name == join.table_join->getOnlyClause().key_names_right.back())
|
||||
continue;
|
||||
|
||||
const auto & left_column = block.getByName(join.required_right_keys_sources[i]);
|
||||
auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &added_columns.filter);
|
||||
block.insert(std::move(right_col));
|
||||
|
||||
if constexpr (join_features.need_replication)
|
||||
right_keys_to_replicate.push_back(block.getPositionByName(right_col_name));
|
||||
}
|
||||
}
|
||||
|
||||
if constexpr (join_features.need_replication)
|
||||
{
|
||||
std::unique_ptr<IColumn::Offsets> & offsets_to_replicate = added_columns.offsets_to_replicate;
|
||||
|
||||
/// If ALL ... JOIN - we replicate all the columns except the new ones.
|
||||
for (size_t i = 0; i < existing_columns; ++i)
|
||||
{
|
||||
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate);
|
||||
}
|
||||
|
||||
/// Replicate additional right keys
|
||||
for (size_t pos : right_keys_to_replicate)
|
||||
{
|
||||
block.safeGetByPosition(pos).column = block.safeGetByPosition(pos).column->replicate(*offsets_to_replicate);
|
||||
}
|
||||
}
|
||||
return remaining_block;
|
||||
}
|
||||
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
template <typename KeyGetter, bool is_asof_join>
|
||||
KeyGetter HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes)
|
||||
{
|
||||
if constexpr (is_asof_join)
|
||||
{
|
||||
auto key_column_copy = key_columns;
|
||||
auto key_size_copy = key_sizes;
|
||||
key_column_copy.pop_back();
|
||||
key_size_copy.pop_back();
|
||||
return KeyGetter(key_column_copy, key_size_copy, nullptr);
|
||||
}
|
||||
else
|
||||
return KeyGetter(key_columns, key_sizes, nullptr);
|
||||
}
|
||||
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
template <typename KeyGetter, typename HashMap>
|
||||
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::insertFromBlockImplTypeCase(
|
||||
HashJoin & join,
|
||||
HashMap & map,
|
||||
size_t rows,
|
||||
const ColumnRawPtrs & key_columns,
|
||||
const Sizes & key_sizes,
|
||||
Block * stored_block,
|
||||
ConstNullMapPtr null_map,
|
||||
UInt8ColumnDataPtr join_mask,
|
||||
Arena & pool,
|
||||
bool & is_inserted)
|
||||
{
|
||||
[[maybe_unused]] constexpr bool mapped_one = std::is_same_v<typename HashMap::mapped_type, RowRef>;
|
||||
constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof;
|
||||
|
||||
const IColumn * asof_column [[maybe_unused]] = nullptr;
|
||||
if constexpr (is_asof_join)
|
||||
asof_column = key_columns.back();
|
||||
|
||||
auto key_getter = createKeyGetter<KeyGetter, is_asof_join>(key_columns, key_sizes);
|
||||
|
||||
/// For ALL and ASOF join always insert values
|
||||
is_inserted = !mapped_one || is_asof_join;
|
||||
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
if (null_map && (*null_map)[i])
|
||||
{
|
||||
/// nulls are not inserted into hash table,
|
||||
/// keep them for RIGHT and FULL joins
|
||||
is_inserted = true;
|
||||
continue;
|
||||
}
|
||||
|
||||
/// Check condition for right table from ON section
|
||||
if (join_mask && !(*join_mask)[i])
|
||||
continue;
|
||||
|
||||
if constexpr (is_asof_join)
|
||||
Inserter<HashMap, KeyGetter>::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column);
|
||||
else if constexpr (mapped_one)
|
||||
is_inserted |= Inserter<HashMap, KeyGetter>::insertOne(join, map, key_getter, stored_block, i, pool);
|
||||
else
|
||||
Inserter<HashMap, KeyGetter>::insertAll(join, map, key_getter, stored_block, i, pool);
|
||||
}
|
||||
return map.getBufferSizeInCells();
|
||||
}
|
||||
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
template <typename AddedColumns>
|
||||
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::switchJoinRightColumns(
|
||||
const std::vector<const MapsTemplate *> & mapv,
|
||||
AddedColumns & added_columns,
|
||||
HashJoin::Type type,
|
||||
JoinStuff::JoinUsedFlags & used_flags)
|
||||
{
|
||||
constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof;
|
||||
switch (type)
|
||||
{
|
||||
case HashJoin::Type::EMPTY: {
|
||||
if constexpr (!is_asof_join)
|
||||
{
|
||||
using KeyGetter = KeyGetterEmpty<typename MapsTemplate::MappedType>;
|
||||
std::vector<KeyGetter> key_getter_vector;
|
||||
key_getter_vector.emplace_back();
|
||||
|
||||
using MapTypeVal = typename KeyGetter::MappedType;
|
||||
std::vector<const MapTypeVal *> a_map_type_vector;
|
||||
a_map_type_vector.emplace_back();
|
||||
return joinRightColumnsSwitchNullability<KeyGetter>(
|
||||
std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags);
|
||||
}
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys. Type: {}", type);
|
||||
}
|
||||
#define M(TYPE) \
|
||||
case HashJoin::Type::TYPE: { \
|
||||
using MapTypeVal = const typename std::remove_reference_t<decltype(MapsTemplate::TYPE)>::element_type; \
|
||||
using KeyGetter = typename KeyGetterForType<HashJoin::Type::TYPE, MapTypeVal>::Type; \
|
||||
std::vector<const MapTypeVal *> a_map_type_vector(mapv.size()); \
|
||||
std::vector<KeyGetter> key_getter_vector; \
|
||||
for (size_t d = 0; d < added_columns.join_on_keys.size(); ++d) \
|
||||
{ \
|
||||
const auto & join_on_key = added_columns.join_on_keys[d]; \
|
||||
a_map_type_vector[d] = mapv[d]->TYPE.get(); \
|
||||
key_getter_vector.push_back( \
|
||||
std::move(createKeyGetter<KeyGetter, is_asof_join>(join_on_key.key_columns, join_on_key.key_sizes))); \
|
||||
} \
|
||||
return joinRightColumnsSwitchNullability<KeyGetter>(std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags); \
|
||||
}
|
||||
APPLY_FOR_JOIN_VARIANTS(M)
|
||||
#undef M
|
||||
|
||||
default:
|
||||
throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", type);
|
||||
}
|
||||
}
|
||||
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
template <typename KeyGetter, typename Map, typename AddedColumns>
|
||||
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumnsSwitchNullability(
|
||||
std::vector<KeyGetter> && key_getter_vector,
|
||||
const std::vector<const Map *> & mapv,
|
||||
AddedColumns & added_columns,
|
||||
JoinStuff::JoinUsedFlags & used_flags)
|
||||
{
|
||||
if (added_columns.need_filter)
|
||||
{
|
||||
return joinRightColumnsSwitchMultipleDisjuncts<KeyGetter, Map, true>(
|
||||
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
|
||||
}
|
||||
else
|
||||
{
|
||||
return joinRightColumnsSwitchMultipleDisjuncts<KeyGetter, Map, false>(
|
||||
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
|
||||
}
|
||||
}
|
||||
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
template <typename KeyGetter, typename Map, bool need_filter, typename AddedColumns>
|
||||
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumnsSwitchMultipleDisjuncts(
|
||||
std::vector<KeyGetter> && key_getter_vector,
|
||||
const std::vector<const Map *> & mapv,
|
||||
AddedColumns & added_columns,
|
||||
JoinStuff::JoinUsedFlags & used_flags)
|
||||
{
|
||||
constexpr JoinFeatures<KIND, STRICTNESS, MapsTemplate> join_features;
|
||||
if constexpr (join_features.is_maps_all)
|
||||
{
|
||||
if (added_columns.additional_filter_expression)
|
||||
{
|
||||
bool mark_per_row_used = join_features.right || join_features.full || mapv.size() > 1;
|
||||
return joinRightColumnsWithAddtitionalFilter<KeyGetter, Map>(
|
||||
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags, need_filter, mark_per_row_used);
|
||||
}
|
||||
}
|
||||
|
||||
if (added_columns.additional_filter_expression)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Additional filter expression is not supported for this JOIN");
|
||||
|
||||
return mapv.size() > 1 ? joinRightColumns<KeyGetter, Map, need_filter, true>(
|
||||
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags)
|
||||
: joinRightColumns<KeyGetter, Map, need_filter, false>(
|
||||
std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
|
||||
}
|
||||
|
||||
|
||||
/// Joins right table columns which indexes are present in right_indexes using specified map.
|
||||
/// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS).
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
template <typename KeyGetter, typename Map, bool need_filter, bool flag_per_row, typename AddedColumns>
|
||||
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumns(
|
||||
std::vector<KeyGetter> && key_getter_vector,
|
||||
const std::vector<const Map *> & mapv,
|
||||
AddedColumns & added_columns,
|
||||
JoinStuff::JoinUsedFlags & used_flags)
|
||||
{
|
||||
constexpr JoinFeatures<KIND, STRICTNESS, MapsTemplate> join_features;
|
||||
|
||||
size_t rows = added_columns.rows_to_add;
|
||||
if constexpr (need_filter)
|
||||
added_columns.filter = IColumn::Filter(rows, 0);
|
||||
|
||||
Arena pool;
|
||||
|
||||
if constexpr (join_features.need_replication)
|
||||
added_columns.offsets_to_replicate = std::make_unique<IColumn::Offsets>(rows);
|
||||
|
||||
IColumn::Offset current_offset = 0;
|
||||
size_t max_joined_block_rows = added_columns.max_joined_block_rows;
|
||||
size_t i = 0;
|
||||
for (; i < rows; ++i)
|
||||
{
|
||||
if constexpr (join_features.need_replication)
|
||||
{
|
||||
if (unlikely(current_offset >= max_joined_block_rows))
|
||||
{
|
||||
added_columns.offsets_to_replicate->resize_assume_reserved(i);
|
||||
added_columns.filter.resize_assume_reserved(i);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
bool right_row_found = false;
|
||||
KnownRowsHolder<flag_per_row> known_rows;
|
||||
for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx)
|
||||
{
|
||||
const auto & join_keys = added_columns.join_on_keys[onexpr_idx];
|
||||
if (join_keys.null_map && (*join_keys.null_map)[i])
|
||||
continue;
|
||||
|
||||
bool row_acceptable = !join_keys.isRowFiltered(i);
|
||||
using FindResult = typename KeyGetter::FindResult;
|
||||
auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult();
|
||||
|
||||
if (find_result.isFound())
|
||||
{
|
||||
right_row_found = true;
|
||||
auto & mapped = find_result.getMapped();
|
||||
if constexpr (join_features.is_asof_join)
|
||||
{
|
||||
const IColumn & left_asof_key = added_columns.leftAsofKey();
|
||||
|
||||
auto row_ref = mapped->findAsof(left_asof_key, i);
|
||||
if (row_ref.block)
|
||||
{
|
||||
setUsed<need_filter>(added_columns.filter, i);
|
||||
if constexpr (flag_per_row)
|
||||
used_flags.template setUsed<join_features.need_flags, flag_per_row>(row_ref.block, row_ref.row_num, 0);
|
||||
else
|
||||
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
|
||||
|
||||
added_columns.appendFromBlock(*row_ref.block, row_ref.row_num, join_features.add_missing);
|
||||
}
|
||||
else
|
||||
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, current_offset);
|
||||
}
|
||||
else if constexpr (join_features.is_all_join)
|
||||
{
|
||||
setUsed<need_filter>(added_columns.filter, i);
|
||||
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
|
||||
auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr;
|
||||
addFoundRowAll<Map, join_features.add_missing>(mapped, added_columns, current_offset, known_rows, used_flags_opt);
|
||||
}
|
||||
else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right)
|
||||
{
|
||||
/// Use first appeared left key + it needs left columns replication
|
||||
bool used_once = used_flags.template setUsedOnce<join_features.need_flags, flag_per_row>(find_result);
|
||||
if (used_once)
|
||||
{
|
||||
auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr;
|
||||
setUsed<need_filter>(added_columns.filter, i);
|
||||
addFoundRowAll<Map, join_features.add_missing>(mapped, added_columns, current_offset, known_rows, used_flags_opt);
|
||||
}
|
||||
}
|
||||
else if constexpr (join_features.is_any_join && join_features.inner)
|
||||
{
|
||||
bool used_once = used_flags.template setUsedOnce<join_features.need_flags, flag_per_row>(find_result);
|
||||
|
||||
/// Use first appeared left key only
|
||||
if (used_once)
|
||||
{
|
||||
setUsed<need_filter>(added_columns.filter, i);
|
||||
added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing);
|
||||
}
|
||||
|
||||
break;
|
||||
}
|
||||
else if constexpr (join_features.is_any_join && join_features.full)
|
||||
{
|
||||
/// TODO
|
||||
}
|
||||
else if constexpr (join_features.is_anti_join)
|
||||
{
|
||||
if constexpr (join_features.right && join_features.need_flags)
|
||||
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
|
||||
}
|
||||
else /// ANY LEFT, SEMI LEFT, old ANY (RightAny)
|
||||
{
|
||||
setUsed<need_filter>(added_columns.filter, i);
|
||||
used_flags.template setUsed<join_features.need_flags, flag_per_row>(find_result);
|
||||
added_columns.appendFromBlock(*mapped.block, mapped.row_num, join_features.add_missing);
|
||||
|
||||
if (join_features.is_any_or_semi_join)
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!right_row_found)
|
||||
{
|
||||
if constexpr (join_features.is_anti_join && join_features.left)
|
||||
setUsed<need_filter>(added_columns.filter, i);
|
||||
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, current_offset);
|
||||
}
|
||||
|
||||
if constexpr (join_features.need_replication)
|
||||
{
|
||||
(*added_columns.offsets_to_replicate)[i] = current_offset;
|
||||
}
|
||||
}
|
||||
|
||||
added_columns.applyLazyDefaults();
|
||||
return i;
|
||||
}
|
||||
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
template <bool need_filter>
|
||||
void HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]])
|
||||
{
|
||||
if constexpr (need_filter)
|
||||
filter[pos] = 1;
|
||||
}
|
||||
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
template <typename AddedColumns>
|
||||
ColumnPtr HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::buildAdditionalFilter(
|
||||
size_t left_start_row,
|
||||
const std::vector<RowRef> & selected_rows,
|
||||
const std::vector<size_t> & row_replicate_offset,
|
||||
AddedColumns & added_columns)
|
||||
{
|
||||
ColumnPtr result_column;
|
||||
do
|
||||
{
|
||||
if (selected_rows.empty())
|
||||
{
|
||||
result_column = ColumnUInt8::create();
|
||||
break;
|
||||
}
|
||||
const Block & sample_right_block = *selected_rows.begin()->block;
|
||||
if (!sample_right_block || !added_columns.additional_filter_expression)
|
||||
{
|
||||
auto filter = ColumnUInt8::create();
|
||||
filter->insertMany(1, selected_rows.size());
|
||||
result_column = std::move(filter);
|
||||
break;
|
||||
}
|
||||
|
||||
auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes();
|
||||
if (required_cols.empty())
|
||||
{
|
||||
Block block;
|
||||
added_columns.additional_filter_expression->execute(block);
|
||||
result_column = block.getByPosition(0).column->cloneResized(selected_rows.size());
|
||||
break;
|
||||
}
|
||||
NameSet required_column_names;
|
||||
for (auto & col : required_cols)
|
||||
required_column_names.insert(col.name);
|
||||
|
||||
Block executed_block;
|
||||
size_t right_col_pos = 0;
|
||||
for (const auto & col : sample_right_block.getColumnsWithTypeAndName())
|
||||
{
|
||||
if (required_column_names.contains(col.name))
|
||||
{
|
||||
auto new_col = col.column->cloneEmpty();
|
||||
for (const auto & selected_row : selected_rows)
|
||||
{
|
||||
const auto & src_col = selected_row.block->getByPosition(right_col_pos);
|
||||
new_col->insertFrom(*src_col.column, selected_row.row_num);
|
||||
}
|
||||
executed_block.insert({std::move(new_col), col.type, col.name});
|
||||
}
|
||||
right_col_pos += 1;
|
||||
}
|
||||
if (!executed_block)
|
||||
{
|
||||
result_column = ColumnUInt8::create();
|
||||
break;
|
||||
}
|
||||
|
||||
for (const auto & col_name : required_column_names)
|
||||
{
|
||||
const auto * src_col = added_columns.left_block.findByName(col_name);
|
||||
if (!src_col)
|
||||
continue;
|
||||
auto new_col = src_col->column->cloneEmpty();
|
||||
size_t prev_left_offset = 0;
|
||||
for (size_t i = 1; i < row_replicate_offset.size(); ++i)
|
||||
{
|
||||
const size_t & left_offset = row_replicate_offset[i];
|
||||
size_t rows = left_offset - prev_left_offset;
|
||||
if (rows)
|
||||
new_col->insertManyFrom(*src_col->column, left_start_row + i - 1, rows);
|
||||
prev_left_offset = left_offset;
|
||||
}
|
||||
executed_block.insert({std::move(new_col), src_col->type, col_name});
|
||||
}
|
||||
if (!executed_block)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"required columns: [{}], but not found any in left/right table. right table: {}, left table: {}",
|
||||
required_cols.toString(),
|
||||
sample_right_block.dumpNames(),
|
||||
added_columns.left_block.dumpNames());
|
||||
}
|
||||
|
||||
for (const auto & col : executed_block.getColumnsWithTypeAndName())
|
||||
if (!col.column || !col.type)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal nullptr column in input block: {}", executed_block.dumpStructure());
|
||||
|
||||
added_columns.additional_filter_expression->execute(executed_block);
|
||||
result_column = executed_block.getByPosition(0).column->convertToFullColumnIfConst();
|
||||
executed_block.clear();
|
||||
} while (false);
|
||||
|
||||
result_column = result_column->convertToFullIfNeeded();
|
||||
if (result_column->isNullable())
|
||||
{
|
||||
/// Convert Nullable(UInt8) to UInt8 ensuring that nulls are zeros
|
||||
/// Trying to avoid copying data, since we are the only owner of the column.
|
||||
ColumnPtr mask_column = assert_cast<const ColumnNullable &>(*result_column).getNullMapColumnPtr();
|
||||
|
||||
MutableColumnPtr mutable_column;
|
||||
{
|
||||
ColumnPtr nested_column = assert_cast<const ColumnNullable &>(*result_column).getNestedColumnPtr();
|
||||
result_column.reset();
|
||||
mutable_column = IColumn::mutate(std::move(nested_column));
|
||||
}
|
||||
|
||||
auto & column_data = assert_cast<ColumnUInt8 &>(*mutable_column).getData();
|
||||
const auto & mask_column_data = assert_cast<const ColumnUInt8 &>(*mask_column).getData();
|
||||
for (size_t i = 0; i < column_data.size(); ++i)
|
||||
{
|
||||
if (mask_column_data[i])
|
||||
column_data[i] = 0;
|
||||
}
|
||||
return mutable_column;
|
||||
}
|
||||
return result_column;
|
||||
}
|
||||
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
template <typename KeyGetter, typename Map, typename AddedColumns>
|
||||
size_t HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::joinRightColumnsWithAddtitionalFilter(
|
||||
std::vector<KeyGetter> && key_getter_vector,
|
||||
const std::vector<const Map *> & mapv,
|
||||
AddedColumns & added_columns,
|
||||
JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]],
|
||||
bool need_filter [[maybe_unused]],
|
||||
bool flag_per_row [[maybe_unused]])
|
||||
{
|
||||
constexpr JoinFeatures<KIND, STRICTNESS, MapsTemplate> join_features;
|
||||
size_t left_block_rows = added_columns.rows_to_add;
|
||||
if (need_filter)
|
||||
added_columns.filter = IColumn::Filter(left_block_rows, 0);
|
||||
|
||||
std::unique_ptr<Arena> pool;
|
||||
|
||||
if constexpr (join_features.need_replication)
|
||||
added_columns.offsets_to_replicate = std::make_unique<IColumn::Offsets>(left_block_rows);
|
||||
|
||||
std::vector<size_t> row_replicate_offset;
|
||||
row_replicate_offset.reserve(left_block_rows);
|
||||
|
||||
using FindResult = typename KeyGetter::FindResult;
|
||||
size_t max_joined_block_rows = added_columns.max_joined_block_rows;
|
||||
size_t left_row_iter = 0;
|
||||
PreSelectedRows selected_rows;
|
||||
selected_rows.reserve(left_block_rows);
|
||||
std::vector<FindResult> find_results;
|
||||
find_results.reserve(left_block_rows);
|
||||
bool exceeded_max_block_rows = false;
|
||||
IColumn::Offset total_added_rows = 0;
|
||||
IColumn::Offset current_added_rows = 0;
|
||||
|
||||
auto collect_keys_matched_rows_refs = [&]()
|
||||
{
|
||||
pool = std::make_unique<Arena>();
|
||||
find_results.clear();
|
||||
row_replicate_offset.clear();
|
||||
row_replicate_offset.push_back(0);
|
||||
current_added_rows = 0;
|
||||
selected_rows.clear();
|
||||
for (; left_row_iter < left_block_rows; ++left_row_iter)
|
||||
{
|
||||
if constexpr (join_features.need_replication)
|
||||
{
|
||||
if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows))
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
KnownRowsHolder<true> all_flag_known_rows;
|
||||
KnownRowsHolder<false> single_flag_know_rows;
|
||||
for (size_t join_clause_idx = 0; join_clause_idx < added_columns.join_on_keys.size(); ++join_clause_idx)
|
||||
{
|
||||
const auto & join_keys = added_columns.join_on_keys[join_clause_idx];
|
||||
if (join_keys.null_map && (*join_keys.null_map)[left_row_iter])
|
||||
continue;
|
||||
|
||||
bool row_acceptable = !join_keys.isRowFiltered(left_row_iter);
|
||||
auto find_result = row_acceptable
|
||||
? key_getter_vector[join_clause_idx].findKey(*(mapv[join_clause_idx]), left_row_iter, *pool)
|
||||
: FindResult();
|
||||
|
||||
if (find_result.isFound())
|
||||
{
|
||||
auto & mapped = find_result.getMapped();
|
||||
find_results.push_back(find_result);
|
||||
/// We don't add missing in addFoundRowAll here. we will add it after filter is applied.
|
||||
/// it's different from `joinRightColumns`.
|
||||
if (flag_per_row)
|
||||
addFoundRowAll<Map, false, true>(mapped, selected_rows, current_added_rows, all_flag_known_rows, nullptr);
|
||||
else
|
||||
addFoundRowAll<Map, false, false>(mapped, selected_rows, current_added_rows, single_flag_know_rows, nullptr);
|
||||
}
|
||||
}
|
||||
row_replicate_offset.push_back(current_added_rows);
|
||||
}
|
||||
};
|
||||
|
||||
auto copy_final_matched_rows = [&](size_t left_start_row, ColumnPtr filter_col)
|
||||
{
|
||||
const PaddedPODArray<UInt8> & filter_flags = assert_cast<const ColumnUInt8 &>(*filter_col).getData();
|
||||
|
||||
size_t prev_replicated_row = 0;
|
||||
auto selected_right_row_it = selected_rows.begin();
|
||||
size_t find_result_index = 0;
|
||||
for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i)
|
||||
{
|
||||
bool any_matched = false;
|
||||
/// right/full join or multiple disjuncts, we need to mark used flags for each row.
|
||||
if (flag_per_row)
|
||||
{
|
||||
for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row)
|
||||
{
|
||||
if (filter_flags[replicated_row])
|
||||
{
|
||||
if constexpr (join_features.is_semi_join || join_features.is_any_join)
|
||||
{
|
||||
/// For LEFT/INNER SEMI/ANY JOIN, we need to add only first appeared row from left,
|
||||
if constexpr (join_features.left || join_features.inner)
|
||||
{
|
||||
if (!any_matched)
|
||||
{
|
||||
// For inner join, we need mark each right row'flag, because we only use each right row once.
|
||||
auto used_once = used_flags.template setUsedOnce<join_features.need_flags, true>(
|
||||
selected_right_row_it->block, selected_right_row_it->row_num, 0);
|
||||
if (used_once)
|
||||
{
|
||||
any_matched = true;
|
||||
total_added_rows += 1;
|
||||
added_columns.appendFromBlock(
|
||||
*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing);
|
||||
}
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
auto used_once = used_flags.template setUsedOnce<join_features.need_flags, true>(
|
||||
selected_right_row_it->block, selected_right_row_it->row_num, 0);
|
||||
if (used_once)
|
||||
{
|
||||
any_matched = true;
|
||||
total_added_rows += 1;
|
||||
added_columns.appendFromBlock(
|
||||
*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing);
|
||||
}
|
||||
}
|
||||
}
|
||||
else if constexpr (join_features.is_anti_join)
|
||||
{
|
||||
any_matched = true;
|
||||
if constexpr (join_features.right && join_features.need_flags)
|
||||
used_flags.template setUsed<true, true>(selected_right_row_it->block, selected_right_row_it->row_num, 0);
|
||||
}
|
||||
else
|
||||
{
|
||||
any_matched = true;
|
||||
total_added_rows += 1;
|
||||
added_columns.appendFromBlock(
|
||||
*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing);
|
||||
used_flags.template setUsed<join_features.need_flags, true>(
|
||||
selected_right_row_it->block, selected_right_row_it->row_num, 0);
|
||||
}
|
||||
}
|
||||
|
||||
++selected_right_row_it;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row)
|
||||
{
|
||||
if constexpr (join_features.is_anti_join)
|
||||
{
|
||||
any_matched |= filter_flags[replicated_row];
|
||||
}
|
||||
else if constexpr (join_features.need_replication)
|
||||
{
|
||||
if (filter_flags[replicated_row])
|
||||
{
|
||||
any_matched = true;
|
||||
added_columns.appendFromBlock(
|
||||
*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing);
|
||||
total_added_rows += 1;
|
||||
}
|
||||
++selected_right_row_it;
|
||||
}
|
||||
else
|
||||
{
|
||||
if (filter_flags[replicated_row])
|
||||
{
|
||||
any_matched = true;
|
||||
added_columns.appendFromBlock(
|
||||
*selected_right_row_it->block, selected_right_row_it->row_num, join_features.add_missing);
|
||||
total_added_rows += 1;
|
||||
selected_right_row_it = selected_right_row_it + row_replicate_offset[i] - replicated_row;
|
||||
break;
|
||||
}
|
||||
else
|
||||
++selected_right_row_it;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
if constexpr (join_features.is_anti_join)
|
||||
{
|
||||
if (!any_matched)
|
||||
{
|
||||
if constexpr (join_features.left)
|
||||
if (need_filter)
|
||||
setUsed<true>(added_columns.filter, left_start_row + i - 1);
|
||||
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, total_added_rows);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!any_matched)
|
||||
{
|
||||
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, total_added_rows);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!flag_per_row)
|
||||
used_flags.template setUsed<join_features.need_flags, false>(find_results[find_result_index]);
|
||||
if (need_filter)
|
||||
setUsed<true>(added_columns.filter, left_start_row + i - 1);
|
||||
if constexpr (join_features.add_missing)
|
||||
added_columns.applyLazyDefaults();
|
||||
}
|
||||
}
|
||||
find_result_index += (prev_replicated_row != row_replicate_offset[i]);
|
||||
|
||||
if constexpr (join_features.need_replication)
|
||||
{
|
||||
(*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows;
|
||||
}
|
||||
prev_replicated_row = row_replicate_offset[i];
|
||||
}
|
||||
};
|
||||
|
||||
while (left_row_iter < left_block_rows && !exceeded_max_block_rows)
|
||||
{
|
||||
auto left_start_row = left_row_iter;
|
||||
collect_keys_matched_rows_refs();
|
||||
if (selected_rows.size() != current_added_rows || row_replicate_offset.size() != left_row_iter - left_start_row + 1)
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::LOGICAL_ERROR,
|
||||
"Sizes are mismatched. selected_rows.size:{}, current_added_rows:{}, row_replicate_offset.size:{}, left_row_iter: {}, "
|
||||
"left_start_row: {}",
|
||||
selected_rows.size(),
|
||||
current_added_rows,
|
||||
row_replicate_offset.size(),
|
||||
left_row_iter,
|
||||
left_start_row);
|
||||
}
|
||||
auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns);
|
||||
copy_final_matched_rows(left_start_row, filter_col);
|
||||
|
||||
if constexpr (join_features.need_replication)
|
||||
{
|
||||
// Add a check for current_added_rows to avoid run the filter expression on too small size batch.
|
||||
if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024)
|
||||
exceeded_max_block_rows = true;
|
||||
}
|
||||
}
|
||||
|
||||
if constexpr (join_features.need_replication)
|
||||
{
|
||||
added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter);
|
||||
added_columns.filter.resize_assume_reserved(left_row_iter);
|
||||
}
|
||||
added_columns.applyLazyDefaults();
|
||||
return left_row_iter;
|
||||
}
|
||||
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
Block HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::sliceBlock(Block & block, size_t num_rows)
|
||||
{
|
||||
size_t total_rows = block.rows();
|
||||
if (num_rows >= total_rows)
|
||||
return {};
|
||||
size_t remaining_rows = total_rows - num_rows;
|
||||
Block remaining_block = block.cloneEmpty();
|
||||
for (size_t i = 0; i < block.columns(); ++i)
|
||||
{
|
||||
auto & col = block.getByPosition(i);
|
||||
remaining_block.getByPosition(i).column = col.column->cut(num_rows, remaining_rows);
|
||||
col.column = col.column->cut(0, num_rows);
|
||||
}
|
||||
return remaining_block;
|
||||
}
|
||||
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
ColumnWithTypeAndName HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::copyLeftKeyColumnToRight(
|
||||
const DataTypePtr & right_key_type,
|
||||
const String & renamed_right_column,
|
||||
const ColumnWithTypeAndName & left_column,
|
||||
const IColumn::Filter * null_map_filter)
|
||||
{
|
||||
ColumnWithTypeAndName right_column = left_column;
|
||||
right_column.name = renamed_right_column;
|
||||
|
||||
if (null_map_filter)
|
||||
right_column.column = JoinCommon::filterWithBlanks(right_column.column, *null_map_filter);
|
||||
|
||||
bool should_be_nullable = isNullableOrLowCardinalityNullable(right_key_type);
|
||||
if (null_map_filter)
|
||||
correctNullabilityInplace(right_column, should_be_nullable, *null_map_filter);
|
||||
else
|
||||
correctNullabilityInplace(right_column, should_be_nullable);
|
||||
|
||||
if (!right_column.type->equals(*right_key_type))
|
||||
{
|
||||
right_column.column = castColumnAccurate(right_column, right_key_type);
|
||||
right_column.type = right_key_type;
|
||||
}
|
||||
|
||||
right_column.column = right_column.column->convertToFullColumnIfConst();
|
||||
return right_column;
|
||||
}
|
||||
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
void HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable)
|
||||
{
|
||||
if (nullable)
|
||||
{
|
||||
JoinCommon::convertColumnToNullable(column);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// We have to replace values masked by NULLs with defaults.
|
||||
if (column.column)
|
||||
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(&*column.column))
|
||||
column.column = JoinCommon::filterWithBlanks(column.column, nullable_column->getNullMapColumn().getData(), true);
|
||||
|
||||
JoinCommon::removeColumnNullability(column);
|
||||
}
|
||||
}
|
||||
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename MapsTemplate>
|
||||
void HashJoinMethods<KIND, STRICTNESS, MapsTemplate>::correctNullabilityInplace(
|
||||
ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map)
|
||||
{
|
||||
if (nullable)
|
||||
{
|
||||
JoinCommon::convertColumnToNullable(column);
|
||||
if (column.type->isNullable() && !negative_null_map.empty())
|
||||
{
|
||||
MutableColumnPtr mutable_column = IColumn::mutate(std::move(column.column));
|
||||
assert_cast<ColumnNullable &>(*mutable_column).applyNegatedNullMap(negative_null_map);
|
||||
column.column = std::move(mutable_column);
|
||||
}
|
||||
}
|
||||
else
|
||||
JoinCommon::removeColumnNullability(column);
|
||||
}
|
||||
}
|
@ -1,10 +1,11 @@
|
||||
|
||||
#include <Interpreters/HashJoin/HashJoinMethods.h>
|
||||
#include <Interpreters/HashJoin/HashJoinMethodsImpl.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
template class HashJoinMethods<JoinKind::Inner, JoinStrictness::RightAny, HashJoin::MapsOne>;
|
||||
template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Any, HashJoin::MapsOne>;
|
||||
template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Any, HashJoin::MapsAll>;
|
||||
template class HashJoinMethods<JoinKind::Inner, JoinStrictness::All, HashJoin::MapsAll>;
|
||||
template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Semi, HashJoin::MapsOne>;
|
||||
template class HashJoinMethods<JoinKind::Inner, JoinStrictness::Anti, HashJoin::MapsOne>;
|
||||
|
@ -3,15 +3,15 @@
|
||||
#include <Interpreters/joinDispatch.h>
|
||||
namespace DB
|
||||
{
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS>
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, typename Map>
|
||||
struct JoinFeatures
|
||||
{
|
||||
static constexpr bool is_any_join = STRICTNESS == JoinStrictness::Any;
|
||||
static constexpr bool is_any_or_semi_join = STRICTNESS == JoinStrictness::Any || STRICTNESS == JoinStrictness::RightAny || (STRICTNESS == JoinStrictness::Semi && KIND == JoinKind::Left);
|
||||
static constexpr bool is_all_join = STRICTNESS == JoinStrictness::All;
|
||||
static constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof;
|
||||
static constexpr bool is_semi_join = STRICTNESS == JoinStrictness::Semi;
|
||||
static constexpr bool is_anti_join = STRICTNESS == JoinStrictness::Anti;
|
||||
static constexpr bool is_any_or_semi_join = is_any_join || STRICTNESS == JoinStrictness::RightAny || (is_semi_join && KIND == JoinKind::Left);
|
||||
|
||||
static constexpr bool left = KIND == JoinKind::Left;
|
||||
static constexpr bool right = KIND == JoinKind::Right;
|
||||
@ -22,7 +22,8 @@ struct JoinFeatures
|
||||
static constexpr bool need_filter = !need_replication && (inner || right || (is_semi_join && left) || (is_anti_join && left));
|
||||
static constexpr bool add_missing = (left || full) && !is_semi_join;
|
||||
|
||||
static constexpr bool need_flags = MapGetter<KIND, STRICTNESS>::flagged;
|
||||
static constexpr bool need_flags = MapGetter<KIND, STRICTNESS, std::is_same_v<std::decay_t<Map>, HashJoin::MapsAll>>::flagged;
|
||||
static constexpr bool is_maps_all = std::is_same_v<std::decay_t<Map>, HashJoin::MapsAll>;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -26,10 +26,10 @@ public:
|
||||
/// Update size for vector with flags.
|
||||
/// Calling this method invalidates existing flags.
|
||||
/// It can be called several times, but all of them should happen before using this structure.
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS>
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, bool prefer_use_maps_all>
|
||||
void reinit(size_t size)
|
||||
{
|
||||
if constexpr (MapGetter<KIND, STRICTNESS>::flagged)
|
||||
if constexpr (MapGetter<KIND, STRICTNESS, prefer_use_maps_all>::flagged)
|
||||
{
|
||||
assert(flags[nullptr].size() <= size);
|
||||
need_flags = true;
|
||||
@ -43,10 +43,10 @@ public:
|
||||
}
|
||||
}
|
||||
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS>
|
||||
template <JoinKind KIND, JoinStrictness STRICTNESS, bool prefer_use_maps_all>
|
||||
void reinit(const Block * block_ptr)
|
||||
{
|
||||
if constexpr (MapGetter<KIND, STRICTNESS>::flagged)
|
||||
if constexpr (MapGetter<KIND, STRICTNESS, prefer_use_maps_all>::flagged)
|
||||
{
|
||||
assert(flags[block_ptr].size() <= block_ptr->rows());
|
||||
need_flags = true;
|
||||
@ -148,6 +148,31 @@ public:
|
||||
}
|
||||
|
||||
}
|
||||
template <bool use_flags, bool flag_per_row>
|
||||
bool setUsedOnce(const Block * block, size_t row_num, size_t offset)
|
||||
{
|
||||
if constexpr (!use_flags)
|
||||
return true;
|
||||
|
||||
if constexpr (flag_per_row)
|
||||
{
|
||||
/// fast check to prevent heavy CAS with seq_cst order
|
||||
if (flags[block][row_num].load(std::memory_order_relaxed))
|
||||
return false;
|
||||
|
||||
bool expected = false;
|
||||
return flags[block][row_num].compare_exchange_strong(expected, true);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// fast check to prevent heavy CAS with seq_cst order
|
||||
if (flags[nullptr][offset].load(std::memory_order_relaxed))
|
||||
return false;
|
||||
|
||||
bool expected = false;
|
||||
return flags[nullptr][offset].compare_exchange_strong(expected, true);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -1,11 +1,14 @@
|
||||
#include <Interpreters/HashJoin/HashJoinMethods.h>
|
||||
#include <Interpreters/HashJoin/HashJoinMethodsImpl.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
template class HashJoinMethods<JoinKind::Left, JoinStrictness::RightAny, HashJoin::MapsOne>;
|
||||
template class HashJoinMethods<JoinKind::Left, JoinStrictness::Any, HashJoin::MapsOne>;
|
||||
template class HashJoinMethods<JoinKind::Left, JoinStrictness::Any, HashJoin::MapsAll>;
|
||||
template class HashJoinMethods<JoinKind::Left, JoinStrictness::All, HashJoin::MapsAll>;
|
||||
template class HashJoinMethods<JoinKind::Left, JoinStrictness::Semi, HashJoin::MapsOne>;
|
||||
template class HashJoinMethods<JoinKind::Left, JoinStrictness::Semi, HashJoin::MapsAll>;
|
||||
template class HashJoinMethods<JoinKind::Left, JoinStrictness::Anti, HashJoin::MapsOne>;
|
||||
template class HashJoinMethods<JoinKind::Left, JoinStrictness::Anti, HashJoin::MapsAll>;
|
||||
template class HashJoinMethods<JoinKind::Left, JoinStrictness::Asof, HashJoin::MapsAsof>;
|
||||
}
|
||||
|
@ -1,4 +1,4 @@
|
||||
#include <Interpreters/HashJoin/HashJoinMethods.h>
|
||||
#include <Interpreters/HashJoin/HashJoinMethodsImpl.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -12,38 +12,53 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <JoinKind kind, JoinStrictness join_strictness>
|
||||
/// 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 <JoinKind kind, JoinStrictness join_strictness, bool prefer_use_maps_all>
|
||||
struct MapGetter;
|
||||
|
||||
template <> struct MapGetter<JoinKind::Left, JoinStrictness::RightAny> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <> struct MapGetter<JoinKind::Inner, JoinStrictness::RightAny> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <> struct MapGetter<JoinKind::Right, JoinStrictness::RightAny> { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; };
|
||||
template <> struct MapGetter<JoinKind::Full, JoinStrictness::RightAny> { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Left, JoinStrictness::RightAny, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Inner, JoinStrictness::RightAny, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Right, JoinStrictness::RightAny, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Full, JoinStrictness::RightAny, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; };
|
||||
|
||||
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Any> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <> struct MapGetter<JoinKind::Inner, JoinStrictness::Any> { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; };
|
||||
template <> struct MapGetter<JoinKind::Right, JoinStrictness::Any> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
|
||||
template <> struct MapGetter<JoinKind::Full, JoinStrictness::Any> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
|
||||
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Any, false> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Any, true> { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; };
|
||||
template <> struct MapGetter<JoinKind::Inner, JoinStrictness::Any, true> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
|
||||
template <> struct MapGetter<JoinKind::Inner, JoinStrictness::Any, false> { using Map = HashJoin::MapsOne; static constexpr bool flagged = true; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Right, JoinStrictness::Any, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Full, JoinStrictness::Any, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
|
||||
|
||||
template <> struct MapGetter<JoinKind::Left, JoinStrictness::All> { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; };
|
||||
template <> struct MapGetter<JoinKind::Inner, JoinStrictness::All> { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; };
|
||||
template <> struct MapGetter<JoinKind::Right, JoinStrictness::All> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
|
||||
template <> struct MapGetter<JoinKind::Full, JoinStrictness::All> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Left, JoinStrictness::All, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Inner, JoinStrictness::All, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Right, JoinStrictness::All, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Full, JoinStrictness::All, prefer_use_maps_all> { 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<JoinKind::Left, JoinStrictness::Semi> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <> struct MapGetter<JoinKind::Inner, JoinStrictness::Semi> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <> struct MapGetter<JoinKind::Right, JoinStrictness::Semi> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
|
||||
template <> struct MapGetter<JoinKind::Full, JoinStrictness::Semi> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Semi, false> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Semi, true> { using Map = HashJoin::MapsAll; static constexpr bool flagged = false; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Inner, JoinStrictness::Semi, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Right, JoinStrictness::Semi, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Full, JoinStrictness::Semi, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
|
||||
/// Only SEMI LEFT and SEMI RIGHT are valid. INNER and FULL are here for templates instantiation.
|
||||
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Anti> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <> struct MapGetter<JoinKind::Inner, JoinStrictness::Anti> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <> struct MapGetter<JoinKind::Right, JoinStrictness::Anti> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
|
||||
template <> struct MapGetter<JoinKind::Full, JoinStrictness::Anti> { 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<JoinKind::Left, JoinStrictness::Anti, false> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <> struct MapGetter<JoinKind::Left, JoinStrictness::Anti, true> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Inner, JoinStrictness::Anti, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Right, JoinStrictness::Anti, prefer_use_maps_all> { using Map = HashJoin::MapsAll; static constexpr bool flagged = true; };
|
||||
template <bool prefer_use_maps_all> struct MapGetter<JoinKind::Full, JoinStrictness::Anti, prefer_use_maps_all> { using Map = HashJoin::MapsOne; static constexpr bool flagged = false; };
|
||||
|
||||
template <JoinKind kind>
|
||||
struct MapGetter<kind, JoinStrictness::Asof> { using Map = HashJoin::MapsAsof; static constexpr bool flagged = false; };
|
||||
template <JoinKind kind, bool prefer_use_maps_all>
|
||||
struct MapGetter<kind, JoinStrictness::Asof, prefer_use_maps_all> { using Map = HashJoin::MapsAsof; static constexpr bool flagged = false; };
|
||||
|
||||
static constexpr std::array<JoinStrictness, 6> STRICTNESSES = {
|
||||
JoinStrictness::RightAny,
|
||||
@ -62,7 +77,7 @@ static constexpr std::array<JoinKind, 4> KINDS = {
|
||||
};
|
||||
|
||||
/// Init specified join map
|
||||
inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin::MapsVariant & maps)
|
||||
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)
|
||||
{
|
||||
@ -70,7 +85,10 @@ inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin:
|
||||
constexpr auto j = ij % STRICTNESSES.size();
|
||||
if (kind == KINDS[i] && strictness == STRICTNESSES[j])
|
||||
{
|
||||
maps = typename MapGetter<KINDS[i], STRICTNESSES[j]>::Map();
|
||||
if (prefer_use_maps_all)
|
||||
maps = typename MapGetter<KINDS[i], STRICTNESSES[j], true>::Map();
|
||||
else
|
||||
maps = typename MapGetter<KINDS[i], STRICTNESSES[j], false>::Map();
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
@ -79,7 +97,7 @@ inline bool joinDispatchInit(JoinKind kind, JoinStrictness strictness, HashJoin:
|
||||
|
||||
/// Call function on specified join map
|
||||
template <typename MapsVariant, typename Func>
|
||||
inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant & maps, Func && func)
|
||||
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)
|
||||
{
|
||||
@ -89,10 +107,16 @@ inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant &
|
||||
constexpr auto j = ij % STRICTNESSES.size();
|
||||
if (kind == KINDS[i] && strictness == STRICTNESSES[j])
|
||||
{
|
||||
func(
|
||||
std::integral_constant<JoinKind, KINDS[i]>(),
|
||||
std::integral_constant<JoinStrictness, STRICTNESSES[j]>(),
|
||||
std::get<typename MapGetter<KINDS[i], STRICTNESSES[j]>::Map>(maps));
|
||||
if (prefer_use_maps_all)
|
||||
func(
|
||||
std::integral_constant<JoinKind, KINDS[i]>(),
|
||||
std::integral_constant<JoinStrictness, STRICTNESSES[j]>(),
|
||||
std::get<typename MapGetter<KINDS[i], STRICTNESSES[j], true>::Map>(maps));
|
||||
else
|
||||
func(
|
||||
std::integral_constant<JoinKind, KINDS[i]>(),
|
||||
std::integral_constant<JoinStrictness, STRICTNESSES[j]>(),
|
||||
std::get<typename MapGetter<KINDS[i], STRICTNESSES[j], false>::Map>(maps));
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
@ -101,7 +125,7 @@ inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, MapsVariant &
|
||||
|
||||
/// Call function on specified join map
|
||||
template <typename MapsVariant, typename Func>
|
||||
inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, std::vector<const MapsVariant *> & mapsv, Func && func)
|
||||
inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, std::vector<const MapsVariant *> & mapsv, bool prefer_use_maps_all, Func && func)
|
||||
{
|
||||
return static_for<0, KINDS.size() * STRICTNESSES.size()>([&](auto ij)
|
||||
{
|
||||
@ -111,17 +135,31 @@ inline bool joinDispatch(JoinKind kind, JoinStrictness strictness, std::vector<c
|
||||
constexpr auto j = ij % STRICTNESSES.size();
|
||||
if (kind == KINDS[i] && strictness == STRICTNESSES[j])
|
||||
{
|
||||
using MapType = typename MapGetter<KINDS[i], STRICTNESSES[j]>::Map;
|
||||
std::vector<const MapType *> v;
|
||||
v.reserve(mapsv.size());
|
||||
for (const auto & el : mapsv)
|
||||
v.push_back(&std::get<MapType>(*el));
|
||||
if (prefer_use_maps_all)
|
||||
{
|
||||
using MapType = typename MapGetter<KINDS[i], STRICTNESSES[j], true>::Map;
|
||||
std::vector<const MapType *> v;
|
||||
v.reserve(mapsv.size());
|
||||
for (const auto & el : mapsv)
|
||||
v.push_back(&std::get<MapType>(*el));
|
||||
|
||||
func(
|
||||
std::integral_constant<JoinKind, KINDS[i]>(),
|
||||
std::integral_constant<JoinStrictness, STRICTNESSES[j]>(),
|
||||
v
|
||||
/*std::get<typename MapGetter<KINDS[i], STRICTNESSES[j]>::Map>(maps)*/);
|
||||
func(
|
||||
std::integral_constant<JoinKind, KINDS[i]>(), std::integral_constant<JoinStrictness, STRICTNESSES[j]>(), v
|
||||
/*std::get<typename MapGetter<KINDS[i], STRICTNESSES[j]>::Map>(maps)*/);
|
||||
}
|
||||
else
|
||||
{
|
||||
using MapType = typename MapGetter<KINDS[i], STRICTNESSES[j], false>::Map;
|
||||
std::vector<const MapType *> v;
|
||||
v.reserve(mapsv.size());
|
||||
for (const auto & el : mapsv)
|
||||
v.push_back(&std::get<MapType>(*el));
|
||||
|
||||
func(
|
||||
std::integral_constant<JoinKind, KINDS[i]>(), std::integral_constant<JoinStrictness, STRICTNESSES[j]>(), v
|
||||
/*std::get<typename MapGetter<KINDS[i], STRICTNESSES[j]>::Map>(maps)*/);
|
||||
|
||||
}
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
|
@ -546,7 +546,11 @@ protected:
|
||||
return {};
|
||||
|
||||
Chunk chunk;
|
||||
if (!joinDispatch(join->kind, join->strictness, join->data->maps.front(),
|
||||
if (!joinDispatch(
|
||||
join->kind,
|
||||
join->strictness,
|
||||
join->data->maps.front(),
|
||||
join->table_join->getMixedJoinExpression() != nullptr,
|
||||
[&](auto kind, auto strictness, auto & map) { chunk = createChunk<kind, strictness>(map); }))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness");
|
||||
return chunk;
|
||||
|
@ -382,6 +382,253 @@ key1 e 5 5 5 key1 C 3 4 5
|
||||
key2 a2 1 1 1 0 0 \N
|
||||
key4 f 2 3 4 key4 F 1 1 1
|
||||
SET join_algorithm='hash';
|
||||
SELECT t1.*, t2.* FROM t1 LEFT ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 c 3 2 1 key1 C 3 4 5
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
key1 e 5 5 5 0 0 \N
|
||||
key2 a2 1 1 1 0 0 \N
|
||||
key4 f 2 3 4 key4 F 1 1 1
|
||||
SELECT t1.*, t2.* from t1 LEFT ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 c 3 2 1 key1 B 2 1 2
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
key1 e 5 5 5 0 0 \N
|
||||
key2 a2 1 1 1 0 0 \N
|
||||
key4 f 2 3 4 0 0 \N
|
||||
SELECT t1.*, t2.* from t1 LEFT ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
key1 b 2 3 2 key1 C 3 4 5
|
||||
key1 c 3 2 1 key1 D 4 1 6
|
||||
key1 d 4 7 2 0 0 \N
|
||||
key1 e 5 5 5 0 0 \N
|
||||
key2 a2 1 1 1 0 0 \N
|
||||
key4 f 2 3 4 0 0 \N
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
1 1 1 1 1 1
|
||||
SELECT t1.*, t2.* FROM t1 LEFT SEMI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 c 3 2 1 key1 C 3 4 5
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
key4 f 2 3 4 key4 F 1 1 1
|
||||
SELECT t1.*, t2.* from t1 LEFT SEMI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 c 3 2 1 key1 B 2 1 2
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
SELECT t1.*, t2.* from t1 LEFT SEMI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
key1 b 2 3 2 key1 C 3 4 5
|
||||
key1 c 3 2 1 key1 D 4 1 6
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT SEMI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
1 1 1 1 1 1
|
||||
SELECT t1.*, t2.* FROM t1 LEFT ANTI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 e 5 5 5 key1 0 0 \N
|
||||
key2 a2 1 1 1 key2 0 0 \N
|
||||
SELECT t1.*, t2.* from t1 LEFT ANTI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 e 5 5 5 key1 0 0 \N
|
||||
key2 a2 1 1 1 key2 0 0 \N
|
||||
key4 f 2 3 4 key4 0 0 \N
|
||||
SELECT t1.*, t2.* from t1 LEFT ANTI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 d 4 7 2 key1 0 0 \N
|
||||
key1 e 5 5 5 key1 0 0 \N
|
||||
key2 a2 1 1 1 key2 0 0 \N
|
||||
key4 f 2 3 4 key4 0 0 \N
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT ANTI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
SELECT t1.*, t2.* FROM t1 RIGHT ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
0 0 \N key3 a3 1 1 1
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
key1 a 1 1 2 key1 C 3 4 5
|
||||
key1 a 1 1 2 key1 D 4 1 6
|
||||
key4 f 2 3 4 key4 F 1 1 1
|
||||
SELECT t1.*, t2.* from t1 RIGHT ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
0 0 \N key3 a3 1 1 1
|
||||
0 0 \N key4 F 1 1 1
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 b 2 3 2 key1 C 3 4 5
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
SELECT t1.*, t2.* from t1 RIGHT ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
0 0 \N key1 A 1 2 1
|
||||
0 0 \N key3 a3 1 1 1
|
||||
0 0 \N key4 F 1 1 1
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
key1 a 1 1 2 key1 C 3 4 5
|
||||
key1 a 1 1 2 key1 D 4 1 6
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
1 1 1 1 1 1
|
||||
SELECT t1.*, t2.* FROM t1 RIGHT SEMI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
key1 a 1 1 2 key1 C 3 4 5
|
||||
key1 a 1 1 2 key1 D 4 1 6
|
||||
key4 f 2 3 4 key4 F 1 1 1
|
||||
SELECT t1.*, t2.* from t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 b 2 3 2 key1 C 3 4 5
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
SELECT t1.*, t2.* from t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
key1 a 1 1 2 key1 C 3 4 5
|
||||
key1 a 1 1 2 key1 D 4 1 6
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT SEMI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
1 1 1 1 1 1
|
||||
SELECT t1.*, t2.* FROM t1 RIGHT ANTI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
0 0 \N key3 a3 1 1 1
|
||||
SELECT t1.*, t2.* from t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
0 0 \N key3 a3 1 1 1
|
||||
0 0 \N key4 F 1 1 1
|
||||
SELECT t1.*, t2.* from t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
0 0 \N key1 A 1 2 1
|
||||
0 0 \N key3 a3 1 1 1
|
||||
0 0 \N key4 F 1 1 1
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT ANTI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
SET join_algorithm='grace_hash';
|
||||
SELECT t1.*, t2.* FROM t1 LEFT ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 c 3 2 1 key1 C 3 4 5
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
key1 e 5 5 5 0 0 \N
|
||||
key2 a2 1 1 1 0 0 \N
|
||||
key4 f 2 3 4 key4 F 1 1 1
|
||||
SELECT t1.*, t2.* from t1 LEFT ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 c 3 2 1 key1 B 2 1 2
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
key1 e 5 5 5 0 0 \N
|
||||
key2 a2 1 1 1 0 0 \N
|
||||
key4 f 2 3 4 0 0 \N
|
||||
SELECT t1.*, t2.* from t1 LEFT ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
key1 b 2 3 2 key1 C 3 4 5
|
||||
key1 c 3 2 1 key1 D 4 1 6
|
||||
key1 d 4 7 2 0 0 \N
|
||||
key1 e 5 5 5 0 0 \N
|
||||
key2 a2 1 1 1 0 0 \N
|
||||
key4 f 2 3 4 0 0 \N
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
1 1 1 1 1 1
|
||||
SELECT t1.*, t2.* FROM t1 LEFT SEMI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 c 3 2 1 key1 C 3 4 5
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
key4 f 2 3 4 key4 F 1 1 1
|
||||
SELECT t1.*, t2.* from t1 LEFT SEMI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 c 3 2 1 key1 B 2 1 2
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
SELECT t1.*, t2.* from t1 LEFT SEMI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
key1 b 2 3 2 key1 C 3 4 5
|
||||
key1 c 3 2 1 key1 D 4 1 6
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT SEMI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
1 1 1 1 1 1
|
||||
SELECT t1.*, t2.* FROM t1 LEFT ANTI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 e 5 5 5 key1 0 0 \N
|
||||
key2 a2 1 1 1 key2 0 0 \N
|
||||
SELECT t1.*, t2.* from t1 LEFT ANTI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 e 5 5 5 key1 0 0 \N
|
||||
key2 a2 1 1 1 key2 0 0 \N
|
||||
key4 f 2 3 4 key4 0 0 \N
|
||||
SELECT t1.*, t2.* from t1 LEFT ANTI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 d 4 7 2 key1 0 0 \N
|
||||
key1 e 5 5 5 key1 0 0 \N
|
||||
key2 a2 1 1 1 key2 0 0 \N
|
||||
key4 f 2 3 4 key4 0 0 \N
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 LEFT ANTI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
SELECT t1.*, t2.* FROM t1 RIGHT ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
0 0 \N key3 a3 1 1 1
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
key1 a 1 1 2 key1 C 3 4 5
|
||||
key1 a 1 1 2 key1 D 4 1 6
|
||||
key4 f 2 3 4 key4 F 1 1 1
|
||||
SELECT t1.*, t2.* from t1 RIGHT ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
0 0 \N key3 a3 1 1 1
|
||||
0 0 \N key4 F 1 1 1
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 b 2 3 2 key1 C 3 4 5
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
SELECT t1.*, t2.* from t1 RIGHT ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
0 0 \N key1 A 1 2 1
|
||||
0 0 \N key3 a3 1 1 1
|
||||
0 0 \N key4 F 1 1 1
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
key1 a 1 1 2 key1 C 3 4 5
|
||||
key1 a 1 1 2 key1 D 4 1 6
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
1 1 1 1 1 1
|
||||
SELECT t1.*, t2.* FROM t1 RIGHT SEMI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
key1 a 1 1 2 key1 C 3 4 5
|
||||
key1 a 1 1 2 key1 D 4 1 6
|
||||
key4 f 2 3 4 key4 F 1 1 1
|
||||
SELECT t1.*, t2.* from t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 b 2 3 2 key1 C 3 4 5
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
SELECT t1.*, t2.* from t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
key1 a 1 1 2 key1 C 3 4 5
|
||||
key1 a 1 1 2 key1 D 4 1 6
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT SEMI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
1 1 1 1 1 1
|
||||
SELECT t1.*, t2.* FROM t1 RIGHT ANTI JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
0 0 \N key3 a3 1 1 1
|
||||
SELECT t1.*, t2.* from t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
0 0 \N key3 a3 1 1 1
|
||||
0 0 \N key4 F 1 1 1
|
||||
SELECT t1.*, t2.* from t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
0 0 \N key1 A 1 2 1
|
||||
0 0 \N key3 a3 1 1 1
|
||||
0 0 \N key4 F 1 1 1
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 RIGHT ANTI JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
SET join_algorithm='hash';
|
||||
SELECT t1.* FROM t1 LEFT ANY JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL;
|
||||
key1 a 1 1 2
|
||||
key1 b 2 3 2
|
||||
key1 c 3 2 1
|
||||
key1 d 4 7 2
|
||||
key1 e 5 5 5
|
||||
key2 a2 1 1 1
|
||||
key4 f 2 3 4
|
||||
SELECT t1.* FROM t1 LEFT SEMI JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL;
|
||||
key1 a 1 1 2
|
||||
key1 b 2 3 2
|
||||
key1 c 3 2 1
|
||||
key1 d 4 7 2
|
||||
key2 a2 1 1 1
|
||||
key4 f 2 3 4
|
||||
SELECT t1.* FROM t1 LEFT ANTI JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL;
|
||||
key1 e 5 5 5
|
||||
SELECT t1.* FROM t1 RIGHT ANY JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL;
|
||||
key1 a 1 1 2
|
||||
key1 a 1 1 2
|
||||
key1 a 1 1 2
|
||||
key1 a 1 1 2
|
||||
key1 a 1 1 2
|
||||
key1 a 1 1 2
|
||||
SELECT t1.* FROM t1 RIGHT SEMI JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL;
|
||||
key1 a 1 1 2
|
||||
key1 a 1 1 2
|
||||
key1 a 1 1 2
|
||||
key1 a 1 1 2
|
||||
key1 a 1 1 2
|
||||
key1 a 1 1 2
|
||||
SELECT t1.* FROM t1 RIGHT ANTI JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL;
|
||||
SET join_algorithm='hash';
|
||||
SELECT t1.*, t2.* FROM t1 LEFT JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
@ -452,3 +699,46 @@ key2 a2 1 1 1 key1 A 1 2 1
|
||||
key2 a2 1 1 1 key3 a3 1 1 1
|
||||
key2 a2 1 1 1 key4 F 1 1 1
|
||||
key4 f 2 3 4 key1 B 2 1 2
|
||||
SET join_algorithm='hash';
|
||||
SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 c 3 2 1 key1 C 3 4 5
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
key4 f 2 3 4 key4 F 1 1 1
|
||||
SELECT t1.*, t2.* from t1 INNER ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 c 3 2 1 key1 B 2 1 2
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
SELECT t1.*, t2.* from t1 INNER ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
key1 b 2 3 2 key1 C 3 4 5
|
||||
key1 c 3 2 1 key1 D 4 1 6
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
1 1 1 1 1 1
|
||||
SET join_algorithm='grace_hash';
|
||||
SELECT t1.*, t2.* FROM t1 INNER ANY JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 c 3 2 1 key1 C 3 4 5
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
key4 f 2 3 4 key4 F 1 1 1
|
||||
SELECT t1.*, t2.* from t1 INNER ANY JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 A 1 2 1
|
||||
key1 b 2 3 2 key1 B 2 1 2
|
||||
key1 c 3 2 1 key1 B 2 1 2
|
||||
key1 d 4 7 2 key1 D 4 1 6
|
||||
SELECT t1.*, t2.* from t1 INNER ANY JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
key1 a 1 1 2 key1 B 2 1 2
|
||||
key1 b 2 3 2 key1 C 3 4 5
|
||||
key1 c 3 2 1 key1 D 4 1 6
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 INNER ANY JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
1 1 1 1 1 1
|
||||
SET join_algorithm='hash';
|
||||
SELECT t1.* FROM t1 INNER ANY JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL;
|
||||
key1 a 1 1 2
|
||||
key1 b 2 3 2
|
||||
key1 c 3 2 1
|
||||
key1 d 4 7 2
|
||||
key2 a2 1 1 1
|
||||
|
@ -22,6 +22,26 @@ SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND (t1.a=2
|
||||
{% endfor -%}
|
||||
{% endfor -%}
|
||||
|
||||
{% for algorithm in ['hash', 'grace_hash'] -%}
|
||||
SET join_algorithm='{{ algorithm }}';
|
||||
{% for join_type in ['LEFT', 'RIGHT'] -%}
|
||||
{% for join_strictness in ['ANY', 'SEMI', 'ANTI'] -%}
|
||||
SELECT t1.*, t2.* FROM t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
SELECT t1.*, t2.* from t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
SELECT t1.*, t2.* from t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 {{ join_type }} {{ join_strictness }} JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
{% endfor -%}
|
||||
{% endfor -%}
|
||||
{% endfor -%}
|
||||
|
||||
{% for algorithm in ['hash'] -%}
|
||||
SET join_algorithm='{{ algorithm }}';
|
||||
{% for join_type in ['LEFT', 'RIGHT'] -%}
|
||||
{% for join_strictness in ['ANY', 'SEMI', 'ANTI'] -%}
|
||||
SELECT t1.* FROM t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL;
|
||||
{% endfor -%}
|
||||
{% endfor -%}
|
||||
{% endfor -%}
|
||||
|
||||
{% for algorithm in ['hash'] -%}
|
||||
SET join_algorithm='{{ algorithm }}';
|
||||
@ -29,6 +49,28 @@ SET join_algorithm='{{ algorithm }}';
|
||||
SELECT t1.*, t2.* FROM t1 {{ join_type }} JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
{% endfor -%}
|
||||
{% endfor -%}
|
||||
|
||||
{% for algorithm in ['hash', 'grace_hash'] -%}
|
||||
SET join_algorithm='{{ algorithm }}';
|
||||
{% for join_type in ['INNER'] -%}
|
||||
{% for join_strictness in ['ANY'] -%}
|
||||
SELECT t1.*, t2.* FROM t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON (t1.a < t2.a OR lower(t1.attr) == lower(t2.attr)) AND t1.key = t2.key ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
SELECT t1.*, t2.* from t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key and (t1.b + t2.b == t1.c + t2.c) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
SELECT t1.*, t2.* from t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key and (t1.a < t2.a) ORDER BY (t1.key, t1.attr, t2.key, t2.attr);
|
||||
SELECT * FROM (SELECT 1 AS a, 1 AS b, 1 AS c) AS t1 {{ join_type }} {{ join_strictness }} JOIN (SELECT 1 AS a, 1 AS b, 1 AS c) AS t2 ON t1.a = t2.a AND (t1.b > 0 OR t2.b > 0);
|
||||
{% endfor -%}
|
||||
{% endfor -%}
|
||||
{% endfor -%}
|
||||
|
||||
{% for algorithm in ['hash'] -%}
|
||||
SET join_algorithm='{{ algorithm }}';
|
||||
{% for join_type in ['INNER'] -%}
|
||||
{% for join_strictness in ['ANY'] -%}
|
||||
SELECT t1.* FROM t1 {{ join_type }} {{ join_strictness }} JOIN t2 ON t1.key = t2.key AND t1.a < t2.a OR t1.a = t2.a ORDER BY ALL;
|
||||
{% endfor -%}
|
||||
{% endfor -%}
|
||||
{% endfor -%}
|
||||
|
||||
-- { echoOff }
|
||||
|
||||
-- test error messages
|
||||
|
Loading…
Reference in New Issue
Block a user