ClickHouse/src/Interpreters/HashJoin.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

2206 lines
83 KiB
C++
Raw Normal View History

2019-07-04 12:12:39 +00:00
#include <any>
2020-04-21 19:01:34 +00:00
#include <limits>
2021-09-06 10:59:18 +00:00
#include <unordered_map>
#include <vector>
2019-07-04 12:12:39 +00:00
2022-10-04 08:20:13 +00:00
#include <Common/StackTrace.h>
2022-04-27 15:05:45 +00:00
#include <Common/logger_useful.h>
ColumnConst unification (#1011) * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * Fixed error in ColumnArray::replicateGeneric [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150]. * ColumnConst: unification (incomplete) [#CLICKHOUSE-3150].
2017-07-21 06:35:58 +00:00
#include <Columns/ColumnConst.h>
#include <Columns/ColumnString.h>
2020-04-08 18:59:52 +00:00
#include <Columns/ColumnVector.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypeNullable.h>
2020-04-08 18:59:52 +00:00
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeTuple.h>
2014-06-12 02:31:30 +00:00
#include <Interpreters/HashJoin.h>
2022-08-04 15:20:19 +00:00
#include <Interpreters/JoinUtils.h>
#include <Interpreters/TableJoin.h>
#include <Interpreters/joinDispatch.h>
#include <Interpreters/NullableUtils.h>
2022-01-21 05:36:36 +00:00
#include <Storages/IStorage.h>
#include <Core/ColumnNumbers.h>
2022-01-21 05:36:36 +00:00
#include <Common/Exception.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
#include <Common/assert_cast.h>
2021-11-08 12:44:13 +00:00
#include <Functions/FunctionHelpers.h>
#include <Interpreters/castColumn.h>
2014-06-12 02:31:30 +00:00
namespace DB
{
2016-01-12 02:21:15 +00:00
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int NOT_IMPLEMENTED;
2020-06-02 20:26:41 +00:00
extern const int NO_SUCH_COLUMN_IN_TABLE;
extern const int INCOMPATIBLE_TYPE_OF_JOIN;
2019-08-05 14:03:14 +00:00
extern const int UNSUPPORTED_JOIN_KEYS;
2016-01-12 02:21:15 +00:00
extern const int LOGICAL_ERROR;
extern const int SYNTAX_ERROR;
2016-01-12 02:21:15 +00:00
extern const int SET_SIZE_LIMIT_EXCEEDED;
extern const int TYPE_MISMATCH;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
2016-01-12 02:21:15 +00:00
}
2020-04-06 13:39:57 +00:00
namespace
{
struct NotProcessedCrossJoin : public ExtraBlock
{
size_t left_position;
size_t right_block;
};
Int64 getCurrentQueryMemoryUsage()
{
/// Use query-level memory tracker
if (auto * memory_tracker_child = CurrentThread::getMemoryTracker())
if (auto * memory_tracker = memory_tracker_child->getParent())
return memory_tracker->get();
return 0;
}
2020-04-06 13:39:57 +00:00
}
namespace JoinStuff
{
/// for single disjunct
bool JoinUsedFlags::getUsedSafe(size_t i) const
{
return getUsedSafe(nullptr, i);
}
/// for multiple disjuncts
bool JoinUsedFlags::getUsedSafe(const Block * block_ptr, size_t row_idx) const
{
if (auto it = flags.find(block_ptr); it != flags.end())
return it->second[row_idx].load();
return !need_flags;
}
/// for single disjunct
2022-07-29 16:30:50 +00:00
template <JoinKind KIND, JoinStrictness STRICTNESS>
void JoinUsedFlags::reinit(size_t size)
{
if constexpr (MapGetter<KIND, STRICTNESS>::flagged)
{
assert(flags[nullptr].size() <= size);
need_flags = true;
2023-07-05 17:03:18 +00:00
// For one disjunct clause case, we don't need to reinit each time we call addBlockToJoin.
// and there is no value inserted in this JoinUsedFlags before addBlockToJoin finish.
// So we reinit only when the hash table is rehashed to a larger size.
if (flags.empty() || flags[nullptr].size() < size) [[unlikely]]
{
flags[nullptr] = std::vector<std::atomic_bool>(size);
}
}
}
/// for multiple disjuncts
2022-07-29 16:30:50 +00:00
template <JoinKind KIND, JoinStrictness STRICTNESS>
void JoinUsedFlags::reinit(const Block * block_ptr)
{
if constexpr (MapGetter<KIND, STRICTNESS>::flagged)
{
assert(flags[block_ptr].size() <= block_ptr->rows());
need_flags = true;
flags[block_ptr] = std::vector<std::atomic_bool>(block_ptr->rows());
}
}
2021-06-25 12:03:10 +00:00
template <bool use_flags, bool multiple_disjuncts, typename FindResult>
void JoinUsedFlags::setUsed(const FindResult & f)
{
if constexpr (!use_flags)
return;
/// Could be set simultaneously from different threads.
if constexpr (multiple_disjuncts)
2021-06-25 12:03:10 +00:00
{
auto & mapped = f.getMapped();
flags[mapped.block][mapped.row_num].store(true, std::memory_order_relaxed);
}
else
{
flags[nullptr][f.getOffset()].store(true, std::memory_order_relaxed);
2021-06-25 12:03:10 +00:00
}
}
2022-02-19 20:01:47 +00:00
template <bool use_flags, bool multiple_disjuncts>
void JoinUsedFlags::setUsed(const Block * block, size_t row_num, size_t offset)
{
if constexpr (!use_flags)
return;
/// Could be set simultaneously from different threads.
if constexpr (multiple_disjuncts)
{
flags[block][row_num].store(true, std::memory_order_relaxed);
}
else
{
flags[nullptr][offset].store(true, std::memory_order_relaxed);
}
}
2021-06-25 12:03:10 +00:00
template <bool use_flags, bool multiple_disjuncts, typename FindResult>
bool JoinUsedFlags::getUsed(const FindResult & f)
{
if constexpr (!use_flags)
return true;
if constexpr (multiple_disjuncts)
2021-06-25 12:03:10 +00:00
{
auto & mapped = f.getMapped();
return flags[mapped.block][mapped.row_num].load();
}
else
{
return flags[nullptr][f.getOffset()].load();
2021-06-25 12:03:10 +00:00
}
}
2021-06-25 12:03:10 +00:00
template <bool use_flags, bool multiple_disjuncts, typename FindResult>
bool JoinUsedFlags::setUsedOnce(const FindResult & f)
{
if constexpr (!use_flags)
return true;
if constexpr (multiple_disjuncts)
{
auto & mapped = f.getMapped();
/// fast check to prevent heavy CAS with seq_cst order
if (flags[mapped.block][mapped.row_num].load(std::memory_order_relaxed))
return false;
bool expected = false;
return flags[mapped.block][mapped.row_num].compare_exchange_strong(expected, true);
}
else
2021-06-25 12:03:10 +00:00
{
auto off = f.getOffset();
2021-06-25 12:03:10 +00:00
/// fast check to prevent heavy CAS with seq_cst order
if (flags[nullptr][off].load(std::memory_order_relaxed))
2021-06-25 12:03:10 +00:00
return false;
2021-06-25 12:03:10 +00:00
bool expected = false;
return flags[nullptr][off].compare_exchange_strong(expected, true);
2021-06-25 12:03:10 +00:00
}
}
}
static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable)
{
if (nullable)
2019-10-29 19:39:42 +00:00
{
2019-09-11 18:03:21 +00:00
JoinCommon::convertColumnToNullable(column);
2019-10-29 19:39:42 +00:00
}
else
{
/// We have to replace values masked by NULLs with defaults.
if (column.column)
2020-04-22 06:01:33 +00:00
if (const auto * nullable_column = checkAndGetColumn<ColumnNullable>(*column.column))
2022-07-07 12:26:34 +00:00
column.column = JoinCommon::filterWithBlanks(column.column, nullable_column->getNullMapColumn().getData(), true);
2019-10-29 19:39:42 +00:00
JoinCommon::removeColumnNullability(column);
}
}
static void correctNullabilityInplace(ColumnWithTypeAndName & column, bool nullable, const IColumn::Filter & negative_null_map)
{
if (nullable)
{
JoinCommon::convertColumnToNullable(column);
2020-03-09 00:28:05 +00:00
if (column.type->isNullable() && !negative_null_map.empty())
{
2020-05-14 08:30:18 +00:00
MutableColumnPtr mutable_column = IColumn::mutate(std::move(column.column));
assert_cast<ColumnNullable &>(*mutable_column).applyNegatedNullMap(negative_null_map);
column.column = std::move(mutable_column);
}
}
2019-10-29 19:39:42 +00:00
else
JoinCommon::removeColumnNullability(column);
}
2023-07-10 08:44:01 +00:00
HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_sample_block_, bool any_take_last_row_, size_t reserve_num)
: table_join(table_join_)
, kind(table_join->kind())
, strictness(table_join->strictness())
, any_take_last_row(any_take_last_row_)
, asof_inequality(table_join->getAsofInequality())
, data(std::make_shared<RightTableData>())
, right_sample_block(right_sample_block_)
, log(&Poco::Logger::get("HashJoin"))
{
2023-01-23 12:34:36 +00:00
LOG_DEBUG(log, "({}) Datatype: {}, kind: {}, strictness: {}, right header: {}", fmt::ptr(this), data->type, kind, strictness, right_sample_block.dumpStructure());
LOG_DEBUG(log, "({}) Keys: {}", fmt::ptr(this), TableJoin::formatClauses(table_join->getClauses(), true));
if (isCrossOrComma(kind))
2021-06-25 12:03:10 +00:00
{
2021-09-10 14:52:44 +00:00
data->type = Type::CROSS;
sample_block_with_columns_to_add = right_sample_block;
2021-06-25 12:03:10 +00:00
}
else if (table_join->getClauses().empty())
{
data->type = Type::EMPTY;
/// We might need to insert default values into the right columns, materialize them
sample_block_with_columns_to_add = materializeBlock(right_sample_block);
}
2021-09-10 14:52:44 +00:00
else if (table_join->oneDisjunct())
2021-06-25 12:03:10 +00:00
{
2021-09-06 10:59:18 +00:00
const auto & key_names_right = table_join->getOnlyClause().key_names_right;
2021-06-25 12:03:10 +00:00
JoinCommon::splitAdditionalColumns(key_names_right, right_sample_block, right_table_keys, sample_block_with_columns_to_add);
required_right_keys = table_join->getRequiredRightKeys(right_table_keys, required_right_keys_sources);
}
2021-09-10 14:52:44 +00:00
else
{
/// required right keys concept does not work well if multiple disjuncts, we need all keys
sample_block_with_columns_to_add = right_table_keys = materializeBlock(right_sample_block);
}
2021-08-18 08:18:33 +00:00
2023-12-11 12:33:11 +00:00
materializeBlockInplace(right_table_keys);
initRightBlockStructure(data->sample_block);
JoinCommon::createMissedColumns(sample_block_with_columns_to_add);
2021-09-06 10:59:18 +00:00
size_t disjuncts_num = table_join->getClauses().size();
2021-09-09 09:47:08 +00:00
data->maps.resize(disjuncts_num);
2021-09-06 10:59:18 +00:00
key_sizes.reserve(disjuncts_num);
2021-09-06 10:59:18 +00:00
for (const auto & clause : table_join->getClauses())
{
2021-09-06 10:59:18 +00:00
const auto & key_names_right = clause.key_names_right;
ColumnRawPtrs key_columns = JoinCommon::extractKeysForJoin(right_table_keys, key_names_right);
2022-08-04 15:15:49 +00:00
if (strictness == JoinStrictness::Asof)
2021-06-25 12:03:10 +00:00
{
2021-09-06 10:59:18 +00:00
assert(disjuncts_num == 1);
2021-06-25 12:03:10 +00:00
/// @note ASOF JOIN is not INNER. It's better avoid use of 'INNER ASOF' combination in messages.
/// In fact INNER means 'LEFT SEMI ASOF' while LEFT means 'LEFT OUTER ASOF'.
if (!isLeft(kind) && !isInner(kind))
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Wrong ASOF JOIN type. Only ASOF and LEFT ASOF joins are supported");
2021-06-25 12:03:10 +00:00
if (key_columns.size() <= 1)
throw Exception(ErrorCodes::SYNTAX_ERROR, "ASOF join needs at least one equi-join column");
2021-06-25 12:03:10 +00:00
size_t asof_size;
asof_type = SortedLookupVectorBase::getTypeSize(*key_columns.back(), asof_size);
2021-06-25 12:03:10 +00:00
key_columns.pop_back();
/// this is going to set up the appropriate hash table for the direct lookup part of the join
/// However, this does not depend on the size of the asof join key (as that goes into the BST)
/// Therefore, add it back in such that it can be extracted appropriately from the full stored
/// key_columns and key_sizes
2021-09-09 09:47:08 +00:00
auto & asof_key_sizes = key_sizes.emplace_back();
2021-11-08 12:44:13 +00:00
data->type = chooseMethod(kind, key_columns, asof_key_sizes);
2021-09-09 09:47:08 +00:00
asof_key_sizes.push_back(asof_size);
2021-06-25 12:03:10 +00:00
}
else
{
/// Choose data structure to use for JOIN.
2021-11-08 12:44:13 +00:00
auto current_join_method = chooseMethod(kind, key_columns, key_sizes.emplace_back());
2021-09-06 10:59:18 +00:00
if (data->type == Type::EMPTY)
data->type = current_join_method;
else if (data->type != current_join_method)
data->type = Type::hashed;
2021-06-25 12:03:10 +00:00
}
}
2021-06-25 12:03:10 +00:00
2021-09-06 10:59:18 +00:00
for (auto & maps : data->maps)
2023-07-10 08:44:01 +00:00
dataMapInit(maps, reserve_num);
}
2022-07-29 16:30:50 +00:00
HashJoin::Type HashJoin::chooseMethod(JoinKind kind, const ColumnRawPtrs & key_columns, Sizes & key_sizes)
2015-03-02 01:10:58 +00:00
{
size_t keys_size = key_columns.size();
2015-07-23 20:23:24 +00:00
if (keys_size == 0)
2021-11-08 12:44:13 +00:00
{
if (isCrossOrComma(kind))
return Type::CROSS;
return Type::EMPTY;
}
bool all_fixed = true;
size_t keys_bytes = 0;
key_sizes.resize(keys_size);
2015-03-02 01:10:58 +00:00
for (size_t j = 0; j < keys_size; ++j)
{
if (!key_columns[j]->isFixedAndContiguous())
2015-03-02 01:10:58 +00:00
{
all_fixed = false;
2015-03-02 01:10:58 +00:00
break;
}
key_sizes[j] = key_columns[j]->sizeOfValueIfFixed();
2015-03-02 01:10:58 +00:00
keys_bytes += key_sizes[j];
}
2017-04-02 17:37:49 +00:00
/// If there is one numeric key that fits in 64 bits
if (keys_size == 1 && key_columns[0]->isNumeric())
{
size_t size_of_field = key_columns[0]->sizeOfValueIfFixed();
if (size_of_field == 1)
return Type::key8;
if (size_of_field == 2)
return Type::key16;
if (size_of_field == 4)
return Type::key32;
if (size_of_field == 8)
return Type::key64;
if (size_of_field == 16)
return Type::keys128;
if (size_of_field == 32)
return Type::keys256;
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
}
2017-04-02 17:37:49 +00:00
/// If the keys fit in N bits, we will use a hash table for N-bit-packed keys
if (all_fixed && keys_bytes <= 16)
return Type::keys128;
if (all_fixed && keys_bytes <= 32)
return Type::keys256;
/// If there is single string key, use hash table of it's values.
2023-12-11 15:50:27 +00:00
if (keys_size == 1)
{
auto is_string_column = [](const IColumn * column_ptr) -> bool
{
if (const auto * lc_column_ptr = typeid_cast<const ColumnLowCardinality *>(column_ptr))
return typeid_cast<const ColumnString *>(lc_column_ptr->getDictionary().getNestedColumn().get());
return typeid_cast<const ColumnString *>(column_ptr);
};
const auto * key_column = key_columns[0];
if (is_string_column(key_column) ||
(isColumnConst(*key_column) && is_string_column(assert_cast<const ColumnConst *>(key_column)->getDataColumnPtr().get())))
return Type::key_string;
}
if (keys_size == 1 && typeid_cast<const ColumnFixedString *>(key_columns[0]))
return Type::key_fixed_string;
/// Otherwise, will use set of cryptographic hashes of unambiguously serialized values.
return Type::hashed;
2015-03-02 01:10:58 +00:00
}
2022-06-14 14:13:01 +00:00
template <typename KeyGetter, bool is_asof_join>
2019-03-28 18:35:50 +00:00
static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes)
{
2019-12-02 18:07:27 +00:00
if constexpr (is_asof_join)
{
2019-03-28 18:35:50 +00:00
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);
}
2019-03-28 18:35:50 +00:00
else
return KeyGetter(key_columns, key_sizes, nullptr);
}
template <typename Mapped, bool need_offset = false>
using FindResultImpl = ColumnsHashing::columns_hashing_impl::FindResultImpl<Mapped, true>;
2021-11-08 12:44:13 +00:00
/// Dummy key getter, always find nothing, used for JOIN ON NULL
template <typename Mapped>
class KeyGetterEmpty
{
public:
struct MappedType
{
using mapped_type = Mapped;
};
using FindResult = ColumnsHashing::columns_hashing_impl::FindResultImpl<Mapped, true>;
KeyGetterEmpty() = default;
FindResult findKey(MappedType, size_t, const Arena &) { return FindResult(); }
};
template <HashJoin::Type type, typename Value, typename Mapped>
2019-01-24 14:56:04 +00:00
struct KeyGetterForTypeImpl;
constexpr bool use_offset = true;
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::key8, Value, Mapped>
2019-01-24 14:56:04 +00:00
{
using Type = ColumnsHashing::HashMethodOneNumber<Value, Mapped, UInt8, false, use_offset>;
2019-01-24 14:56:04 +00:00
};
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::key16, Value, Mapped>
2019-01-24 14:56:04 +00:00
{
using Type = ColumnsHashing::HashMethodOneNumber<Value, Mapped, UInt16, false, use_offset>;
2019-01-24 14:56:04 +00:00
};
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::key32, Value, Mapped>
2019-01-24 14:56:04 +00:00
{
using Type = ColumnsHashing::HashMethodOneNumber<Value, Mapped, UInt32, false, use_offset>;
2019-01-24 14:56:04 +00:00
};
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::key64, Value, Mapped>
2019-01-24 14:56:04 +00:00
{
using Type = ColumnsHashing::HashMethodOneNumber<Value, Mapped, UInt64, false, use_offset>;
2019-01-24 14:56:04 +00:00
};
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::key_string, Value, Mapped>
2019-01-24 14:56:04 +00:00
{
using Type = ColumnsHashing::HashMethodString<Value, Mapped, true, false, use_offset>;
2019-01-24 14:56:04 +00:00
};
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::key_fixed_string, Value, Mapped>
2019-01-24 14:56:04 +00:00
{
using Type = ColumnsHashing::HashMethodFixedString<Value, Mapped, true, false, use_offset>;
2019-01-24 14:56:04 +00:00
};
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::keys128, Value, Mapped>
2019-01-24 14:56:04 +00:00
{
using Type = ColumnsHashing::HashMethodKeysFixed<Value, UInt128, Mapped, false, false, false, use_offset>;
2019-01-24 14:56:04 +00:00
};
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::keys256, Value, Mapped>
2019-01-24 14:56:04 +00:00
{
2021-04-25 09:30:43 +00:00
using Type = ColumnsHashing::HashMethodKeysFixed<Value, UInt256, Mapped, false, false, false, use_offset>;
2019-01-24 14:56:04 +00:00
};
template <typename Value, typename Mapped> struct KeyGetterForTypeImpl<HashJoin::Type::hashed, Value, Mapped>
2019-01-24 14:56:04 +00:00
{
using Type = ColumnsHashing::HashMethodHashed<Value, Mapped, false, use_offset>;
2019-01-24 14:56:04 +00:00
};
template <HashJoin::Type type, typename Data>
2019-01-24 14:56:04 +00:00
struct KeyGetterForType
{
using Value = typename Data::value_type;
using Mapped_t = typename Data::mapped_type;
using Mapped = std::conditional_t<std::is_const_v<Data>, const Mapped_t, Mapped_t>;
using Type = typename KeyGetterForTypeImpl<type, Value, Mapped>::Type;
};
2023-07-10 08:44:01 +00:00
void HashJoin::dataMapInit(MapsVariant & map, size_t reserve_num)
{
2022-07-08 13:11:27 +00:00
2022-08-04 15:15:49 +00:00
if (kind == JoinKind::Cross)
2015-07-23 20:23:24 +00:00
return;
2021-06-25 12:03:10 +00:00
joinDispatchInit(kind, strictness, map);
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.create(data->type); });
2023-07-10 08:44:01 +00:00
if (reserve_num)
{
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { map_.reserve(data->type, reserve_num); });
}
if (!data)
throw Exception(ErrorCodes::LOGICAL_ERROR, "HashJoin::dataMapInit called with empty data");
}
2021-01-04 23:49:31 +00:00
bool HashJoin::empty() const
{
return data->type == Type::EMPTY;
}
bool HashJoin::alwaysReturnsEmptySet() const
2020-12-30 13:52:37 +00:00
{
2022-07-08 13:11:27 +00:00
return isInnerOrRight(getKind()) && data->empty;
2020-12-30 13:52:37 +00:00
}
size_t HashJoin::getTotalRowCount() const
2020-12-30 13:52:37 +00:00
{
2022-12-20 12:50:27 +00:00
if (!data)
return 0;
size_t res = 0;
2019-12-19 15:50:28 +00:00
if (data->type == Type::CROSS)
2015-07-23 20:23:24 +00:00
{
2019-12-19 15:50:28 +00:00
for (const auto & block : data->blocks)
res += block.rows();
2015-07-23 20:23:24 +00:00
}
2022-07-08 13:11:27 +00:00
else
2015-07-23 20:23:24 +00:00
{
2021-06-25 12:03:10 +00:00
for (const auto & map : data->maps)
{
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { res += map_.getTotalRowCount(data->type); });
}
2015-07-23 20:23:24 +00:00
}
return res;
}
size_t HashJoin::getTotalByteCount() const
{
2022-12-20 12:50:27 +00:00
if (!data)
return 0;
2023-03-16 13:05:06 +00:00
#ifndef NDEBUG
size_t debug_blocks_allocated_size = 0;
for (const auto & block : data->blocks)
debug_blocks_allocated_size += block.allocatedBytes();
if (data->blocks_allocated_size != debug_blocks_allocated_size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "data->blocks_allocated_size != debug_blocks_allocated_size ({} != {})",
data->blocks_allocated_size, debug_blocks_allocated_size);
size_t debug_blocks_nullmaps_allocated_size = 0;
for (const auto & nullmap : data->blocks_nullmaps)
debug_blocks_nullmaps_allocated_size += nullmap.second->allocatedBytes();
if (data->blocks_nullmaps_allocated_size != debug_blocks_nullmaps_allocated_size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "data->blocks_nullmaps_allocated_size != debug_blocks_nullmaps_allocated_size ({} != {})",
data->blocks_nullmaps_allocated_size, debug_blocks_nullmaps_allocated_size);
#endif
size_t res = 0;
res += data->blocks_allocated_size;
res += data->blocks_nullmaps_allocated_size;
res += data->pool.allocatedBytes();
if (data->type != Type::CROSS)
2015-07-23 20:23:24 +00:00
{
2021-06-25 12:03:10 +00:00
for (const auto & map : data->maps)
{
joinDispatch(kind, strictness, map, [&](auto, auto, auto & map_) { res += map_.getTotalByteCountImpl(data->type); });
}
2015-07-23 20:23:24 +00:00
}
return res;
}
namespace
{
2017-04-02 17:37:49 +00:00
/// Inserting an element into a hash table of the form `key -> reference to a string`, which will then be used by JOIN.
template <typename Map, typename KeyGetter>
2019-12-02 18:07:27 +00:00
struct Inserter
{
static ALWAYS_INLINE bool insertOne(const HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i,
2019-12-02 18:07:27 +00:00
Arena & pool)
{
2019-01-24 14:56:04 +00:00
auto emplace_result = key_getter.emplaceKey(map, i, pool);
if (emplace_result.isInserted() || join.anyTakeLastRow())
{
2019-01-24 14:56:04 +00:00
new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i);
return true;
}
return false;
}
static ALWAYS_INLINE void insertAll(const HashJoin &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool)
{
2019-01-24 14:56:04 +00:00
auto emplace_result = key_getter.emplaceKey(map, i, pool);
2019-01-24 14:56:04 +00:00
if (emplace_result.isInserted())
new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i);
else
{
2019-05-14 14:40:43 +00:00
/// The first element of the list is stored in the value of the hash table, the rest in the pool.
2019-05-14 14:39:03 +00:00
emplace_result.getMapped().insert({stored_block, i}, pool);
}
}
static ALWAYS_INLINE void insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool,
const IColumn & asof_column)
{
auto emplace_result = key_getter.emplaceKey(map, i, pool);
2022-02-18 16:45:17 +00:00
typename Map::mapped_type * time_series_map = &emplace_result.getMapped();
TypeIndex asof_type = *join.getAsofType();
if (emplace_result.isInserted())
2022-02-18 15:16:29 +00:00
time_series_map = new (time_series_map) typename Map::mapped_type(createAsofRowRef(asof_type, join.getAsofInequality()));
2022-02-18 10:02:14 +00:00
(*time_series_map)->insert(asof_column, stored_block, i);
}
};
template <JoinStrictness STRICTNESS, typename KeyGetter, typename Map>
size_t NO_INLINE insertFromBlockImplTypeCase(
HashJoin & join, Map & 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 Map::mapped_type, RowRef>;
2022-07-29 16:30:50 +00:00
constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof;
2019-12-02 18:07:27 +00:00
2019-03-28 18:35:50 +00:00
const IColumn * asof_column [[maybe_unused]] = nullptr;
2019-12-02 18:07:27 +00:00
if constexpr (is_asof_join)
asof_column = key_columns.back();
2019-03-28 18:35:50 +00:00
2019-12-02 18:07:27 +00:00
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;
2019-12-02 18:07:27 +00:00
if constexpr (is_asof_join)
Inserter<Map, KeyGetter>::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column);
2019-12-02 18:07:27 +00:00
else if constexpr (mapped_one)
is_inserted |= Inserter<Map, KeyGetter>::insertOne(join, map, key_getter, stored_block, i, pool);
else
Inserter<Map, KeyGetter>::insertAll(join, map, key_getter, stored_block, i, pool);
}
return map.getBufferSizeInCells();
}
2022-07-29 16:30:50 +00:00
template <JoinStrictness STRICTNESS, typename Maps>
size_t insertFromBlockImpl(
HashJoin & join, HashJoin::Type type, Maps & 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<STRICTNESS, 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
}
UNREACHABLE();
}
}
void HashJoin::initRightBlockStructure(Block & saved_block_sample)
{
if (isCrossOrComma(kind))
{
/// cross join doesn't have keys, just add all columns
saved_block_sample = sample_block_with_columns_to_add.cloneEmpty();
return;
}
2021-09-06 10:59:18 +00:00
bool multiple_disjuncts = !table_join->oneDisjunct();
2020-02-18 12:41:23 +00:00
/// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any).
2022-09-26 13:46:50 +00:00
bool save_key_columns = table_join->isEnabledAlgorithm(JoinAlgorithm::AUTO) ||
table_join->isEnabledAlgorithm(JoinAlgorithm::GRACE_HASH) ||
isRightOrFull(kind) ||
2022-09-26 13:46:50 +00:00
multiple_disjuncts;
2020-02-18 12:41:23 +00:00
if (save_key_columns)
2019-03-31 23:09:00 +00:00
{
saved_block_sample = right_table_keys.cloneEmpty();
}
2022-07-29 16:30:50 +00:00
else if (strictness == JoinStrictness::Asof)
{
/// Save ASOF key
saved_block_sample.insert(right_table_keys.safeGetByPosition(right_table_keys.columns() - 1));
}
2019-03-31 23:09:00 +00:00
/// Save non key columns
for (auto & column : sample_block_with_columns_to_add)
2021-06-25 12:03:10 +00:00
{
if (auto * col = saved_block_sample.findByName(column.name))
*col = column;
else
2021-06-25 12:03:10 +00:00
saved_block_sample.insert(column);
}
}
2019-03-31 23:09:00 +00:00
2022-12-20 12:50:27 +00:00
Block HashJoin::prepareRightBlock(const Block & block, const Block & saved_block_sample_)
{
Block structured_block;
2022-12-20 12:50:27 +00:00
for (const auto & sample_column : saved_block_sample_.getColumnsWithTypeAndName())
{
ColumnWithTypeAndName column = block.getByName(sample_column.name);
2023-06-06 19:03:23 +00:00
/// There's no optimization for right side const columns. Remove constness if any.
column.column = recursiveRemoveSparse(column.column->convertToFullColumnIfConst());
2022-12-28 16:02:32 +00:00
if (column.column->lowCardinality() && !sample_column.column->lowCardinality())
{
column.column = column.column->convertToFullColumnIfLowCardinality();
column.type = removeLowCardinality(column.type);
}
2023-06-06 19:03:23 +00:00
if (sample_column.column->isNullable())
JoinCommon::convertColumnToNullable(column);
2022-12-20 12:50:27 +00:00
structured_block.insert(std::move(column));
2019-03-31 23:09:00 +00:00
}
return structured_block;
2019-03-31 23:09:00 +00:00
}
2022-12-20 12:50:27 +00:00
Block HashJoin::prepareRightBlock(const Block & block) const
{
return prepareRightBlock(block, savedBlockSample());
}
2023-07-05 17:03:18 +00:00
bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits)
{
2022-12-20 12:50:27 +00:00
if (!data)
2023-01-24 14:29:19 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Join data was released");
2022-12-20 12:50:27 +00:00
2020-04-21 19:09:18 +00:00
/// RowRef::SizeT is uint32_t (not size_t) for hash table Cell memory efficiency.
2020-04-21 19:01:34 +00:00
/// It's possible to split bigger blocks and insert them by parts here. But it would be a dead code.
if (unlikely(source_block_.rows() > std::numeric_limits<RowRef::SizeT>::max()))
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Too many rows in right table block for HashJoin: {}", source_block_.rows());
/** We do not allocate memory for stored blocks inside HashJoin, only for hash table.
* In case when we have all the blocks allocated before the first `addBlockToJoin` call, will already be quite high.
* In that case memory consumed by stored blocks will be underestimated.
*/
if (!memory_usage_before_adding_blocks)
memory_usage_before_adding_blocks = getCurrentQueryMemoryUsage();
Block source_block = source_block_;
if (strictness == JoinStrictness::Asof)
{
chassert(kind == JoinKind::Left || kind == JoinKind::Inner);
/// Filter out rows with NULLs in ASOF key, nulls are not joined with anything since they are not comparable
/// We support only INNER/LEFT ASOF join, so rows with NULLs never return from the right joined table.
/// So filter them out here not to handle in implementation.
const auto & asof_key_name = table_join->getOnlyClause().key_names_right.back();
auto & asof_column = source_block.getByName(asof_key_name);
if (asof_column.type->isNullable())
{
/// filter rows with nulls in asof key
if (const auto * asof_const_column = typeid_cast<const ColumnConst *>(asof_column.column.get()))
{
if (asof_const_column->isNullAt(0))
return false;
}
else
{
const auto & asof_column_nullable = assert_cast<const ColumnNullable &>(*asof_column.column).getNullMapData();
NullMap negative_null_map(asof_column_nullable.size());
for (size_t i = 0; i < asof_column_nullable.size(); ++i)
negative_null_map[i] = !asof_column_nullable[i];
for (auto & column : source_block)
column.column = column.column->filter(negative_null_map, -1);
}
}
}
2020-04-21 19:01:34 +00:00
2022-12-27 10:13:12 +00:00
size_t rows = source_block.rows();
2023-12-11 15:50:27 +00:00
const auto & right_key_names = table_join->getAllNames(JoinTableSide::Right);
ColumnPtrMap all_key_columns(right_key_names.size());
for (const auto & column_name : right_key_names)
{
const auto & column = source_block.getByName(column_name).column;
all_key_columns[column_name] = recursiveRemoveLowCardinality(recursiveRemoveSparse(column->convertToFullColumnIfConst()));
}
2022-12-27 10:13:12 +00:00
Block block_to_save = prepareRightBlock(source_block);
if (shrink_blocks)
block_to_save = block_to_save.shrinkToFit();
2019-12-03 14:30:51 +00:00
size_t total_rows = 0;
size_t total_bytes = 0;
{
if (storage_join_lock)
2023-07-05 17:03:18 +00:00
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "addBlockToJoin called when HashJoin locked to prevent updates");
2022-12-27 10:13:12 +00:00
data->blocks_allocated_size += block_to_save.allocatedBytes();
2023-12-11 15:50:27 +00:00
assertBlocksHaveEqualStructure(data->sample_block, block_to_save, "Saved joined block structure mismatch");
2022-12-27 10:13:12 +00:00
data->blocks.emplace_back(std::move(block_to_save));
2019-12-19 15:50:28 +00:00
Block * stored_block = &data->blocks.back();
2019-12-03 14:30:51 +00:00
if (rows)
2019-12-19 15:50:28 +00:00
data->empty = false;
2019-10-12 10:06:07 +00:00
2021-09-06 10:59:18 +00:00
bool multiple_disjuncts = !table_join->oneDisjunct();
const auto & onexprs = table_join->getClauses();
for (size_t onexpr_idx = 0; onexpr_idx < onexprs.size(); ++onexpr_idx)
2015-07-23 20:23:24 +00:00
{
2021-09-06 10:59:18 +00:00
ColumnRawPtrs key_columns;
for (const auto & name : onexprs[onexpr_idx].key_names_right)
2022-12-27 17:49:17 +00:00
key_columns.push_back(all_key_columns[name].get());
2021-06-25 12:03:10 +00:00
/// We will insert to the map only keys, where all components are not NULL.
2021-09-06 10:59:18 +00:00
ConstNullMapPtr null_map{};
ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map);
2021-06-25 12:03:10 +00:00
2021-09-06 10:59:18 +00:00
/// If RIGHT or FULL save blocks with nulls for NotJoinedBlocks
2021-06-25 12:03:10 +00:00
UInt8 save_nullmap = 0;
2021-09-06 10:59:18 +00:00
if (isRightOrFull(kind) && null_map)
2019-12-03 14:30:51 +00:00
{
2021-09-06 10:59:18 +00:00
/// Save rows with NULL keys
for (size_t i = 0; !save_nullmap && i < null_map->size(); ++i)
save_nullmap |= (*null_map)[i];
2021-06-25 12:03:10 +00:00
}
2022-12-27 10:13:12 +00:00
auto join_mask_col = JoinCommon::getColumnAsMask(source_block, onexprs[onexpr_idx].condColumnNames().second);
/// Save blocks that do not hold conditions in ON section
ColumnUInt8::MutablePtr not_joined_map = nullptr;
2022-12-28 13:20:58 +00:00
if (!multiple_disjuncts && isRightOrFull(kind) && join_mask_col.hasData())
{
const auto & join_mask = join_mask_col.getData();
/// Save rows that do not hold conditions
2022-12-20 12:50:27 +00:00
not_joined_map = ColumnUInt8::create(rows, 0);
for (size_t i = 0, sz = join_mask->size(); i < sz; ++i)
{
/// Condition hold, do not save row
if ((*join_mask)[i])
continue;
2019-12-03 14:30:51 +00:00
/// NULL key will be saved anyway because, do not save twice
2021-09-06 10:59:18 +00:00
if (save_nullmap && (*null_map)[i])
continue;
not_joined_map->getData()[i] = 1;
}
}
bool is_inserted = false;
2022-07-29 16:30:50 +00:00
if (kind != JoinKind::Cross)
2021-09-06 10:59:18 +00:00
{
joinDispatch(kind, strictness, data->maps[onexpr_idx], [&](auto kind_, auto strictness_, auto & map)
{
size_t size = insertFromBlockImpl<strictness_>(
*this, data->type, map, rows, key_columns, key_sizes[onexpr_idx], stored_block, null_map,
/// If mask is false constant, rows are added to hashmap anyway. It's not a happy-flow, so this case is not optimized
join_mask_col.getData(),
data->pool, is_inserted);
2021-09-06 10:59:18 +00:00
if (multiple_disjuncts)
used_flags.reinit<kind_, strictness_>(stored_block);
else if (is_inserted)
2021-09-06 10:59:18 +00:00
/// Number of buckets + 1 value from zero storage
used_flags.reinit<kind_, strictness_>(size + 1);
});
}
if (!multiple_disjuncts && save_nullmap && is_inserted)
{
data->blocks_nullmaps_allocated_size += null_map_holder->allocatedBytes();
2021-09-06 10:59:18 +00:00
data->blocks_nullmaps.emplace_back(stored_block, null_map_holder);
}
2021-09-06 10:59:18 +00:00
if (!multiple_disjuncts && not_joined_map && is_inserted)
{
data->blocks_nullmaps_allocated_size += not_joined_map->allocatedBytes();
data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map));
}
2021-09-06 10:59:18 +00:00
if (!multiple_disjuncts && !is_inserted)
{
LOG_TRACE(log, "Skipping inserting block with {} rows", rows);
data->blocks_allocated_size -= stored_block->allocatedBytes();
data->blocks.pop_back();
}
2021-09-06 10:59:18 +00:00
if (!check_limits)
return true;
2021-09-06 10:59:18 +00:00
/// TODO: Do not calculate them every time
total_rows = getTotalRowCount();
total_bytes = getTotalByteCount();
2021-06-25 12:03:10 +00:00
}
}
2019-07-03 19:06:34 +00:00
shrinkStoredBlocksToFit(total_bytes);
return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
}
void HashJoin::shrinkStoredBlocksToFit(size_t & total_bytes_in_join)
{
if (shrink_blocks)
return; /// Already shrunk
Int64 current_memory_usage = getCurrentQueryMemoryUsage();
Int64 query_memory_usage_delta = current_memory_usage - memory_usage_before_adding_blocks;
Int64 max_total_bytes_for_query = memory_usage_before_adding_blocks ? table_join->getMaxMemoryUsage() : 0;
auto max_total_bytes_in_join = table_join->sizeLimits().max_bytes;
/** If accounted data size is more than half of `max_bytes_in_join`
* or query memory consumption growth from the beginning of adding blocks (estimation of memory consumed by join using memory tracker)
* is bigger than half of all memory available for query,
* then shrink stored blocks to fit.
*/
2023-09-14 12:58:35 +00:00
shrink_blocks = (max_total_bytes_in_join && total_bytes_in_join > max_total_bytes_in_join / 2) ||
(max_total_bytes_for_query && query_memory_usage_delta > max_total_bytes_for_query / 2);
if (!shrink_blocks)
return;
LOG_DEBUG(log, "Shrinking stored blocks, memory consumption is {} {} calculated by join, {} {} by memory tracker",
ReadableSize(total_bytes_in_join), max_total_bytes_in_join ? fmt::format("/ {}", ReadableSize(max_total_bytes_in_join)) : "",
ReadableSize(query_memory_usage_delta), max_total_bytes_for_query ? fmt::format("/ {}", ReadableSize(max_total_bytes_for_query)) : "");
2023-09-14 12:58:35 +00:00
for (auto & stored_block : data->blocks)
{
size_t old_size = stored_block.allocatedBytes();
stored_block = stored_block.shrinkToFit();
size_t new_size = stored_block.allocatedBytes();
if (old_size >= new_size)
{
if (data->blocks_allocated_size < old_size - new_size)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Blocks allocated size value is broken: "
"blocks_allocated_size = {}, old_size = {}, new_size = {}",
data->blocks_allocated_size, old_size, new_size);
data->blocks_allocated_size -= old_size - new_size;
}
else
/// Sometimes after clone resized block can be bigger than original
data->blocks_allocated_size += new_size - old_size;
}
2023-09-14 12:58:35 +00:00
auto new_total_bytes_in_join = getTotalByteCount();
Int64 new_current_memory_usage = getCurrentQueryMemoryUsage();
LOG_DEBUG(log, "Shrunk stored blocks {} freed ({} by memory tracker), new memory consumption is {} ({} by memory tracker)",
ReadableSize(total_bytes_in_join - new_total_bytes_in_join), ReadableSize(current_memory_usage - new_current_memory_usage),
ReadableSize(new_total_bytes_in_join), ReadableSize(new_current_memory_usage));
total_bytes_in_join = new_total_bytes_in_join;
2014-06-12 02:31:30 +00:00
}
namespace
{
2021-09-06 10:59:18 +00:00
struct JoinOnKeyColumns
{
2021-09-09 09:47:08 +00:00
Names key_names;
2021-09-06 10:59:18 +00:00
Columns materialized_keys_holder;
ColumnRawPtrs key_columns;
ConstNullMapPtr null_map;
2021-09-09 09:47:08 +00:00
ColumnPtr null_map_holder;
2021-09-06 10:59:18 +00:00
/// Only rows where mask == true can be joined
JoinCommon::JoinMask join_mask_column;
2021-09-06 10:59:18 +00:00
Sizes key_sizes;
explicit JoinOnKeyColumns(const Block & block, const Names & key_names_, const String & cond_column_name, const Sizes & key_sizes_)
: key_names(key_names_)
, materialized_keys_holder(JoinCommon::materializeColumns(block, key_names)) /// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them.
, key_columns(JoinCommon::getRawPointers(materialized_keys_holder))
2021-09-09 09:47:08 +00:00
, null_map(nullptr)
2021-09-06 10:59:18 +00:00
, null_map_holder(extractNestedColumnsAndNullMap(key_columns, null_map))
, join_mask_column(JoinCommon::getColumnAsMask(block, cond_column_name))
, key_sizes(key_sizes_)
{
}
bool isRowFiltered(size_t i) const { return join_mask_column.isRowFiltered(i); }
2021-09-06 10:59:18 +00:00
};
2019-03-20 14:49:05 +00:00
class AddedColumns
{
public:
struct TypeAndName
{
DataTypePtr type;
String name;
String qualified_name;
TypeAndName(DataTypePtr type_, const String & name_, const String & qualified_name_)
: type(type_), name(name_), qualified_name(qualified_name_)
{
}
};
AddedColumns(
const Block & block_with_columns_to_add,
const Block & block,
const Block & saved_block_sample,
const HashJoin & join,
2021-09-06 10:59:18 +00:00
std::vector<JoinOnKeyColumns> && join_on_keys_,
bool is_asof_join,
bool is_join_get_)
2021-09-06 10:59:18 +00:00
: join_on_keys(join_on_keys_)
2019-11-06 19:39:52 +00:00
, rows_to_add(block.rows())
, sample_block(saved_block_sample)
, is_join_get(is_join_get_)
2019-03-20 14:49:05 +00:00
{
size_t num_columns_to_add = block_with_columns_to_add.columns();
if (is_asof_join)
++num_columns_to_add;
2019-03-20 15:15:44 +00:00
columns.reserve(num_columns_to_add);
type_name.reserve(num_columns_to_add);
right_indexes.reserve(num_columns_to_add);
2020-04-22 06:01:33 +00:00
for (const auto & src_column : block_with_columns_to_add)
2019-03-20 15:15:44 +00:00
{
2021-05-10 13:39:32 +00:00
/// Column names `src_column.name` and `qualified_name` can differ for StorageJoin,
/// because it uses not qualified right block column names
auto qualified_name = join.getTableJoin().renamedRightColumnName(src_column.name);
2020-04-07 14:52:32 +00:00
/// Don't insert column if it's in left block
if (!block.has(qualified_name))
addColumn(src_column, qualified_name);
}
if (is_asof_join)
{
2021-09-06 10:59:18 +00:00
assert(join_on_keys.size() == 1);
const ColumnWithTypeAndName & right_asof_column = join.rightAsofKeyColumn();
addColumn(right_asof_column, right_asof_column.name);
2021-09-09 09:47:08 +00:00
left_asof_key = join_on_keys[0].key_columns.back();
}
2019-04-01 10:35:37 +00:00
for (auto & tn : type_name)
right_indexes.push_back(saved_block_sample.getPositionByName(tn.name));
2019-03-20 14:49:05 +00:00
}
2019-03-20 15:15:44 +00:00
size_t size() const { return columns.size(); }
2019-03-20 14:49:05 +00:00
ColumnWithTypeAndName moveColumn(size_t i)
{
return ColumnWithTypeAndName(std::move(columns[i]), type_name[i].type, type_name[i].qualified_name);
2019-03-20 14:49:05 +00:00
}
2019-11-08 16:13:43 +00:00
template <bool has_defaults>
void appendFromBlock(const Block & block, size_t row_num)
2019-03-20 14:49:05 +00:00
{
2019-11-08 16:13:43 +00:00
if constexpr (has_defaults)
applyLazyDefaults();
if (is_join_get)
{
/// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin.
for (size_t j = 0, size = right_indexes.size(); j < size; ++j)
{
const auto & column_from_block = block.getByPosition(right_indexes[j]);
2022-03-25 11:12:02 +00:00
if (auto * nullable_col = typeid_cast<ColumnNullable *>(columns[j].get());
nullable_col && !column_from_block.column->isNullable())
nullable_col->insertFromNotNullable(*column_from_block.column, row_num);
else if (auto * lowcard_col = typeid_cast<ColumnLowCardinality *>(columns[j].get());
lowcard_col && !typeid_cast<const ColumnLowCardinality *>(column_from_block.column.get()))
lowcard_col->insertFromFullColumn(*column_from_block.column, row_num);
else
2022-03-25 11:12:02 +00:00
columns[j]->insertFrom(*column_from_block.column, row_num);
}
}
else
{
for (size_t j = 0, size = right_indexes.size(); j < size; ++j)
{
const auto & column_from_block = block.getByPosition(right_indexes[j]);
if (auto * lowcard_col = typeid_cast<ColumnLowCardinality *>(columns[j].get());
lowcard_col && !typeid_cast<const ColumnLowCardinality *>(column_from_block.column.get()))
lowcard_col->insertFromFullColumn(*column_from_block.column, row_num);
else
columns[j]->insertFrom(*column_from_block.column, row_num);
}
}
}
2019-03-20 14:49:05 +00:00
void appendDefaultRow()
{
2019-11-06 19:39:52 +00:00
++lazy_defaults_count;
2019-03-20 14:49:05 +00:00
}
2019-11-06 19:39:52 +00:00
void applyLazyDefaults()
{
if (lazy_defaults_count)
{
2021-01-28 08:26:10 +00:00
for (size_t j = 0, size = right_indexes.size(); j < size; ++j)
JoinCommon::addDefaultValues(*columns[j], type_name[j].type, lazy_defaults_count);
2019-11-06 19:39:52 +00:00
lazy_defaults_count = 0;
}
}
const IColumn & leftAsofKey() const { return *left_asof_key; }
2021-09-06 10:59:18 +00:00
std::vector<JoinOnKeyColumns> join_on_keys;
2019-11-06 19:39:52 +00:00
size_t rows_to_add;
std::unique_ptr<IColumn::Offsets> offsets_to_replicate;
bool need_filter = false;
2019-11-06 19:39:52 +00:00
private:
std::vector<TypeAndName> type_name;
MutableColumns columns;
std::vector<size_t> right_indexes;
2019-11-06 19:39:52 +00:00
size_t lazy_defaults_count = 0;
/// for ASOF
const IColumn * left_asof_key = nullptr;
Block sample_block;
bool is_join_get;
2019-03-31 23:09:00 +00:00
void addColumn(const ColumnWithTypeAndName & src_column, const std::string & qualified_name)
2019-03-31 23:09:00 +00:00
{
columns.push_back(src_column.column->cloneEmpty());
columns.back()->reserve(src_column.column->size());
type_name.emplace_back(src_column.type, src_column.name, qualified_name);
2019-03-31 23:09:00 +00:00
}
2019-03-20 15:15:44 +00:00
};
2022-07-29 16:30:50 +00:00
template <JoinKind KIND, JoinStrictness STRICTNESS>
2021-06-25 12:03:10 +00:00
struct JoinFeatures
{
2022-07-29 16:30:50 +00:00
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 left = KIND == JoinKind::Left;
static constexpr bool right = KIND == JoinKind::Right;
static constexpr bool inner = KIND == JoinKind::Inner;
static constexpr bool full = KIND == JoinKind::Full;
2021-06-25 12:03:10 +00:00
static constexpr bool need_replication = is_all_join || (is_any_join && right) || (is_semi_join && right);
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;
};
template <bool multiple_disjuncts>
class KnownRowsHolder;
/// Keep already joined rows to prevent duplication if many disjuncts
/// if for a particular pair of rows condition looks like TRUE or TRUE or TRUE
/// we want to have it once in resultset
2021-06-25 12:03:10 +00:00
template<>
class KnownRowsHolder<true>
{
public:
using Type = std::pair<const Block *, DB::RowRef::SizeT>;
2021-06-25 12:03:10 +00:00
private:
static const size_t MAX_LINEAR = 16; // threshold to switch from Array to Set
using ArrayHolder = std::array<Type, MAX_LINEAR>;
using SetHolder = std::set<Type>;
using SetHolderPtr = std::unique_ptr<SetHolder>;
2021-06-25 12:03:10 +00:00
ArrayHolder array_holder;
SetHolderPtr set_holder_ptr;
2021-06-25 12:03:10 +00:00
size_t items;
public:
KnownRowsHolder()
: items(0)
{
}
template<class InputIt>
void add(InputIt from, InputIt to)
{
const size_t new_items = std::distance(from, to);
2021-06-25 12:03:10 +00:00
if (items + new_items <= MAX_LINEAR)
{
std::copy(from, to, &array_holder[items]);
2021-06-25 12:03:10 +00:00
}
else
{
if (items <= MAX_LINEAR)
{
set_holder_ptr = std::make_unique<SetHolder>();
set_holder_ptr->insert(std::cbegin(array_holder), std::cbegin(array_holder) + items);
2021-06-25 12:03:10 +00:00
}
set_holder_ptr->insert(from, to);
2021-06-25 12:03:10 +00:00
}
items += new_items;
}
template<class Needle>
bool isKnown(const Needle & needle)
{
return items <= MAX_LINEAR
? std::find(std::cbegin(array_holder), std::cbegin(array_holder) + items, needle) != std::cbegin(array_holder) + items
: set_holder_ptr->find(needle) != set_holder_ptr->end();
2021-06-25 12:03:10 +00:00
}
};
template<>
class KnownRowsHolder<false>
{
public:
template<class InputIt>
void add(InputIt, InputIt)
{
}
template<class Needle>
static bool isKnown(const Needle &)
{
return false;
}
};
template <typename Map, bool add_missing, bool multiple_disjuncts>
void addFoundRowAll(
const typename Map::mapped_type & mapped,
AddedColumns & added,
IColumn::Offset & current_offset,
KnownRowsHolder<multiple_disjuncts> & known_rows [[maybe_unused]],
JoinStuff::JoinUsedFlags * used_flags [[maybe_unused]])
2019-03-20 14:49:05 +00:00
{
if constexpr (add_missing)
added.applyLazyDefaults();
2021-06-25 12:03:10 +00:00
if constexpr (multiple_disjuncts)
2019-03-19 16:53:36 +00:00
{
2021-06-25 12:03:10 +00:00
std::unique_ptr<std::vector<KnownRowsHolder<true>::Type>> new_known_rows_ptr;
for (auto it = mapped.begin(); it.ok(); ++it)
{
if (!known_rows.isKnown(std::make_pair(it->block, it->row_num)))
{
added.appendFromBlock<false>(*it->block, it->row_num);
++current_offset;
if (!new_known_rows_ptr)
{
new_known_rows_ptr = std::make_unique<std::vector<KnownRowsHolder<true>::Type>>();
}
new_known_rows_ptr->push_back(std::make_pair(it->block, it->row_num));
if (used_flags)
{
used_flags->JoinStuff::JoinUsedFlags::setUsedOnce<true, multiple_disjuncts>(
FindResultImpl<const RowRef, false>(*it, true, 0));
}
2021-06-25 12:03:10 +00:00
}
}
if (new_known_rows_ptr)
{
known_rows.add(std::cbegin(*new_known_rows_ptr), std::cend(*new_known_rows_ptr));
}
}
else
{
for (auto it = mapped.begin(); it.ok(); ++it)
{
added.appendFromBlock<false>(*it->block, it->row_num);
++current_offset;
}
2019-03-19 16:53:36 +00:00
}
2022-05-16 18:59:27 +00:00
}
template <bool add_missing, bool need_offset>
2019-03-20 14:49:05 +00:00
void addNotFoundRow(AddedColumns & added [[maybe_unused]], IColumn::Offset & current_offset [[maybe_unused]])
{
2019-11-08 16:13:43 +00:00
if constexpr (add_missing)
{
2019-03-20 14:49:05 +00:00
added.appendDefaultRow();
if constexpr (need_offset)
++current_offset;
2019-03-20 14:49:05 +00:00
}
}
2019-03-19 16:53:36 +00:00
template <bool need_filter>
void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]])
{
if constexpr (need_filter)
filter[pos] = 1;
}
2019-03-20 14:49:05 +00:00
/// 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 KeyGetter, typename Map, bool need_filter, bool multiple_disjuncts>
NO_INLINE IColumn::Filter joinRightColumns(
2021-06-25 12:03:10 +00:00
std::vector<KeyGetter> && key_getter_vector,
const std::vector<const Map *> & mapv,
AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]])
2019-03-20 14:49:05 +00:00
{
2023-03-19 06:17:59 +00:00
constexpr JoinFeatures<KIND, STRICTNESS> join_features;
2019-11-06 19:39:52 +00:00
size_t rows = added_columns.rows_to_add;
IColumn::Filter filter;
if constexpr (need_filter)
filter = IColumn::Filter(rows, 0);
2019-03-20 14:49:05 +00:00
Arena pool;
2023-03-19 06:17:59 +00:00
if constexpr (join_features.need_replication)
2019-11-11 11:42:10 +00:00
added_columns.offsets_to_replicate = std::make_unique<IColumn::Offsets>(rows);
IColumn::Offset current_offset = 0;
2019-03-20 14:49:05 +00:00
for (size_t i = 0; i < rows; ++i)
{
2021-06-25 12:03:10 +00:00
bool right_row_found = false;
KnownRowsHolder<multiple_disjuncts> known_rows;
2021-09-06 10:59:18 +00:00
for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx)
{
2021-09-06 10:59:18 +00:00
const auto & join_keys = added_columns.join_on_keys[onexpr_idx];
if (join_keys.null_map && (*join_keys.null_map)[i])
continue;
2019-11-11 11:42:10 +00:00
2021-09-06 10:59:18 +00:00
bool row_acceptable = !join_keys.isRowFiltered(i);
2021-06-25 12:03:10 +00:00
using FindResult = typename KeyGetter::FindResult;
2021-09-06 10:59:18 +00:00
auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult();
2019-11-11 11:42:10 +00:00
2021-06-25 12:03:10 +00:00
if (find_result.isFound())
{
2021-06-25 12:03:10 +00:00
right_row_found = true;
auto & mapped = find_result.getMapped();
2023-03-19 06:17:59 +00:00
if constexpr (join_features.is_asof_join)
2021-06-25 12:03:10 +00:00
{
const IColumn & left_asof_key = added_columns.leftAsofKey();
2022-03-23 11:19:38 +00:00
auto row_ref = mapped->findAsof(left_asof_key, i);
if (row_ref.block)
2021-06-25 12:03:10 +00:00
{
setUsed<need_filter>(filter, i);
if constexpr (multiple_disjuncts)
2023-03-19 06:17:59 +00:00
used_flags.template setUsed<join_features.need_flags, multiple_disjuncts>(row_ref.block, row_ref.row_num, 0);
else
2023-03-19 06:17:59 +00:00
used_flags.template setUsed<join_features.need_flags, multiple_disjuncts>(find_result);
2023-03-19 06:17:59 +00:00
added_columns.appendFromBlock<join_features.add_missing>(*row_ref.block, row_ref.row_num);
2021-06-25 12:03:10 +00:00
}
else
2023-03-19 06:17:59 +00:00
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, current_offset);
2021-06-25 12:03:10 +00:00
}
2023-03-19 06:17:59 +00:00
else if constexpr (join_features.is_all_join)
2019-03-30 21:30:21 +00:00
{
setUsed<need_filter>(filter, i);
2023-03-19 06:17:59 +00:00
used_flags.template setUsed<join_features.need_flags, multiple_disjuncts>(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);
2019-03-30 21:30:21 +00:00
}
2023-03-19 06:17:59 +00:00
else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right)
2021-06-25 12:03:10 +00:00
{
/// Use first appeared left key + it needs left columns replication
2023-03-19 06:17:59 +00:00
bool used_once = used_flags.template setUsedOnce<join_features.need_flags, multiple_disjuncts>(find_result);
2021-06-25 12:03:10 +00:00
if (used_once)
{
2023-03-19 06:17:59 +00:00
auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr;
2021-06-25 12:03:10 +00:00
setUsed<need_filter>(filter, i);
2023-03-19 06:17:59 +00:00
addFoundRowAll<Map, join_features.add_missing>(mapped, added_columns, current_offset, known_rows, used_flags_opt);
2021-06-25 12:03:10 +00:00
}
}
2023-03-19 06:17:59 +00:00
else if constexpr (join_features.is_any_join && KIND == JoinKind::Inner)
2021-06-25 12:03:10 +00:00
{
2023-03-19 06:17:59 +00:00
bool used_once = used_flags.template setUsedOnce<join_features.need_flags, multiple_disjuncts>(find_result);
2021-06-25 12:03:10 +00:00
/// Use first appeared left key only
if (used_once)
{
setUsed<need_filter>(filter, i);
2023-03-19 06:17:59 +00:00
added_columns.appendFromBlock<join_features.add_missing>(*mapped.block, mapped.row_num);
2021-06-25 12:03:10 +00:00
}
2021-06-25 12:03:10 +00:00
break;
}
2023-03-19 06:17:59 +00:00
else if constexpr (join_features.is_any_join && join_features.full)
2021-06-25 12:03:10 +00:00
{
/// TODO
}
2023-03-19 06:17:59 +00:00
else if constexpr (join_features.is_anti_join)
2021-06-25 12:03:10 +00:00
{
2023-03-19 06:17:59 +00:00
if constexpr (join_features.right && join_features.need_flags)
used_flags.template setUsed<join_features.need_flags, multiple_disjuncts>(find_result);
2021-06-25 12:03:10 +00:00
}
else /// ANY LEFT, SEMI LEFT, old ANY (RightAny)
{
setUsed<need_filter>(filter, i);
2023-03-19 06:17:59 +00:00
used_flags.template setUsed<join_features.need_flags, multiple_disjuncts>(find_result);
added_columns.appendFromBlock<join_features.add_missing>(*mapped.block, mapped.row_num);
2021-06-25 12:03:10 +00:00
2023-03-19 06:17:59 +00:00
if (join_features.is_any_or_semi_join)
2021-06-25 12:03:10 +00:00
{
break;
}
}
}
2021-09-06 10:59:18 +00:00
}
2021-06-25 12:03:10 +00:00
if (!right_row_found)
2019-12-02 18:07:27 +00:00
{
2023-03-19 06:17:59 +00:00
if constexpr (join_features.is_anti_join && join_features.left)
setUsed<need_filter>(filter, i);
2023-03-19 06:17:59 +00:00
addNotFoundRow<join_features.add_missing, join_features.need_replication>(added_columns, current_offset);
}
2023-03-19 06:17:59 +00:00
if constexpr (join_features.need_replication)
2021-06-25 12:03:10 +00:00
{
2021-09-06 10:59:18 +00:00
(*added_columns.offsets_to_replicate)[i] = current_offset;
2021-06-25 12:03:10 +00:00
}
2019-03-19 16:53:36 +00:00
}
2019-11-06 19:39:52 +00:00
added_columns.applyLazyDefaults();
2019-11-11 11:42:10 +00:00
return filter;
2019-03-20 14:49:05 +00:00
}
template <JoinKind KIND, JoinStrictness STRICTNESS, typename KeyGetter, typename Map, bool need_filter>
2021-06-25 12:03:10 +00:00
IColumn::Filter joinRightColumnsSwitchMultipleDisjuncts(
std::vector<KeyGetter> && key_getter_vector,
const std::vector<const Map *> & mapv,
AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]])
{
return mapv.size() > 1
? joinRightColumns<KIND, STRICTNESS, KeyGetter, Map, need_filter, true>(std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags)
: joinRightColumns<KIND, STRICTNESS, KeyGetter, Map, need_filter, false>(std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
2021-06-25 12:03:10 +00:00
}
2022-07-29 16:30:50 +00:00
template <JoinKind KIND, JoinStrictness STRICTNESS, typename KeyGetter, typename Map>
IColumn::Filter joinRightColumnsSwitchNullability(
2021-06-25 12:03:10 +00:00
std::vector<KeyGetter> && key_getter_vector,
2021-09-06 10:59:18 +00:00
const std::vector<const Map *> & mapv,
2021-06-25 12:03:10 +00:00
AddedColumns & added_columns,
JoinStuff::JoinUsedFlags & used_flags)
2019-03-20 14:49:05 +00:00
{
if (added_columns.need_filter)
{
return joinRightColumnsSwitchMultipleDisjuncts<KIND, STRICTNESS, KeyGetter, Map, true>(std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
}
2019-03-20 14:49:05 +00:00
else
{
return joinRightColumnsSwitchMultipleDisjuncts<KIND, STRICTNESS, KeyGetter, Map, true>(std::forward<std::vector<KeyGetter>>(key_getter_vector), mapv, added_columns, used_flags);
}
2019-03-20 14:49:05 +00:00
}
2019-03-19 16:53:36 +00:00
2022-07-29 16:30:50 +00:00
template <JoinKind KIND, JoinStrictness STRICTNESS, typename Maps>
IColumn::Filter switchJoinRightColumns(
2021-09-06 10:59:18 +00:00
const std::vector<const Maps *> & mapv,
AddedColumns & added_columns,
HashJoin::Type type,
JoinStuff::JoinUsedFlags & used_flags)
2019-03-20 14:49:05 +00:00
{
2022-07-29 16:30:50 +00:00
constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof;
2019-03-20 14:49:05 +00:00
switch (type)
{
2021-11-08 12:44:13 +00:00
case HashJoin::Type::EMPTY:
{
if constexpr (!is_asof_join)
{
using KeyGetter = KeyGetterEmpty<typename Maps::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<KIND, STRICTNESS, 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);
}
2019-03-20 14:49:05 +00:00
#define M(TYPE) \
case HashJoin::Type::TYPE: \
2021-06-25 12:03:10 +00:00
{ \
2021-09-06 10:59:18 +00:00
using MapTypeVal = const typename std::remove_reference_t<decltype(Maps::TYPE)>::element_type; \
using KeyGetter = typename KeyGetterForType<HashJoin::Type::TYPE, MapTypeVal>::Type; \
2021-11-08 12:44:13 +00:00
std::vector<const MapTypeVal *> a_map_type_vector(mapv.size()); \
2021-06-25 12:03:10 +00:00
std::vector<KeyGetter> key_getter_vector; \
2021-09-06 10:59:18 +00:00
for (size_t d = 0; d < added_columns.join_on_keys.size(); ++d) \
{ \
const auto & join_on_key = added_columns.join_on_keys[d]; \
2021-06-25 12:03:10 +00:00
a_map_type_vector[d] = mapv[d]->TYPE.get(); \
2021-09-06 10:59:18 +00:00
key_getter_vector.push_back(std::move(createKeyGetter<KeyGetter, is_asof_join>(join_on_key.key_columns, join_on_key.key_sizes))); \
2021-06-25 12:03:10 +00:00
} \
return joinRightColumnsSwitchNullability<KIND, STRICTNESS, KeyGetter>( \
2021-09-06 10:59:18 +00:00
std::move(key_getter_vector), a_map_type_vector, added_columns, used_flags); \
2021-06-25 12:03:10 +00:00
}
2019-03-20 14:49:05 +00:00
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
default:
2021-11-08 12:44:13 +00:00
throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", type);
}
}
/** Since we do not store right key columns,
* this function is used to copy left key columns to right key columns.
* If the user requests some right columns, we just copy left key columns to right, since they are equal.
* Example: SELECT t1.key, t2.key FROM t1 FULL JOIN t2 ON t1.key = t2.key;
* In that case for matched rows in t2.key we will use values from t1.key.
* However, in some cases we might need to adjust the type of column, e.g. t1.key :: LowCardinality(String) and t2.key :: String
* Also, the nullability of the column might be different.
* Returns the right column after with necessary adjustments.
*/
ColumnWithTypeAndName copyLeftKeyColumnToRight(
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;
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;
}
2019-03-20 14:49:05 +00:00
} /// nameless
2022-07-29 16:30:50 +00:00
template <JoinKind KIND, JoinStrictness STRICTNESS, typename Maps>
void HashJoin::joinBlockImpl(
Block & block,
const Block & block_with_columns_to_add,
2021-09-06 10:59:18 +00:00
const std::vector<const Maps *> & maps_,
bool is_join_get) const
{
2023-03-19 06:17:59 +00:00
constexpr JoinFeatures<KIND, STRICTNESS> join_features;
2021-09-06 10:59:18 +00:00
std::vector<JoinOnKeyColumns> join_on_keys;
const auto & onexprs = table_join->getClauses();
for (size_t i = 0; i < onexprs.size(); ++i)
2021-06-25 12:03:10 +00:00
{
2021-09-06 10:59:18 +00:00
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, key_sizes[i]);
2021-06-25 12:03:10 +00:00
}
size_t existing_columns = block.columns();
2017-04-02 17:37:49 +00:00
/** 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.
*/
2023-03-19 06:17:59 +00:00
if constexpr (join_features.right || join_features.full)
{
2019-09-12 12:59:53 +00:00
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.
2019-03-31 23:09:00 +00:00
* For ASOF, the last column is used as the ASOF column
*/
AddedColumns added_columns(
block_with_columns_to_add,
block,
savedBlockSample(),
*this,
2021-09-06 10:59:18 +00:00
std::move(join_on_keys),
2023-03-19 06:17:59 +00:00
join_features.is_asof_join,
is_join_get);
bool has_required_right_keys = (required_right_keys.columns() != 0);
2023-03-19 06:17:59 +00:00
added_columns.need_filter = join_features.need_filter || has_required_right_keys;
2019-03-20 12:08:38 +00:00
2022-07-08 13:11:27 +00:00
IColumn::Filter row_filter = switchJoinRightColumns<KIND, STRICTNESS>(maps_, added_columns, data->type, used_flags);
2019-11-06 19:39:52 +00:00
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]];
2023-03-19 06:17:59 +00:00
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(row_filter, -1);
2021-09-06 10:59:18 +00:00
/// Add join key columns from right block if needed using value from left table because of equality
for (size_t i = 0; i < required_right_keys.columns(); ++i)
{
const auto & right_key = required_right_keys.getByPosition(i);
2021-06-25 12:03:10 +00:00
// renamed ???
if (!block.findByName(right_key.name))
{
/// asof column is already in block.
2023-03-19 06:17:59 +00:00
if (join_features.is_asof_join && right_key.name == table_join->getOnlyClause().key_names_right.back())
2021-06-25 12:03:10 +00:00
continue;
const auto & left_column = block.getByName(required_right_keys_sources[i]);
const auto & right_col_name = getTableJoin().renamedRightColumnName(right_key.name);
auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column);
block.insert(std::move(right_col));
2021-06-25 12:03:10 +00:00
}
}
}
else if (has_required_right_keys)
{
2019-10-29 19:39:42 +00:00
/// Add join key columns from right block if needed.
for (size_t i = 0; i < required_right_keys.columns(); ++i)
{
const auto & right_key = required_right_keys.getByPosition(i);
2021-06-25 12:03:10 +00:00
auto right_col_name = getTableJoin().renamedRightColumnName(right_key.name);
2022-12-07 16:27:01 +00:00
if (!block.findByName(right_col_name))
2021-06-25 12:03:10 +00:00
{
/// asof column is already in block.
2023-03-19 06:17:59 +00:00
if (join_features.is_asof_join && right_key.name == table_join->getOnlyClause().key_names_right.back())
2021-06-25 12:03:10 +00:00
continue;
2020-11-03 11:28:28 +00:00
const auto & left_column = block.getByName(required_right_keys_sources[i]);
auto right_col = copyLeftKeyColumnToRight(right_key.type, right_col_name, left_column, &row_filter);
block.insert(std::move(right_col));
2023-03-19 06:17:59 +00:00
if constexpr (join_features.need_replication)
2022-12-07 16:27:01 +00:00
right_keys_to_replicate.push_back(block.getPositionByName(right_col_name));
2021-06-25 12:03:10 +00:00
}
}
}
2023-03-19 06:17:59 +00:00
if constexpr (join_features.need_replication)
{
2019-11-06 19:39:52 +00:00
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);
}
}
void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const
2015-07-23 20:23:24 +00:00
{
2020-04-06 13:39:57 +00:00
size_t max_joined_block_rows = table_join->maxJoinedBlockRows();
size_t start_left_row = 0;
size_t start_right_block = 0;
if (not_processed)
{
auto & continuation = static_cast<NotProcessedCrossJoin &>(*not_processed);
start_left_row = continuation.left_position;
start_right_block = continuation.right_block;
not_processed.reset();
}
size_t num_existing_columns = block.columns();
2015-07-23 20:23:24 +00:00
size_t num_columns_to_add = sample_block_with_columns_to_add.columns();
2020-04-06 13:39:57 +00:00
ColumnRawPtrs src_left_columns;
MutableColumns dst_columns;
2015-07-23 20:23:24 +00:00
{
2020-04-06 13:39:57 +00:00
src_left_columns.reserve(num_existing_columns);
dst_columns.reserve(num_existing_columns + num_columns_to_add);
2020-04-06 13:39:57 +00:00
for (const ColumnWithTypeAndName & left_column : block)
{
src_left_columns.push_back(left_column.column.get());
dst_columns.emplace_back(src_left_columns.back()->cloneEmpty());
}
for (const ColumnWithTypeAndName & right_column : sample_block_with_columns_to_add)
dst_columns.emplace_back(right_column.column->cloneEmpty());
for (auto & dst : dst_columns)
dst->reserve(max_joined_block_rows);
2015-07-23 20:23:24 +00:00
}
2020-04-06 13:39:57 +00:00
size_t rows_left = block.rows();
size_t rows_added = 0;
2020-04-06 13:39:57 +00:00
for (size_t left_row = start_left_row; left_row < rows_left; ++left_row)
2015-07-23 20:23:24 +00:00
{
2020-04-06 13:39:57 +00:00
size_t block_number = 0;
2019-12-19 15:50:28 +00:00
for (const Block & block_right : data->blocks)
2015-07-23 20:23:24 +00:00
{
2020-04-06 13:39:57 +00:00
++block_number;
if (block_number < start_right_block)
continue;
size_t rows_right = block_right.rows();
2020-04-06 13:39:57 +00:00
rows_added += rows_right;
2015-07-23 20:23:24 +00:00
for (size_t col_num = 0; col_num < num_existing_columns; ++col_num)
2020-04-06 13:39:57 +00:00
dst_columns[col_num]->insertManyFrom(*src_left_columns[col_num], left_row, rows_right);
2015-07-23 20:23:24 +00:00
for (size_t col_num = 0; col_num < num_columns_to_add; ++col_num)
{
2020-04-06 13:39:57 +00:00
const IColumn & column_right = *block_right.getByPosition(col_num).column;
dst_columns[num_existing_columns + col_num]->insertRangeFrom(column_right, 0, rows_right);
2015-07-23 20:23:24 +00:00
}
}
2020-04-06 13:39:57 +00:00
start_right_block = 0;
if (rows_added > max_joined_block_rows)
{
not_processed = std::make_shared<NotProcessedCrossJoin>(
NotProcessedCrossJoin{{block.cloneEmpty()}, left_row, block_number + 1});
not_processed->block.swap(block);
break;
}
2015-07-23 20:23:24 +00:00
}
2020-04-06 13:39:57 +00:00
for (const ColumnWithTypeAndName & src_column : sample_block_with_columns_to_add)
block.insert(src_column);
block = block.cloneWithColumns(std::move(dst_columns));
2015-07-23 20:23:24 +00:00
}
DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const
{
size_t num_keys = data_types.size();
if (right_table_keys.columns() != num_keys)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function joinGet{} doesn't match: passed, should be equal to {}",
toString(or_null ? "OrNull" : ""), toString(num_keys));
for (size_t i = 0; i < num_keys; ++i)
{
const auto & left_type_origin = data_types[i];
const auto & [c2, right_type_origin, right_name] = right_table_keys.safeGetByPosition(i);
2021-02-24 08:44:51 +00:00
auto left_type = removeNullable(recursiveRemoveLowCardinality(left_type_origin));
auto right_type = removeNullable(recursiveRemoveLowCardinality(right_type_origin));
if (!left_type->equals(*right_type))
throw Exception(ErrorCodes::TYPE_MISMATCH, "Type mismatch in joinGet key {}: "
"found type {}, while the needed type is {}", i, left_type->getName(), right_type->getName());
}
if (!sample_block_with_columns_to_add.has(column_name))
throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, "StorageJoin doesn't contain column {}", column_name);
2020-04-12 03:18:21 +00:00
auto elem = sample_block_with_columns_to_add.getByName(column_name);
2022-05-30 13:01:27 +00:00
if (or_null && JoinCommon::canBecomeNullable(elem.type))
2020-04-12 03:25:47 +00:00
elem.type = makeNullable(elem.type);
2020-04-12 03:18:21 +00:00
return elem.type;
}
/// TODO: return multiple columns as named tuple
2022-07-29 16:30:50 +00:00
/// TODO: return array of values when strictness == JoinStrictness::All
ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const
{
2022-07-29 16:30:50 +00:00
bool is_valid = (strictness == JoinStrictness::Any || strictness == JoinStrictness::RightAny)
&& kind == JoinKind::Left;
if (!is_valid)
throw Exception(ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN, "joinGet only supports StorageJoin of type Left Any");
2021-09-06 10:59:18 +00:00
const auto & key_names_right = table_join->getOnlyClause().key_names_right;
/// Assemble the key block with correct names.
Block keys;
for (size_t i = 0; i < block.columns(); ++i)
{
auto key = block.getByPosition(i);
key.name = key_names_right[i];
keys.insert(std::move(key));
}
2022-07-29 16:30:50 +00:00
static_assert(!MapGetter<JoinKind::Left, JoinStrictness::Any>::flagged,
"joinGet are not protected from hash table changes between block processing");
2021-06-25 12:03:10 +00:00
2021-09-06 10:59:18 +00:00
std::vector<const MapsOne *> maps_vector;
2021-06-25 12:03:10 +00:00
maps_vector.push_back(&std::get<MapsOne>(data->maps[0]));
2022-07-29 16:30:50 +00:00
joinBlockImpl<JoinKind::Left, JoinStrictness::Any>(
2021-09-06 10:59:18 +00:00
keys, block_with_columns_to_add, maps_vector, true);
return keys.getByPosition(keys.columns() - 1);
}
void HashJoin::checkTypesOfKeys(const Block & block) const
{
2021-09-20 19:30:34 +00:00
for (const auto & onexpr : table_join->getClauses())
{
JoinCommon::checkTypesOfKeys(block, onexpr.key_names_left, right_table_keys, onexpr.key_names_right);
}
}
void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed)
{
2022-12-20 12:50:27 +00:00
if (!data)
2022-10-19 10:20:41 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot join after data has been released");
2021-09-06 10:59:18 +00:00
for (const auto & onexpr : table_join->getClauses())
2021-06-25 12:03:10 +00:00
{
2021-09-06 10:59:18 +00:00
auto cond_column_name = onexpr.condColumnNames();
JoinCommon::checkTypesOfKeys(
block, onexpr.key_names_left, cond_column_name.first,
right_sample_block, onexpr.key_names_right, cond_column_name.second);
2021-06-25 12:03:10 +00:00
}
2022-07-29 16:30:50 +00:00
if (kind == JoinKind::Cross)
2021-06-25 12:03:10 +00:00
{
joinBlockImplCross(block, not_processed);
return;
}
2021-09-09 09:47:08 +00:00
2022-07-29 16:30:50 +00:00
if (kind == JoinKind::Right || kind == JoinKind::Full)
{
2021-06-25 12:03:10 +00:00
materializeBlockInplace(block);
}
2020-04-08 18:59:52 +00:00
{
2021-09-06 10:59:18 +00:00
std::vector<const std::decay_t<decltype(data->maps[0])> * > maps_vector;
for (size_t i = 0; i < table_join->getClauses().size(); ++i)
2021-06-25 12:03:10 +00:00
maps_vector.push_back(&data->maps[i]);
if (joinDispatch(kind, strictness, maps_vector, [&](auto kind_, auto strictness_, auto & maps_vector_)
2019-01-14 21:40:02 +00:00
{
2021-09-06 10:59:18 +00:00
joinBlockImpl<kind_, strictness_>(block, sample_block_with_columns_to_add, maps_vector_);
}))
{
/// Joined
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong JOIN combination: {} {}", strictness, kind);
2019-01-14 21:40:02 +00:00
}
}
HashJoin::~HashJoin()
{
if (!data)
{
2023-01-23 12:34:36 +00:00
LOG_TRACE(log, "({}) Join data has been already released", fmt::ptr(this));
return;
}
2023-01-23 12:34:36 +00:00
LOG_TRACE(log, "({}) Join data is being destroyed, {} bytes and {} rows in hash table", fmt::ptr(this), getTotalByteCount(), getTotalRowCount());
}
template <typename Mapped>
2019-12-02 18:07:27 +00:00
struct AdderNonJoined
{
static void add(const Mapped & mapped, size_t & rows_added, MutableColumns & columns_right)
{
constexpr bool mapped_asof = std::is_same_v<Mapped, AsofRowRefs>;
[[maybe_unused]] constexpr bool mapped_one = std::is_same_v<Mapped, RowRef>;
2019-12-02 18:07:27 +00:00
if constexpr (mapped_asof)
{
2019-12-02 18:07:27 +00:00
/// Do nothing
}
2019-12-02 18:07:27 +00:00
else if constexpr (mapped_one)
{
for (size_t j = 0; j < columns_right.size(); ++j)
{
2019-12-02 18:07:27 +00:00
const auto & mapped_column = mapped.block->getByPosition(j).column;
columns_right[j]->insertFrom(*mapped_column, mapped.row_num);
}
++rows_added;
}
2019-12-02 18:07:27 +00:00
else
{
for (auto it = mapped.begin(); it.ok(); ++it)
{
for (size_t j = 0; j < columns_right.size(); ++j)
{
const auto & mapped_column = it->block->getByPosition(j).column;
columns_right[j]->insertFrom(*mapped_column, it->row_num);
}
2019-12-02 18:07:27 +00:00
++rows_added;
}
}
}
};
2017-04-02 17:37:49 +00:00
/// Stream from not joined earlier rows of the right table.
2021-09-06 10:59:18 +00:00
/// Based on:
/// - map offsetInternal saved in used_flags for single disjuncts
/// - flags in BlockWithFlags for multiple disjuncts
template <bool multiple_disjuncts>
class NotJoinedHash final : public NotJoinedBlocks::RightColumnsFiller
{
public:
NotJoinedHash(const HashJoin & parent_, UInt64 max_block_size_)
2021-11-08 12:44:13 +00:00
: parent(parent_), max_block_size(max_block_size_), current_block_start(0)
2022-12-20 12:50:27 +00:00
{
if (parent.data == nullptr)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot join after data has been released");
}
Block getEmptyBlock() override { return parent.savedBlockSample().cloneEmpty(); }
2021-08-06 14:15:11 +00:00
size_t fillColumns(MutableColumns & columns_right) override
{
size_t rows_added = 0;
2021-11-08 12:44:13 +00:00
if (unlikely(parent.data->type == HashJoin::Type::EMPTY))
{
rows_added = fillColumnsFromData(parent.data->blocks, columns_right);
}
else
{
auto fill_callback = [&](auto, auto, auto & map)
2021-11-08 12:44:13 +00:00
{
rows_added = fillColumnsFromMap(map, columns_right);
2021-11-08 12:44:13 +00:00
};
2021-11-08 12:44:13 +00:00
if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps.front(), fill_callback))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown JOIN strictness '{}' (must be on of: ANY, ALL, ASOF)", parent.strictness);
}
2021-06-25 12:03:10 +00:00
if constexpr (!multiple_disjuncts)
{
fillNullsFromBlocks(columns_right, rows_added);
}
2021-08-06 14:15:11 +00:00
return rows_added;
}
2021-08-06 11:54:04 +00:00
2021-08-06 14:15:11 +00:00
private:
const HashJoin & parent;
UInt64 max_block_size;
2021-08-06 11:54:04 +00:00
2021-11-08 12:44:13 +00:00
size_t current_block_start;
2021-08-06 14:15:11 +00:00
std::any position;
std::optional<HashJoin::BlockNullmapList::const_iterator> nulls_position;
std::optional<BlocksList::const_iterator> used_position;
2021-11-08 12:44:13 +00:00
size_t fillColumnsFromData(const BlocksList & blocks, MutableColumns & columns_right)
{
if (!position.has_value())
position = std::make_any<BlocksList::const_iterator>(blocks.begin());
auto & block_it = std::any_cast<BlocksList::const_iterator &>(position);
auto end = blocks.end();
size_t rows_added = 0;
for (; block_it != end; ++block_it)
{
size_t rows_from_block = std::min<size_t>(max_block_size - rows_added, block_it->rows() - current_block_start);
for (size_t j = 0; j < columns_right.size(); ++j)
{
const auto & col = block_it->getByPosition(j).column;
columns_right[j]->insertRangeFrom(*col, current_block_start, rows_from_block);
}
rows_added += rows_from_block;
if (rows_added >= max_block_size)
{
/// How many rows have been read
current_block_start += rows_from_block;
if (block_it->rows() <= current_block_start)
{
/// current block was fully read
++block_it;
current_block_start = 0;
}
break;
}
current_block_start = 0;
}
return rows_added;
}
template <typename Maps>
2019-07-03 19:06:34 +00:00
size_t fillColumnsFromMap(const Maps & maps, MutableColumns & columns_keys_and_right)
{
2019-12-19 15:50:28 +00:00
switch (parent.data->type)
2019-07-03 19:06:34 +00:00
{
#define M(TYPE) \
case HashJoin::Type::TYPE: \
return fillColumns(*maps.TYPE, columns_keys_and_right);
2019-07-03 19:06:34 +00:00
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
default:
2023-12-11 12:33:11 +00:00
throw Exception(ErrorCodes::UNSUPPORTED_JOIN_KEYS, "Unsupported JOIN keys (type: {})", parent.data->type);
2019-07-03 19:06:34 +00:00
}
UNREACHABLE();
2019-07-03 19:06:34 +00:00
}
template <typename Map>
size_t fillColumns(const Map & map, MutableColumns & columns_keys_and_right)
{
size_t rows_added = 0;
2021-06-25 12:03:10 +00:00
if constexpr (multiple_disjuncts)
{
if (!used_position.has_value())
used_position = parent.data->blocks.begin();
2021-06-25 12:03:10 +00:00
auto end = parent.data->blocks.end();
2021-06-25 12:03:10 +00:00
for (auto & it = *used_position; it != end && rows_added < max_block_size; ++it)
{
const Block & mapped_block = *it;
2021-06-25 12:03:10 +00:00
for (size_t row = 0; row < mapped_block.rows(); ++row)
2021-06-25 12:03:10 +00:00
{
if (!parent.isUsed(&mapped_block, row))
2021-06-25 12:03:10 +00:00
{
for (size_t colnum = 0; colnum < columns_keys_and_right.size(); ++colnum)
{
columns_keys_and_right[colnum]->insertFrom(*mapped_block.getByPosition(colnum).column, row);
2021-06-25 12:03:10 +00:00
}
++rows_added;
}
}
}
}
else
{
2021-06-25 12:03:10 +00:00
using Mapped = typename Map::mapped_type;
using Iterator = typename Map::const_iterator;
2019-12-02 18:07:27 +00:00
2021-06-25 12:03:10 +00:00
if (!position.has_value())
position = std::make_any<Iterator>(map.begin());
Iterator & it = std::any_cast<Iterator &>(position);
auto end = map.end();
2021-06-25 12:03:10 +00:00
for (; it != end; ++it)
{
2021-06-25 12:03:10 +00:00
const Mapped & mapped = it->getMapped();
2023-12-11 12:33:11 +00:00
size_t offset = map.offsetInternal(it.getPtr());
if (parent.isUsed(offset))
2021-06-25 12:03:10 +00:00
continue;
AdderNonJoined<Mapped>::add(mapped, rows_added, columns_keys_and_right);
if (rows_added >= max_block_size)
{
++it;
break;
}
}
}
return rows_added;
}
2019-07-03 19:06:34 +00:00
void fillNullsFromBlocks(MutableColumns & columns_keys_and_right, size_t & rows_added)
{
2019-07-04 12:12:39 +00:00
if (!nulls_position.has_value())
2019-12-19 15:50:28 +00:00
nulls_position = parent.data->blocks_nullmaps.begin();
2019-07-03 19:06:34 +00:00
2019-12-19 15:50:28 +00:00
auto end = parent.data->blocks_nullmaps.end();
2019-07-03 19:06:34 +00:00
2019-07-04 12:12:39 +00:00
for (auto & it = *nulls_position; it != end && rows_added < max_block_size; ++it)
2019-07-03 19:06:34 +00:00
{
const auto * block = it->first;
ConstNullMapPtr nullmap = nullptr;
if (it->second)
nullmap = &assert_cast<const ColumnUInt8 &>(*it->second).getData();
2019-07-03 19:06:34 +00:00
for (size_t row = 0; row < block->rows(); ++row)
2019-07-03 19:06:34 +00:00
{
if (nullmap && (*nullmap)[row])
2019-07-03 19:06:34 +00:00
{
for (size_t col = 0; col < columns_keys_and_right.size(); ++col)
columns_keys_and_right[col]->insertFrom(*block->getByPosition(col).column, row);
++rows_added;
}
}
}
}
};
2022-10-18 11:43:01 +00:00
IBlocksStreamPtr HashJoin::getNonJoinedBlocks(const Block & left_sample_block,
2021-08-23 11:09:33 +00:00
const Block & result_sample_block,
UInt64 max_block_size) const
{
if (!JoinCommon::hasNonJoinedBlocks(*table_join))
2019-12-02 18:07:27 +00:00
return {};
2021-09-06 10:59:18 +00:00
bool multiple_disjuncts = !table_join->oneDisjunct();
2021-06-25 12:03:10 +00:00
if (multiple_disjuncts)
{
/// ... calculate `left_columns_count` ...
2021-08-23 11:09:33 +00:00
size_t left_columns_count = left_sample_block.columns();
2021-06-25 12:03:10 +00:00
auto non_joined = std::make_unique<NotJoinedHash<true>>(*this, max_block_size);
return std::make_unique<NotJoinedBlocks>(std::move(non_joined), result_sample_block, left_columns_count, *table_join);
2021-06-25 12:03:10 +00:00
}
else
{
2021-08-23 11:09:33 +00:00
size_t left_columns_count = left_sample_block.columns();
assert(left_columns_count == result_sample_block.columns() - required_right_keys.columns() - sample_block_with_columns_to_add.columns());
2021-06-25 12:03:10 +00:00
auto non_joined = std::make_unique<NotJoinedHash<false>>(*this, max_block_size);
return std::make_unique<NotJoinedBlocks>(std::move(non_joined), result_sample_block, left_columns_count, *table_join);
2021-06-25 12:03:10 +00:00
}
}
void HashJoin::reuseJoinedData(const HashJoin & join)
{
data = join.data;
2021-04-28 17:32:12 +00:00
from_storage_join = true;
2021-08-09 15:58:41 +00:00
2021-09-06 10:59:18 +00:00
bool multiple_disjuncts = !table_join->oneDisjunct();
2021-08-09 15:58:41 +00:00
if (multiple_disjuncts)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "StorageJoin with ORs is not supported");
2021-08-09 15:58:41 +00:00
2021-06-25 12:03:10 +00:00
for (auto & map : data->maps)
{
2021-06-25 12:03:10 +00:00
joinDispatch(kind, strictness, map, [this](auto kind_, auto strictness_, auto & map_)
{
used_flags.reinit<kind_, strictness_>(map_.getBufferSizeInCells(data->type) + 1);
});
}
}
2021-09-09 09:47:08 +00:00
2022-12-20 12:50:27 +00:00
BlocksList HashJoin::releaseJoinedBlocks(bool restructure)
2022-09-27 12:33:09 +00:00
{
2023-01-23 12:34:36 +00:00
LOG_TRACE(log, "({}) Join data is being released, {} bytes and {} rows in hash table", fmt::ptr(this), getTotalByteCount(), getTotalRowCount());
2022-12-27 10:13:12 +00:00
2022-10-18 12:45:06 +00:00
BlocksList right_blocks = std::move(data->blocks);
2022-12-20 12:50:27 +00:00
if (!restructure)
{
data.reset();
return right_blocks;
}
data->maps.clear();
data->blocks_nullmaps.clear();
2022-06-16 12:09:23 +00:00
BlocksList restored_blocks;
/// names to positions optimization
std::vector<size_t> positions;
std::vector<bool> is_nullable;
if (!right_blocks.empty())
{
positions.reserve(right_sample_block.columns());
const Block & tmp_block = *right_blocks.begin();
for (const auto & sample_column : right_sample_block)
{
positions.emplace_back(tmp_block.getPositionByName(sample_column.name));
is_nullable.emplace_back(isNullableOrLowCardinalityNullable(sample_column.type));
2022-06-16 12:09:23 +00:00
}
}
for (Block & saved_block : right_blocks)
{
Block restored_block;
for (size_t i = 0; i < positions.size(); ++i)
{
auto & column = saved_block.getByPosition(positions[i]);
correctNullabilityInplace(column, is_nullable[i]);
restored_block.insert(column);
2022-06-16 12:09:23 +00:00
}
restored_blocks.emplace_back(std::move(restored_block));
}
2022-12-20 12:50:27 +00:00
data.reset();
2022-06-16 12:09:23 +00:00
return restored_blocks;
}
2021-09-09 09:47:08 +00:00
const ColumnWithTypeAndName & HashJoin::rightAsofKeyColumn() const
{
/// It should be nullable when right side is nullable
2021-09-09 09:47:08 +00:00
return savedBlockSample().getByName(table_join->getOnlyClause().key_names_right.back());
}
2014-06-12 02:31:30 +00:00
}