ClickHouse/src/Interpreters/HashJoin.cpp

1512 lines
55 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>
2019-07-04 12:12:39 +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 <DataTypes/DataTypeNullable.h>
2020-04-08 18:59:52 +00:00
#include <DataTypes/DataTypeLowCardinality.h>
2014-06-12 02:31:30 +00:00
#include <Interpreters/HashJoin.h>
#include <Interpreters/join_common.h>
#include <Interpreters/TableJoin.h>
#include <Interpreters/joinDispatch.h>
#include <Interpreters/NullableUtils.h>
2020-04-08 18:59:52 +00:00
#include <Interpreters/DictionaryReader.h>
#include <Storages/StorageDictionary.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataStreams/materializeBlock.h>
#include <Core/ColumnNumbers.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
#include <Common/assert_cast.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;
extern const int LOGICAL_ERROR;
extern const int SYNTAX_ERROR;
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;
};
}
namespace JoinStuff
{
/// Version of `getUsed` with dynamic dispatch
bool JoinUsedFlags::getUsedSafe(size_t i) const
{
if (flags.empty())
return !need_flags;
return flags[i].load();
}
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS>
void JoinUsedFlags::reinit(size_t size)
{
if constexpr (MapGetter<KIND, STRICTNESS>::flagged)
{
assert(flags.size() <= size);
need_flags = true;
flags = std::vector<std::atomic_bool>(size);
}
}
template <>
void JoinUsedFlags::setUsed<false>(size_t i [[maybe_unused]]) {}
template <>
bool JoinUsedFlags::getUsed<false>(size_t i [[maybe_unused]]) { return true; }
template <>
bool JoinUsedFlags::setUsedOnce<false>(size_t i [[maybe_unused]]) { return true; }
template <>
void JoinUsedFlags::setUsed<true>(size_t i)
{
/// Could be set simultaneously from different threads.
flags[i].store(true, std::memory_order_relaxed);
}
template <>
bool JoinUsedFlags::getUsed<true>(size_t i) { return flags[i].load(); }
template <>
bool JoinUsedFlags::setUsedOnce<true>(size_t i)
{
/// fast check to prevent heavy CAS with seq_cst order
if (flags[i].load(std::memory_order_relaxed))
return false;
bool expected = false;
return flags[i].compare_exchange_strong(expected, true);
}
}
2019-10-29 19:39:42 +00:00
static ColumnPtr filterWithBlanks(ColumnPtr src_column, const IColumn::Filter & filter, bool inverse_filter = false)
{
ColumnPtr column = src_column->convertToFullColumnIfConst();
MutableColumnPtr mut_column = column->cloneEmpty();
mut_column->reserve(column->size());
if (inverse_filter)
{
for (size_t row = 0; row < filter.size(); ++row)
{
if (filter[row])
mut_column->insertDefault();
else
mut_column->insertFrom(*column, row);
}
}
else
{
for (size_t row = 0; row < filter.size(); ++row)
{
if (filter[row])
mut_column->insertFrom(*column, row);
else
mut_column->insertDefault();
}
}
return mut_column;
}
static ColumnWithTypeAndName correctNullability(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))
2019-10-29 19:39:42 +00:00
column.column = filterWithBlanks(column.column, nullable_column->getNullMapColumn().getData(), true);
JoinCommon::removeColumnNullability(column);
}
return std::move(column);
}
static ColumnWithTypeAndName correctNullability(ColumnWithTypeAndName && column, bool nullable, const ColumnUInt8 & negative_null_map)
{
if (nullable)
{
JoinCommon::convertColumnToNullable(column, true);
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);
return std::move(column);
}
HashJoin::HashJoin(std::shared_ptr<TableJoin> table_join_, const Block & right_sample_block_, bool any_take_last_row_)
: table_join(table_join_)
, kind(table_join->kind())
, strictness(table_join->strictness())
, key_names_right(table_join->keyNamesRight())
, nullable_right_side(table_join->forceNullableRight())
, nullable_left_side(table_join->forceNullableLeft())
, 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"))
{
LOG_DEBUG(log, "Right sample block: {}", right_sample_block.dumpStructure());
table_join->splitAdditionalColumns(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);
JoinCommon::removeLowCardinalityInplace(right_table_keys);
initRightBlockStructure(data->sample_block);
ColumnRawPtrs key_columns = JoinCommon::extractKeysForJoin(right_table_keys, key_names_right);
JoinCommon::createMissedColumns(sample_block_with_columns_to_add);
if (nullable_right_side)
JoinCommon::convertColumnsToNullable(sample_block_with_columns_to_add);
if (table_join->dictionary_reader)
{
data->type = Type::DICT;
std::get<MapsOne>(data->maps).create(Type::DICT);
chooseMethod(key_columns, key_sizes); /// init key_sizes
}
else if (strictness == ASTTableJoin::Strictness::Asof)
{
/// @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("Wrong ASOF JOIN type. Only ASOF and LEFT ASOF joins are supported", ErrorCodes::NOT_IMPLEMENTED);
if (key_columns.size() <= 1)
throw Exception("ASOF join needs at least one equi-join column", ErrorCodes::SYNTAX_ERROR);
if (right_table_keys.getByName(key_names_right.back()).type->isNullable())
throw Exception("ASOF join over right table Nullable column is not implemented", ErrorCodes::NOT_IMPLEMENTED);
size_t asof_size;
asof_type = AsofRowRefs::getTypeSize(*key_columns.back(), asof_size);
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
init(chooseMethod(key_columns, key_sizes));
key_sizes.push_back(asof_size);
}
else
{
/// Choose data structure to use for JOIN.
init(chooseMethod(key_columns, key_sizes));
}
}
HashJoin::Type HashJoin::chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes)
2015-03-02 01:10:58 +00:00
{
size_t keys_size = key_columns.size();
if (keys_size == 0)
return Type::CROSS;
bool all_fixed = true;
size_t keys_bytes = 0;
key_sizes.resize(keys_size);
for (size_t j = 0; j < keys_size; ++j)
{
if (!key_columns[j]->isFixedAndContiguous())
{
all_fixed = false;
break;
}
key_sizes[j] = key_columns[j]->sizeOfValueIfFixed();
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("Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.", ErrorCodes::LOGICAL_ERROR);
}
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.
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
if (keys_size == 1
&& (typeid_cast<const ColumnString *>(key_columns[0])
|| (isColumnConst(*key_columns[0]) && typeid_cast<const ColumnString *>(&assert_cast<const ColumnConst *>(key_columns[0])->getDataColumn()))))
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
}
2019-12-02 18:07:27 +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);
}
2020-04-08 18:59:52 +00:00
class KeyGetterForDict
{
public:
using Mapped = RowRef;
using FindResult = ColumnsHashing::columns_hashing_impl::FindResultImpl<Mapped, true>;
2020-04-08 18:59:52 +00:00
2020-04-10 11:03:08 +00:00
KeyGetterForDict(const ColumnRawPtrs & key_columns_, const Sizes &, void *)
2020-04-08 18:59:52 +00:00
: key_columns(key_columns_)
{}
2020-04-13 17:03:11 +00:00
FindResult findKey(const TableJoin & table_join, size_t row, const Arena &)
2020-04-08 18:59:52 +00:00
{
2020-04-13 17:03:11 +00:00
const DictionaryReader & reader = *table_join.dictionary_reader;
2020-04-08 18:59:52 +00:00
if (!read_result)
{
reader.readKeys(*key_columns[0], read_result, found, positions);
2020-04-08 18:59:52 +00:00
result.block = &read_result;
2020-04-13 17:03:11 +00:00
if (table_join.forceNullableRight())
for (auto & column : read_result)
if (table_join.rightBecomeNullable(column.type))
JoinCommon::convertColumnToNullable(column);
2020-04-08 18:59:52 +00:00
}
2020-04-13 17:03:11 +00:00
result.row_num = positions[row];
return FindResult(&result, found[row], 0);
2020-04-08 18:59:52 +00:00
}
private:
const ColumnRawPtrs & key_columns;
Block read_result;
Mapped result;
ColumnVector<UInt8>::Container found;
std::vector<size_t> positions;
};
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
{
using Type = ColumnsHashing::HashMethodKeysFixed<Value, DummyUInt256, 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;
};
void HashJoin::init(Type type_)
{
2019-12-19 15:50:28 +00:00
data->type = type_;
if (kind == ASTTableJoin::Kind::Cross)
return;
2019-12-19 15:50:28 +00:00
joinDispatchInit(kind, strictness, data->maps);
joinDispatch(kind, strictness, data->maps, [&](auto, auto, auto & map) { map.create(data->type); });
}
2021-01-04 23:49:31 +00:00
bool HashJoin::overDictionary() const
{
return data->type == Type::DICT;
}
bool HashJoin::empty() const
{
return data->type == Type::EMPTY;
}
bool HashJoin::alwaysReturnsEmptySet() const
2020-12-30 13:52:37 +00:00
{
2021-01-04 23:49:31 +00:00
return isInnerOrRight(getKind()) && data->empty && !overDictionary();
2020-12-30 13:52:37 +00:00
}
size_t HashJoin::getTotalRowCount() const
2020-12-30 13:52:37 +00:00
{
size_t res = 0;
2019-12-19 15:50:28 +00:00
if (data->type == Type::CROSS)
{
2019-12-19 15:50:28 +00:00
for (const auto & block : data->blocks)
res += block.rows();
}
2020-04-08 18:59:52 +00:00
else if (data->type != Type::DICT)
{
2019-12-19 15:50:28 +00:00
joinDispatch(kind, strictness, data->maps, [&](auto, auto, auto & map) { res += map.getTotalRowCount(data->type); });
}
return res;
}
size_t HashJoin::getTotalByteCount() const
{
size_t res = 0;
2019-12-19 15:50:28 +00:00
if (data->type == Type::CROSS)
{
2019-12-19 15:50:28 +00:00
for (const auto & block : data->blocks)
res += block.bytes();
}
2020-04-08 18:59:52 +00:00
else if (data->type != Type::DICT)
{
2019-12-19 15:50:28 +00:00
joinDispatch(kind, strictness, data->maps, [&](auto, auto, auto & map) { res += map.getTotalByteCountImpl(data->type); });
res += data->pool.size();
}
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 void 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);
}
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);
typename Map::mapped_type * time_series_map = &emplace_result.getMapped();
TypeIndex asof_type = *join.getAsofType();
if (emplace_result.isInserted())
time_series_map = new (time_series_map) typename Map::mapped_type(asof_type);
time_series_map->insert(asof_type, asof_column, stored_block, i);
}
};
template <ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map, bool has_null_map>
size_t NO_INLINE insertFromBlockImplTypeCase(
HashJoin & join, Map & map, size_t rows, const ColumnRawPtrs & key_columns,
2019-01-24 14:56:04 +00:00
const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool)
{
[[maybe_unused]] constexpr bool mapped_one = std::is_same_v<typename Map::mapped_type, RowRef>;
2019-12-02 18:07:27 +00:00
constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof;
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 (size_t i = 0; i < rows; ++i)
{
if (has_null_map && (*null_map)[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)
Inserter<Map, KeyGetter>::insertOne(join, map, key_getter, stored_block, i, pool);
else
2019-12-02 18:07:27 +00:00
Inserter<Map, KeyGetter>::insertAll(join, map, key_getter, stored_block, i, pool);
}
return map.getBufferSizeInCells();
}
template <ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map>
size_t insertFromBlockImplType(
HashJoin & join, Map & map, size_t rows, const ColumnRawPtrs & key_columns,
2019-01-24 14:56:04 +00:00
const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool)
{
if (null_map)
return insertFromBlockImplTypeCase<STRICTNESS, KeyGetter, Map, true>(join, map, rows, key_columns, key_sizes, stored_block, null_map, pool);
else
return insertFromBlockImplTypeCase<STRICTNESS, KeyGetter, Map, false>(join, map, rows, key_columns, key_sizes, stored_block, null_map, pool);
}
template <ASTTableJoin::Strictness STRICTNESS, typename Maps>
size_t insertFromBlockImpl(
HashJoin & join, HashJoin::Type type, Maps & maps, size_t rows, const ColumnRawPtrs & key_columns,
2019-01-24 14:56:04 +00:00
const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool)
{
switch (type)
{
case HashJoin::Type::EMPTY: return 0;
case HashJoin::Type::CROSS: return 0; /// Do nothing. We have already saved block, and it is enough.
case HashJoin::Type::DICT: return 0; /// No one should call it with Type::DICT.
#define M(TYPE) \
case HashJoin::Type::TYPE: \
return insertFromBlockImplType<STRICTNESS, typename KeyGetterForType<HashJoin::Type::TYPE, std::remove_reference_t<decltype(*maps.TYPE)>>::Type>(\
2019-03-30 01:32:08 +00:00
join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, pool); \
break;
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
}
__builtin_unreachable();
}
}
void HashJoin::initRightBlockStructure(Block & saved_block_sample)
{
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).
bool save_key_columns = !table_join->forceHashJoin() || isRightOrFull(kind);
if (save_key_columns)
2019-03-31 23:09:00 +00:00
{
saved_block_sample = right_table_keys.cloneEmpty();
}
else if (strictness == ASTTableJoin::Strictness::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)
saved_block_sample.insert(column);
2019-03-31 23:09:00 +00:00
if (nullable_right_side)
JoinCommon::convertColumnsToNullable(saved_block_sample, (isFull(kind) ? right_table_keys.columns() : 0));
}
2019-03-31 23:09:00 +00:00
Block HashJoin::structureRightBlock(const Block & block) const
{
Block structured_block;
2020-04-22 06:01:33 +00:00
for (const auto & sample_column : savedBlockSample().getColumnsWithTypeAndName())
{
ColumnWithTypeAndName column = block.getByName(sample_column.name);
if (sample_column.column->isNullable())
JoinCommon::convertColumnToNullable(column);
structured_block.insert(column);
2019-03-31 23:09:00 +00:00
}
return structured_block;
2019-03-31 23:09:00 +00:00
}
bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits)
{
if (empty())
throw Exception("Logical error: HashJoin was not initialized", ErrorCodes::LOGICAL_ERROR);
2020-04-08 18:59:52 +00:00
if (overDictionary())
throw Exception("Logical error: insert into hash-map in HashJoin over dictionary", ErrorCodes::LOGICAL_ERROR);
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("Too many rows in right table block for HashJoin: " + toString(source_block.rows()), ErrorCodes::NOT_IMPLEMENTED);
/// There's no optimization for right side const columns. Remove constness if any.
Block block = materializeBlock(source_block);
size_t rows = block.rows();
ColumnRawPtrs key_columns = JoinCommon::materializeColumnsInplace(block, key_names_right);
/// We will insert to the map only keys, where all components are not NULL.
ConstNullMapPtr null_map{};
2019-07-03 19:06:34 +00:00
ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map);
2019-12-03 14:30:51 +00:00
/// If RIGHT or FULL save blocks with nulls for NonJoinedBlockInputStream
UInt8 save_nullmap = 0;
if (isRightOrFull(kind) && null_map)
{
2019-12-03 14:30:51 +00:00
for (size_t i = 0; !save_nullmap && i < null_map->size(); ++i)
save_nullmap |= (*null_map)[i];
}
Block structured_block = structureRightBlock(block);
2019-12-03 14:30:51 +00:00
size_t total_rows = 0;
size_t total_bytes = 0;
{
assert(storage_join_lock.mutex() == nullptr);
2019-12-19 15:50:28 +00:00
data->blocks.emplace_back(std::move(structured_block));
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
2019-12-03 14:30:51 +00:00
if (kind != ASTTableJoin::Kind::Cross)
{
joinDispatch(kind, strictness, data->maps, [&](auto kind_, auto strictness_, auto & map)
2019-12-03 14:30:51 +00:00
{
size_t size = insertFromBlockImpl<strictness_>(*this, data->type, map, rows, key_columns, key_sizes, stored_block, null_map, data->pool);
/// Number of buckets + 1 value from zero storage
used_flags.reinit<kind_, strictness_>(size + 1);
2019-12-03 14:30:51 +00:00
});
}
2019-12-03 14:30:51 +00:00
if (save_nullmap)
2019-12-19 15:50:28 +00:00
data->blocks_nullmaps.emplace_back(stored_block, null_map_holder);
2019-12-03 14:30:51 +00:00
if (!check_limits)
return true;
2019-12-03 14:30:51 +00:00
/// TODO: Do not calculate them every time
total_rows = getTotalRowCount();
total_bytes = getTotalByteCount();
}
2019-07-03 19:06:34 +00:00
2019-12-03 14:30:51 +00:00
return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
2014-06-12 02:31:30 +00:00
}
namespace
{
2019-03-20 14:49:05 +00:00
class AddedColumns
{
public:
using TypeAndNames = std::vector<std::pair<decltype(ColumnWithTypeAndName::type), decltype(ColumnWithTypeAndName::name)>>;
AddedColumns(const Block & block_with_columns_to_add,
2019-03-31 23:09:00 +00:00
const Block & block,
const Block & saved_block_sample,
const HashJoin & join,
2019-11-06 19:39:52 +00:00
const ColumnRawPtrs & key_columns_,
const Sizes & key_sizes_,
bool is_asof_join)
: key_columns(key_columns_)
2019-11-06 19:39:52 +00:00
, key_sizes(key_sizes_)
, rows_to_add(block.rows())
, asof_type(join.getAsofType())
, asof_inequality(join.getAsofInequality())
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
{
2020-04-07 14:52:32 +00:00
/// Don't insert column if it's in left block
if (!block.has(src_column.name))
addColumn(src_column);
}
if (is_asof_join)
{
const ColumnWithTypeAndName & right_asof_column = join.rightAsofKeyColumn();
addColumn(right_asof_column);
left_asof_key = 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.second));
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].first, type_name[i].second);
}
2019-11-08 16:13:43 +00:00
template <bool has_defaults>
2019-03-20 14:49:05 +00:00
void appendFromBlock(const Block & block, size_t row_num)
{
2019-11-08 16:13:43 +00:00
if constexpr (has_defaults)
applyLazyDefaults();
2021-01-28 08:26:10 +00:00
for (size_t j = 0, size = right_indexes.size(); j < size; ++j)
2019-03-20 14:49:05 +00:00
columns[j]->insertFrom(*block.getByPosition(right_indexes[j]).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].first, lazy_defaults_count);
2019-11-06 19:39:52 +00:00
lazy_defaults_count = 0;
}
}
TypeIndex asofType() const { return *asof_type; }
ASOF::Inequality asofInequality() const { return asof_inequality; }
const IColumn & leftAsofKey() const { return *left_asof_key; }
2019-11-06 19:39:52 +00:00
const ColumnRawPtrs & key_columns;
const Sizes & key_sizes;
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:
TypeAndNames 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
std::optional<TypeIndex> asof_type;
ASOF::Inequality asof_inequality;
const IColumn * left_asof_key = nullptr;
2019-03-31 23:09:00 +00:00
void addColumn(const ColumnWithTypeAndName & src_column)
{
columns.push_back(src_column.column->cloneEmpty());
columns.back()->reserve(src_column.column->size());
type_name.emplace_back(src_column.type, src_column.name);
}
2019-03-20 15:15:44 +00:00
};
2019-11-08 16:13:43 +00:00
template <typename Map, bool add_missing>
2019-11-06 19:39:52 +00:00
void addFoundRowAll(const typename Map::mapped_type & mapped, AddedColumns & added, IColumn::Offset & current_offset)
2019-03-20 14:49:05 +00:00
{
if constexpr (add_missing)
added.applyLazyDefaults();
2019-11-06 19:39:52 +00:00
for (auto it = mapped.begin(); it.ok(); ++it)
2019-03-19 16:53:36 +00:00
{
added.appendFromBlock<false>(*it->block, it->row_num);
2019-11-06 19:39:52 +00:00
++current_offset;
2019-03-19 16:53:36 +00:00
}
2019-03-20 14:49:05 +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 <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map, bool need_filter, bool has_null_map>
NO_INLINE IColumn::Filter joinRightColumns(
const Map & map,
AddedColumns & added_columns,
const ConstNullMapPtr & null_map [[maybe_unused]],
JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]])
2019-03-20 14:49:05 +00:00
{
constexpr bool is_any_join = STRICTNESS == ASTTableJoin::Strictness::Any;
constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All;
constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof;
2019-12-02 18:07:27 +00:00
constexpr bool is_semi_join = STRICTNESS == ASTTableJoin::Strictness::Semi;
constexpr bool is_anti_join = STRICTNESS == ASTTableJoin::Strictness::Anti;
constexpr bool left = KIND == ASTTableJoin::Kind::Left;
constexpr bool right = KIND == ASTTableJoin::Kind::Right;
2019-12-02 18:07:27 +00:00
constexpr bool full = KIND == ASTTableJoin::Kind::Full;
constexpr bool need_flags = MapGetter<KIND, STRICTNESS>::flagged;
2019-12-02 18:07:27 +00:00
constexpr bool add_missing = (left || full) && !is_semi_join;
constexpr bool need_replication = is_all_join || (is_any_join && right) || (is_semi_join && right);
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;
2019-11-11 11:42:10 +00:00
if constexpr (need_replication)
added_columns.offsets_to_replicate = std::make_unique<IColumn::Offsets>(rows);
2019-12-02 18:07:27 +00:00
auto key_getter = createKeyGetter<KeyGetter, is_asof_join>(added_columns.key_columns, added_columns.key_sizes);
IColumn::Offset current_offset = 0;
2019-03-20 14:49:05 +00:00
for (size_t i = 0; i < rows; ++i)
{
if constexpr (has_null_map)
{
2019-11-11 11:42:10 +00:00
if ((*null_map)[i])
{
addNotFoundRow<add_missing, need_replication>(added_columns, current_offset);
2019-11-11 11:42:10 +00:00
if constexpr (need_replication)
(*added_columns.offsets_to_replicate)[i] = current_offset;
continue;
}
2019-03-20 14:49:05 +00:00
}
2019-11-11 11:42:10 +00:00
auto find_result = key_getter.findKey(map, i, pool);
if (find_result.isFound())
2019-03-20 14:49:05 +00:00
{
2019-11-11 11:42:10 +00:00
auto & mapped = find_result.getMapped();
2019-03-20 14:49:05 +00:00
2019-11-11 11:42:10 +00:00
if constexpr (is_asof_join)
{
TypeIndex asof_type = added_columns.asofType();
ASOF::Inequality asof_inequality = added_columns.asofInequality();
const IColumn & left_asof_key = added_columns.leftAsofKey();
if (const RowRef * found = mapped.findAsof(asof_type, asof_inequality, left_asof_key, i))
2019-03-30 21:30:21 +00:00
{
setUsed<need_filter>(filter, i);
used_flags.template setUsed<need_flags>(find_result.getOffset());
2019-11-11 11:42:10 +00:00
added_columns.appendFromBlock<add_missing>(*found->block, found->row_num);
2019-03-30 21:30:21 +00:00
}
else
addNotFoundRow<add_missing, need_replication>(added_columns, current_offset);
2019-11-11 11:42:10 +00:00
}
else if constexpr (is_all_join)
{
setUsed<need_filter>(filter, i);
used_flags.template setUsed<need_flags>(find_result.getOffset());
2019-11-11 11:42:10 +00:00
addFoundRowAll<Map, add_missing>(mapped, added_columns, current_offset);
}
2019-12-02 18:07:27 +00:00
else if constexpr ((is_any_join || is_semi_join) && right)
2019-11-11 11:42:10 +00:00
{
2020-01-11 09:50:41 +00:00
/// Use first appeared left key + it needs left columns replication
bool used_once = used_flags.template setUsedOnce<need_flags>(find_result.getOffset());
if (used_once)
{
setUsed<need_filter>(filter, i);
2019-11-11 11:42:10 +00:00
addFoundRowAll<Map, add_missing>(mapped, added_columns, current_offset);
}
}
2019-11-11 11:42:10 +00:00
else if constexpr (is_any_join && KIND == ASTTableJoin::Kind::Inner)
{
bool used_once = used_flags.template setUsedOnce<need_flags>(find_result.getOffset());
2020-01-11 09:50:41 +00:00
/// Use first appeared left key only
if (used_once)
{
setUsed<need_filter>(filter, i);
2019-11-08 16:13:43 +00:00
added_columns.appendFromBlock<add_missing>(*mapped.block, mapped.row_num);
}
}
2019-12-02 18:07:27 +00:00
else if constexpr (is_any_join && full)
2019-11-11 11:42:10 +00:00
{
/// TODO
}
2019-12-02 18:07:27 +00:00
else if constexpr (is_anti_join)
{
if constexpr (right && need_flags)
used_flags.template setUsed<need_flags>(find_result.getOffset());
2019-12-02 18:07:27 +00:00
}
else /// ANY LEFT, SEMI LEFT, old ANY (RightAny)
2019-11-11 11:42:10 +00:00
{
setUsed<need_filter>(filter, i);
used_flags.template setUsed<need_flags>(find_result.getOffset());
2019-11-11 11:42:10 +00:00
added_columns.appendFromBlock<add_missing>(*mapped.block, mapped.row_num);
}
}
2019-11-11 11:42:10 +00:00
else
2019-12-02 18:07:27 +00:00
{
if constexpr (is_anti_join && left)
setUsed<need_filter>(filter, i);
addNotFoundRow<add_missing, need_replication>(added_columns, current_offset);
}
if constexpr (need_replication)
2019-11-06 19:39:52 +00:00
(*added_columns.offsets_to_replicate)[i] = current_offset;
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
}
2019-03-20 14:49:05 +00:00
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename KeyGetter, typename Map>
IColumn::Filter joinRightColumnsSwitchNullability(
const Map & map, AddedColumns & added_columns, const ConstNullMapPtr & null_map, JoinStuff::JoinUsedFlags & used_flags)
2019-03-20 14:49:05 +00:00
{
if (added_columns.need_filter)
{
if (null_map)
return joinRightColumns<KIND, STRICTNESS, KeyGetter, Map, true, true>(map, added_columns, null_map, used_flags);
else
return joinRightColumns<KIND, STRICTNESS, KeyGetter, Map, true, false>(map, added_columns, nullptr, used_flags);
}
2019-03-20 14:49:05 +00:00
else
{
if (null_map)
return joinRightColumns<KIND, STRICTNESS, KeyGetter, Map, false, true>(map, added_columns, null_map, used_flags);
else
return joinRightColumns<KIND, STRICTNESS, KeyGetter, Map, false, false>(map, added_columns, nullptr, used_flags);
}
2019-03-20 14:49:05 +00:00
}
2019-03-19 16:53:36 +00:00
2019-03-20 14:49:05 +00:00
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename Maps>
IColumn::Filter switchJoinRightColumns(
const Maps & maps_, AddedColumns & added_columns, HashJoin::Type type, const ConstNullMapPtr & null_map, JoinStuff::JoinUsedFlags & used_flags)
2019-03-20 14:49:05 +00:00
{
switch (type)
{
#define M(TYPE) \
case HashJoin::Type::TYPE: \
2019-11-11 11:42:10 +00:00
return joinRightColumnsSwitchNullability<KIND, STRICTNESS,\
typename KeyGetterForType<HashJoin::Type::TYPE, const std::remove_reference_t<decltype(*maps_.TYPE)>>::Type>(\
*maps_.TYPE, added_columns, null_map, used_flags);
2019-03-20 14:49:05 +00:00
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
default:
2019-08-05 14:03:14 +00:00
throw Exception("Unsupported JOIN keys. Type: " + toString(static_cast<UInt32>(type)), ErrorCodes::UNSUPPORTED_JOIN_KEYS);
}
}
2020-04-08 18:59:52 +00:00
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS>
2020-04-13 17:03:11 +00:00
IColumn::Filter dictionaryJoinRightColumns(const TableJoin & table_join, AddedColumns & added_columns, const ConstNullMapPtr & null_map)
2020-04-08 18:59:52 +00:00
{
if constexpr (KIND == ASTTableJoin::Kind::Left &&
(STRICTNESS == ASTTableJoin::Strictness::Any ||
STRICTNESS == ASTTableJoin::Strictness::Semi ||
STRICTNESS == ASTTableJoin::Strictness::Anti))
{
JoinStuff::JoinUsedFlags flags;
return joinRightColumnsSwitchNullability<KIND, STRICTNESS, KeyGetterForDict>(table_join, added_columns, null_map, flags);
2020-04-08 18:59:52 +00:00
}
throw Exception("Logical error: wrong JOIN combination", ErrorCodes::LOGICAL_ERROR);
}
2019-03-20 14:49:05 +00:00
} /// nameless
template <ASTTableJoin::Kind KIND, ASTTableJoin::Strictness STRICTNESS, typename Maps>
void HashJoin::joinBlockImpl(
Block & block,
const Names & key_names_left,
const Block & block_with_columns_to_add,
const Maps & maps_) const
{
constexpr bool is_any_join = STRICTNESS == ASTTableJoin::Strictness::Any;
constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All;
constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof;
2019-12-02 18:07:27 +00:00
constexpr bool is_semi_join = STRICTNESS == ASTTableJoin::Strictness::Semi;
constexpr bool is_anti_join = STRICTNESS == ASTTableJoin::Strictness::Anti;
constexpr bool left = KIND == ASTTableJoin::Kind::Left;
constexpr bool right = KIND == ASTTableJoin::Kind::Right;
2019-12-02 18:07:27 +00:00
constexpr bool inner = KIND == ASTTableJoin::Kind::Inner;
constexpr bool full = KIND == ASTTableJoin::Kind::Full;
2019-12-02 18:07:27 +00:00
constexpr bool need_replication = is_all_join || (is_any_join && right) || (is_semi_join && right);
constexpr bool need_filter = !need_replication && (inner || right || (is_semi_join && left) || (is_anti_join && left));
/// Rare case, when keys are constant or low cardinality. To avoid code bloat, simply materialize them.
Columns materialized_keys = JoinCommon::materializeColumns(block, key_names_left);
ColumnRawPtrs left_key_columns = JoinCommon::getRawPointers(materialized_keys);
/// Keys with NULL value in any column won't join to anything.
ConstNullMapPtr null_map{};
ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(left_key_columns, null_map);
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.
*/
2019-12-02 18:07:27 +00:00
if constexpr (right || full)
{
2019-09-12 12:59:53 +00:00
materializeBlockInplace(block);
2019-09-11 18:03:21 +00:00
2019-09-12 14:09:05 +00:00
if (nullable_left_side)
2019-09-11 18:03:21 +00:00
JoinCommon::convertColumnsToNullable(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, left_key_columns, key_sizes, is_asof_join);
bool has_required_right_keys = (required_right_keys.columns() != 0);
added_columns.need_filter = need_filter || has_required_right_keys;
2019-03-20 12:08:38 +00:00
2020-04-08 18:59:52 +00:00
IColumn::Filter row_filter = overDictionary() ?
2020-04-13 17:03:11 +00:00
dictionaryJoinRightColumns<KIND, STRICTNESS>(*table_join, added_columns, null_map) :
switchJoinRightColumns<KIND, STRICTNESS>(maps_, added_columns, data->type, null_map, 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]];
if constexpr (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);
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);
const auto & left_name = required_right_keys_sources[i];
2020-11-03 11:28:28 +00:00
/// asof column is already in block.
if (is_asof_join && right_key.name == key_names_right.back())
continue;
const auto & col = block.getByName(left_name);
bool is_nullable = nullable_right_side || right_key.type->isNullable();
block.insert(correctNullability({col.column, col.type, right_key.name}, is_nullable));
}
}
else if (has_required_right_keys)
{
/// Some trash to represent IColumn::Filter as ColumnUInt8 needed for ColumnNullable::applyNullMap()
auto null_map_filter_ptr = ColumnUInt8::create();
ColumnUInt8 & null_map_filter = assert_cast<ColumnUInt8 &>(*null_map_filter_ptr);
null_map_filter.getData().swap(row_filter);
const IColumn::Filter & filter = null_map_filter.getData();
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);
const auto & left_name = required_right_keys_sources[i];
2020-11-03 11:28:28 +00:00
/// asof column is already in block.
if (is_asof_join && right_key.name == key_names_right.back())
continue;
const auto & col = block.getByName(left_name);
bool is_nullable = nullable_right_side || right_key.type->isNullable();
ColumnPtr thin_column = filterWithBlanks(col.column, filter);
block.insert(correctNullability({thin_column, col.type, right_key.name}, is_nullable, null_map_filter));
if constexpr (need_replication)
right_keys_to_replicate.push_back(block.getPositionByName(right_key.name));
}
}
if constexpr (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();
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;
{
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);
}
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)
{
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)
{
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;
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);
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);
}
}
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;
}
}
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(
"Number of arguments for function joinGet" + toString(or_null ? "OrNull" : "")
+ " doesn't match: passed, should be equal to " + toString(num_keys),
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
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);
auto left_type = removeNullable(left_type_origin);
auto right_type = removeNullable(right_type_origin);
if (!left_type->equals(*right_type))
throw Exception(
"Type mismatch in joinGet key " + toString(i) + ": found type " + left_type->getName() + ", while the needed type is "
+ right_type->getName(),
ErrorCodes::TYPE_MISMATCH);
}
if (!sample_block_with_columns_to_add.has(column_name))
2020-06-02 20:26:41 +00:00
throw Exception("StorageJoin doesn't contain column " + column_name, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
2020-04-12 03:18:21 +00:00
auto elem = sample_block_with_columns_to_add.getByName(column_name);
2020-04-07 14:52:32 +00:00
if (or_null)
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
/// TODO: return array of values when strictness == ASTTableJoin::Strictness::All
ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const
{
bool is_valid = (strictness == ASTTableJoin::Strictness::Any || strictness == ASTTableJoin::Strictness::RightAny)
&& kind == ASTTableJoin::Kind::Left;
if (!is_valid)
throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN);
/// 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));
}
static_assert(!MapGetter<ASTTableJoin::Kind::Left, ASTTableJoin::Strictness::Any>::flagged,
"joinGet are not protected from hash table changes between block processing");
joinBlockImpl<ASTTableJoin::Kind::Left, ASTTableJoin::Strictness::Any>(
keys, key_names_right, block_with_columns_to_add, std::get<MapsOne>(data->maps));
return keys.getByPosition(keys.columns() - 1);
}
void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed)
{
const Names & key_names_left = table_join->keyNamesLeft();
2019-09-11 18:03:21 +00:00
JoinCommon::checkTypesOfKeys(block, key_names_left, right_table_keys, key_names_right);
2020-04-08 18:59:52 +00:00
if (overDictionary())
{
using Kind = ASTTableJoin::Kind;
using Strictness = ASTTableJoin::Strictness;
auto & map = std::get<MapsOne>(data->maps);
if (kind == Kind::Left)
{
switch (strictness)
{
case Strictness::Any:
case Strictness::All:
joinBlockImpl<Kind::Left, Strictness::Any>(block, key_names_left, sample_block_with_columns_to_add, map);
break;
case Strictness::Semi:
joinBlockImpl<Kind::Left, Strictness::Semi>(block, key_names_left, sample_block_with_columns_to_add, map);
break;
case Strictness::Anti:
joinBlockImpl<Kind::Left, Strictness::Anti>(block, key_names_left, sample_block_with_columns_to_add, map);
break;
default:
throw Exception("Logical error: wrong JOIN combination", ErrorCodes::LOGICAL_ERROR);
}
}
else if (kind == Kind::Inner && strictness == Strictness::All)
joinBlockImpl<Kind::Left, Strictness::Semi>(block, key_names_left, sample_block_with_columns_to_add, map);
else
throw Exception("Logical error: wrong JOIN combination", ErrorCodes::LOGICAL_ERROR);
}
else if (joinDispatch(kind, strictness, data->maps, [&](auto kind_, auto strictness_, auto & map)
2019-01-14 21:40:02 +00:00
{
2019-09-10 18:39:10 +00:00
joinBlockImpl<kind_, strictness_>(block, key_names_left, sample_block_with_columns_to_add, map);
}))
2019-01-14 21:40:02 +00:00
{
/// Joined
}
else if (kind == ASTTableJoin::Kind::Cross)
2020-04-06 13:39:57 +00:00
joinBlockImplCross(block, not_processed);
else
throw Exception("Logical error: unknown combination of JOIN", ErrorCodes::LOGICAL_ERROR);
}
void HashJoin::joinTotals(Block & block) const
{
2019-09-19 14:53:03 +00:00
JoinCommon::joinTotals(totals, sample_block_with_columns_to_add, key_names_right, block);
}
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;
}
}
}
};
2019-12-02 18:07:27 +00:00
2017-04-02 17:37:49 +00:00
/// Stream from not joined earlier rows of the right table.
class NonJoinedBlockInputStream : private NotJoined, public IBlockInputStream
{
public:
NonJoinedBlockInputStream(const HashJoin & parent_, const Block & result_sample_block_, UInt64 max_block_size_)
: NotJoined(*parent_.table_join,
parent_.savedBlockSample(),
parent_.right_sample_block,
result_sample_block_)
, parent(parent_)
2019-07-03 19:06:34 +00:00
, max_block_size(max_block_size_)
{}
String getName() const override { return "NonJoined"; }
Block getHeader() const override { return result_sample_block; }
protected:
Block readImpl() override
{
2019-12-19 15:50:28 +00:00
if (parent.data->blocks.empty())
return Block();
2019-07-03 19:06:34 +00:00
return createBlock();
}
private:
const HashJoin & parent;
2019-02-10 16:55:12 +00:00
UInt64 max_block_size;
2019-07-04 12:12:39 +00:00
std::any position;
std::optional<HashJoin::BlockNullmapList::const_iterator> nulls_position;
Block createBlock()
{
MutableColumns columns_right = saved_block_sample.cloneEmptyColumns();
size_t rows_added = 0;
2019-07-03 19:06:34 +00:00
auto fill_callback = [&](auto, auto strictness, auto & map)
{
rows_added = fillColumnsFromMap<strictness>(map, columns_right);
2019-07-03 19:06:34 +00:00
};
2019-12-19 15:50:28 +00:00
if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps, fill_callback))
2019-07-03 19:06:34 +00:00
throw Exception("Logical error: unknown JOIN strictness (must be on of: ANY, ALL, ASOF)", ErrorCodes::LOGICAL_ERROR);
fillNullsFromBlocks(columns_right, rows_added);
if (!rows_added)
return {};
correctLowcardAndNullability(columns_right);
Block res = result_sample_block.cloneEmpty();
addLeftColumns(res, rows_added);
addRightColumns(res, columns_right);
copySameKeys(res);
return res;
}
2019-07-03 19:06:34 +00:00
template <ASTTableJoin::Strictness STRICTNESS, typename Maps>
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: \
2019-07-03 19:06:34 +00:00
return fillColumns<STRICTNESS>(*maps.TYPE, columns_keys_and_right);
APPLY_FOR_JOIN_VARIANTS(M)
#undef M
default:
2019-12-19 15:50:28 +00:00
throw Exception("Unsupported JOIN keys. Type: " + toString(static_cast<UInt32>(parent.data->type)),
2019-08-05 14:03:14 +00:00
ErrorCodes::UNSUPPORTED_JOIN_KEYS);
2019-07-03 19:06:34 +00:00
}
__builtin_unreachable();
}
template <ASTTableJoin::Strictness STRICTNESS, typename Map>
size_t fillColumns(const Map & map, MutableColumns & columns_keys_and_right)
{
2019-07-03 19:06:34 +00:00
using Mapped = typename Map::mapped_type;
using Iterator = typename Map::const_iterator;
size_t rows_added = 0;
2019-07-04 12:12:39 +00:00
if (!position.has_value())
position = std::make_any<Iterator>(map.begin());
2019-07-04 12:12:39 +00:00
Iterator & it = std::any_cast<Iterator &>(position);
auto end = map.end();
for (; it != end; ++it)
{
2019-10-29 15:16:51 +00:00
const Mapped & mapped = it->getMapped();
2019-12-02 18:07:27 +00:00
size_t off = map.offsetInternal(it.getPtr());
if (parent.isUsed(off))
continue;
2019-12-02 18:07:27 +00:00
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
{
2019-07-04 12:12:39 +00:00
const Block * block = it->first;
const NullMap & nullmap = assert_cast<const ColumnUInt8 &>(*it->second).getData();
2019-07-03 19:06:34 +00:00
for (size_t row = 0; row < nullmap.size(); ++row)
{
if (nullmap[row])
{
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;
}
}
}
}
};
BlockInputStreamPtr HashJoin::createStreamWithNonJoinedRows(const Block & result_sample_block, UInt64 max_block_size) const
{
2019-12-02 18:07:27 +00:00
if (table_join->strictness() == ASTTableJoin::Strictness::Asof ||
table_join->strictness() == ASTTableJoin::Strictness::Semi)
return {};
if (isRightOrFull(table_join->kind()))
2019-11-05 20:22:20 +00:00
return std::make_shared<NonJoinedBlockInputStream>(*this, result_sample_block, max_block_size);
return {};
}
void HashJoin::reuseJoinedData(const HashJoin & join)
{
data = join.data;
joinDispatch(kind, strictness, data->maps, [this](auto kind_, auto strictness_, auto & map)
{
used_flags.reinit<kind_, strictness_>(map.getBufferSizeInCells(data->type) + 1);
});
}
2014-06-12 02:31:30 +00:00
}