ClickHouse/src/Interpreters/ConcurrentHashJoin.cpp

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

227 lines
7.1 KiB
C++
Raw Normal View History

#include <memory>
#include <mutex>
#include <Columns/ColumnSparse.h>
2022-04-19 08:53:24 +00:00
#include <Columns/FilterDescription.h>
#include <Columns/IColumn.h>
#include <Core/ColumnsWithTypeAndName.h>
#include <Core/NamesAndTypes.h>
#include <Interpreters/ConcurrentHashJoin.h>
#include <Interpreters/Context.h>
2022-04-19 08:53:24 +00:00
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/PreparedSets.h>
#include <Interpreters/TableJoin.h>
2022-04-21 08:59:30 +00:00
#include <Interpreters/createBlockSelector.h>
2022-04-19 08:53:24 +00:00
#include <Parsers/DumpASTNode.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/IAST_fwd.h>
#include <Parsers/parseQuery.h>
#include <Common/Exception.h>
#include <Common/WeakHash.h>
2022-04-21 08:59:30 +00:00
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int SET_SIZE_LIMIT_EXCEEDED;
}
static UInt32 toPowerOfTwo(UInt32 x)
{
if (x <= 1)
return 1;
return static_cast<UInt32>(1) << (32 - std::countl_zero(x - 1));
}
2022-05-05 04:04:11 +00:00
ConcurrentHashJoin::ConcurrentHashJoin(ContextPtr context_, std::shared_ptr<TableJoin> table_join_, size_t slots_, const Block & right_sample_block, bool any_take_last_row_)
: context(context_)
, table_join(table_join_)
, slots(toPowerOfTwo(std::min<size_t>(slots_, 256)))
{
for (size_t i = 0; i < slots; ++i)
{
2022-04-21 04:14:36 +00:00
auto inner_hash_join = std::make_shared<InternalHashJoin>();
inner_hash_join->data = std::make_unique<HashJoin>(table_join_, right_sample_block, any_take_last_row_);
hash_joins.emplace_back(std::move(inner_hash_join));
}
}
bool ConcurrentHashJoin::addJoinedBlock(const Block & right_block, bool check_limits)
{
Blocks dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_right, right_block);
2022-04-21 05:19:33 +00:00
2022-05-10 12:09:49 +00:00
size_t blocks_left = 0;
for (const auto & block : dispatched_blocks)
{
if (block)
{
++blocks_left;
}
}
while (blocks_left > 0)
{
/// insert blocks into corresponding HashJoin instances
for (size_t i = 0; i < dispatched_blocks.size(); ++i)
2022-04-21 05:19:33 +00:00
{
auto & hash_join = hash_joins[i];
auto & dispatched_block = dispatched_blocks[i];
2022-05-10 12:09:49 +00:00
if (dispatched_block)
2022-04-21 05:19:33 +00:00
{
/// if current hash_join is already processed by another thread, skip it and try later
std::unique_lock<std::mutex> lock(hash_join->mutex, std::try_to_lock);
if (!lock.owns_lock())
continue;
2022-04-21 04:14:36 +00:00
bool limit_exceeded = !hash_join->data->addJoinedBlock(dispatched_block, check_limits);
dispatched_block = {};
blocks_left--;
if (limit_exceeded)
return false;
2022-04-21 05:19:33 +00:00
}
}
}
if (check_limits)
2022-04-21 05:19:33 +00:00
return table_join->sizeLimits().check(getTotalRowCount(), getTotalByteCount(), "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
return true;
}
2022-04-21 05:19:33 +00:00
void ConcurrentHashJoin::joinBlock(Block & block, std::shared_ptr<ExtraBlock> & /*not_processed*/)
{
2022-05-05 04:04:11 +00:00
Blocks dispatched_blocks = dispatchBlock(table_join->getOnlyClause().key_names_left, block);
2022-06-14 14:13:01 +00:00
block = {};
for (size_t i = 0; i < dispatched_blocks.size(); ++i)
{
std::shared_ptr<ExtraBlock> none_extra_block;
auto & hash_join = hash_joins[i];
auto & dispatched_block = dispatched_blocks[i];
hash_join->data->joinBlock(dispatched_block, none_extra_block);
if (none_extra_block && !none_extra_block->empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "not_processed should be empty");
}
2022-04-21 06:10:09 +00:00
block = concatenateBlocks(dispatched_blocks);
}
void ConcurrentHashJoin::checkTypesOfKeys(const Block & block) const
{
hash_joins[0]->data->checkTypesOfKeys(block);
}
void ConcurrentHashJoin::setTotals(const Block & block)
{
if (block)
{
std::lock_guard lock(totals_mutex);
totals = block;
}
}
const Block & ConcurrentHashJoin::getTotals() const
{
return totals;
}
size_t ConcurrentHashJoin::getTotalRowCount() const
{
size_t res = 0;
for (const auto & hash_join : hash_joins)
{
2022-04-25 08:16:50 +00:00
std::lock_guard lock(hash_join->mutex);
res += hash_join->data->getTotalRowCount();
}
return res;
}
size_t ConcurrentHashJoin::getTotalByteCount() const
{
size_t res = 0;
for (const auto & hash_join : hash_joins)
{
2022-04-25 08:16:50 +00:00
std::lock_guard lock(hash_join->mutex);
res += hash_join->data->getTotalByteCount();
}
return res;
}
bool ConcurrentHashJoin::alwaysReturnsEmptySet() const
{
for (const auto & hash_join : hash_joins)
{
2022-04-25 08:16:50 +00:00
std::lock_guard lock(hash_join->mutex);
2022-04-21 05:19:33 +00:00
if (!hash_join->data->alwaysReturnsEmptySet())
return false;
}
return true;
}
std::shared_ptr<NotJoinedBlocks> ConcurrentHashJoin::getNonJoinedBlocks(
const Block & /*left_sample_block*/, const Block & /*result_sample_block*/, UInt64 /*max_block_size*/) const
{
if (!JoinCommon::hasNonJoinedBlocks(*table_join))
return {};
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid join type. join kind: {}, strictness: {}", table_join->kind(), table_join->strictness());
}
2022-06-14 14:13:01 +00:00
static ALWAYS_INLINE IColumn::Selector hashToSelector(const WeakHash32 & hash, size_t num_shards)
{
assert(num_shards > 0 && (num_shards & (num_shards - 1)) == 0);
const auto & data = hash.getData();
size_t num_rows = data.size();
IColumn::Selector selector(num_rows);
for (size_t i = 0; i < num_rows; ++i)
/// Apply intHash64 to mix bits in data.
/// HashTable internally uses WeakHash32, and we need to get different lower bits not to cause collisions.
selector[i] = intHash64(data[i]) & (num_shards - 1);
return selector;
}
2022-05-05 04:04:11 +00:00
2022-06-14 14:13:01 +00:00
IColumn::Selector ConcurrentHashJoin::selectDispatchBlock(const Strings & key_columns_names, const Block & from_block)
{
2022-05-05 04:04:11 +00:00
size_t num_rows = from_block.rows();
2022-06-14 14:13:01 +00:00
size_t num_shards = hash_joins.size();
2022-05-05 04:04:11 +00:00
WeakHash32 hash(num_rows);
2022-05-05 04:04:11 +00:00
for (const auto & key_name : key_columns_names)
{
const auto & key_col = from_block.getByName(key_name).column->convertToFullColumnIfConst();
const auto & key_col_no_lc = recursiveRemoveLowCardinality(recursiveRemoveSparse(key_col));
2022-06-07 15:10:41 +00:00
key_col_no_lc->updateWeakHash32(hash);
2022-04-21 08:59:30 +00:00
}
2022-06-14 14:13:01 +00:00
return hashToSelector(hash, num_shards);
}
2022-05-05 04:04:11 +00:00
Blocks ConcurrentHashJoin::dispatchBlock(const Strings & key_columns_names, const Block & from_block)
{
/// TODO: use JoinCommon::scatterBlockByHash
2022-05-05 04:04:11 +00:00
size_t num_shards = hash_joins.size();
2022-06-14 14:13:01 +00:00
size_t num_cols = from_block.columns();
IColumn::Selector selector = selectDispatchBlock(key_columns_names, from_block);
Blocks result(num_shards);
2022-05-05 04:04:11 +00:00
for (size_t i = 0; i < num_shards; ++i)
2022-06-14 14:13:01 +00:00
result[i] = from_block.cloneEmpty();
2022-04-21 08:59:30 +00:00
2022-05-05 04:04:11 +00:00
for (size_t i = 0; i < num_cols; ++i)
2022-04-21 08:59:30 +00:00
{
auto dispatched_columns = from_block.getByPosition(i).column->scatter(num_shards, selector);
2022-05-05 04:04:11 +00:00
assert(result.size() == dispatched_columns.size());
2022-04-21 08:59:30 +00:00
for (size_t block_index = 0; block_index < num_shards; ++block_index)
{
2022-05-05 04:04:11 +00:00
result[block_index].getByPosition(i).column = std::move(dispatched_columns[block_index]);
}
}
2022-05-05 04:04:11 +00:00
return result;
}
}