ClickHouse/src/Interpreters/GraceHashJoin.cpp

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

737 lines
23 KiB
C++
Raw Normal View History

2022-06-06 17:26:22 +00:00
#include <Interpreters/GraceHashJoin.h>
2022-06-11 11:03:44 +00:00
#include <Interpreters/HashJoin.h>
2022-06-16 12:09:23 +00:00
#include <Interpreters/TableJoin.h>
#include <Interpreters/Context.h>
2022-06-16 12:09:23 +00:00
#include <Formats/NativeWriter.h>
#include <Interpreters/TemporaryDataOnDisk.h>
2022-06-16 12:09:23 +00:00
#include <Compression/CompressedWriteBuffer.h>
#include <Core/ProtocolDefines.h>
2022-06-16 12:09:23 +00:00
#include <Disks/IVolume.h>
#include <Disks/TemporaryFileOnDisk.h>
#include <Common/logger_useful.h>
#include <Common/thread_local_rng.h>
2022-06-16 12:09:23 +00:00
#include <base/FnTraits.h>
2022-06-16 12:09:23 +00:00
#include <fmt/format.h>
2022-06-06 17:26:22 +00:00
2022-10-04 08:20:13 +00:00
#include <Formats/formatBlock.h>
namespace CurrentMetrics
{
extern const Metric TemporaryFilesForJoin;
}
2022-06-06 17:26:22 +00:00
namespace DB
{
2022-06-16 12:09:23 +00:00
namespace ErrorCodes
2022-06-11 11:03:44 +00:00
{
2022-06-19 19:18:37 +00:00
extern const int LIMIT_EXCEEDED;
2022-06-18 00:25:26 +00:00
extern const int LOGICAL_ERROR;
2022-06-23 15:52:30 +00:00
extern const int NOT_IMPLEMENTED;
2022-06-16 12:09:23 +00:00
}
namespace
{
2022-10-05 12:40:32 +00:00
class AccumulatedBlockReader
2022-06-16 12:09:23 +00:00
{
public:
2022-10-05 12:40:32 +00:00
AccumulatedBlockReader(TemporaryFileStream & reader_,
std::mutex & mutex_,
2022-12-20 12:50:27 +00:00
size_t result_block_size_ = 0)
2022-10-05 12:40:32 +00:00
: reader(reader_)
, mutex(mutex_)
, result_block_size(result_block_size_)
{
if (!reader.isWriteFinished())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Reading not finished file");
}
Block read()
{
2023-03-02 13:36:47 +00:00
std::lock_guard lock(mutex);
2022-10-26 09:54:07 +00:00
if (eof)
return {};
2022-10-05 12:40:32 +00:00
Blocks blocks;
size_t rows_read = 0;
2023-01-20 16:30:34 +00:00
do
2022-10-05 12:40:32 +00:00
{
Block block = reader.read();
rows_read += block.rows();
if (!block)
{
eof = true;
2023-01-20 16:30:34 +00:00
if (blocks.size() == 1)
return blocks.front();
2022-10-05 12:40:32 +00:00
return concatenateBlocks(blocks);
}
2022-10-05 12:40:32 +00:00
blocks.push_back(std::move(block));
2023-01-20 16:30:34 +00:00
} while (rows_read < result_block_size);
2023-01-20 16:30:34 +00:00
if (blocks.size() == 1)
return blocks.front();
2022-10-05 12:40:32 +00:00
return concatenateBlocks(blocks);
}
private:
TemporaryFileStream & reader;
2022-10-05 12:40:32 +00:00
std::mutex & mutex;
const size_t result_block_size;
bool eof = false;
};
std::deque<size_t> generateRandomPermutation(size_t from, size_t to)
{
size_t size = to - from;
std::deque<size_t> indices(size);
std::iota(indices.begin(), indices.end(), from);
std::shuffle(indices.begin(), indices.end(), thread_local_rng);
return indices;
}
// Try to apply @callback in the order specified in @indices
// Until it returns true for each index in the @indices.
void retryForEach(std::deque<size_t> indices, Fn<bool(size_t)> auto callback)
{
while (!indices.empty())
{
2022-11-27 11:16:31 +00:00
size_t bucket_index = indices.front();
indices.pop_front();
2022-11-27 11:16:31 +00:00
if (!callback(bucket_index))
indices.push_back(bucket_index);
}
}
2022-06-11 11:03:44 +00:00
}
class GraceHashJoin::FileBucket : boost::noncopyable
2022-06-16 12:09:23 +00:00
{
enum class State : int
{
WRITING_BLOCKS,
JOINING_BLOCKS,
FINISHED,
};
public:
2022-10-05 12:40:32 +00:00
using BucketLock = std::unique_lock<std::mutex>;
2023-01-20 16:30:34 +00:00
explicit FileBucket(size_t bucket_index_, TemporaryFileStream & left_file_, TemporaryFileStream & right_file_, Poco::Logger * log_)
2022-10-05 12:40:32 +00:00
: idx{bucket_index_}
, left_file{left_file_}
, right_file{right_file_}
2022-06-16 12:09:23 +00:00
, state{State::WRITING_BLOCKS}
2023-01-20 16:30:34 +00:00
, log{log_}
2022-06-16 12:09:23 +00:00
{
}
void addLeftBlock(const Block & block)
{
2022-10-04 08:20:13 +00:00
std::unique_lock<std::mutex> lock(left_file_mutex);
addBlockImpl(block, left_file, lock);
}
void addRightBlock(const Block & block)
{
std::unique_lock<std::mutex> lock(right_file_mutex);
addBlockImpl(block, right_file, lock);
}
bool tryAddLeftBlock(const Block & block)
{
std::unique_lock<std::mutex> lock(left_file_mutex, std::try_to_lock);
return addBlockImpl(block, left_file, lock);
}
2022-10-04 08:20:13 +00:00
bool tryAddRightBlock(const Block & block)
{
std::unique_lock<std::mutex> lock(right_file_mutex, std::try_to_lock);
return addBlockImpl(block, right_file, lock);
}
2022-06-16 12:09:23 +00:00
2022-10-05 12:40:32 +00:00
bool finished() const
2022-06-16 12:09:23 +00:00
{
2022-10-05 12:40:32 +00:00
std::unique_lock<std::mutex> left_lock(left_file_mutex);
return left_file.isEof();
2022-06-16 12:09:23 +00:00
}
bool empty() const { return is_empty.load(); }
2022-06-16 12:09:23 +00:00
2022-10-05 12:40:32 +00:00
AccumulatedBlockReader startJoining()
2022-06-16 12:09:23 +00:00
{
2022-10-05 12:40:32 +00:00
LOG_TRACE(log, "Joining file bucket {}", idx);
{
std::unique_lock<std::mutex> left_lock(left_file_mutex);
std::unique_lock<std::mutex> right_lock(right_file_mutex);
2022-06-16 12:09:23 +00:00
2023-01-23 18:09:26 +00:00
left_file.finishWriting();
right_file.finishWriting();
2022-10-04 08:20:13 +00:00
2022-10-05 12:40:32 +00:00
state = State::JOINING_BLOCKS;
}
return AccumulatedBlockReader(right_file, right_file_mutex);
2022-10-04 08:20:13 +00:00
}
2022-06-17 17:36:24 +00:00
2022-10-05 12:40:32 +00:00
AccumulatedBlockReader getLeftTableReader()
2022-10-04 08:20:13 +00:00
{
2022-10-05 12:40:32 +00:00
ensureState(State::JOINING_BLOCKS);
return AccumulatedBlockReader(left_file, left_file_mutex);
2022-10-04 08:20:13 +00:00
}
2022-10-05 12:40:32 +00:00
const size_t idx;
2022-06-16 12:09:23 +00:00
private:
2022-10-04 08:20:13 +00:00
bool addBlockImpl(const Block & block, TemporaryFileStream & writer, std::unique_lock<std::mutex> & lock)
{
ensureState(State::WRITING_BLOCKS);
2022-10-07 09:16:09 +00:00
if (!lock.owns_lock())
return false;
2022-10-07 09:16:09 +00:00
if (block.rows())
is_empty = false;
writer.write(block);
2022-10-04 08:20:13 +00:00
return true;
}
2022-06-17 17:36:24 +00:00
void transition(State expected, State desired)
{
State prev = state.exchange(desired);
if (prev != expected)
2022-10-05 12:40:32 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid state transition from {} (got {}) to {}", expected, prev, desired);
2022-06-17 17:36:24 +00:00
}
2022-06-16 12:09:23 +00:00
2022-10-05 12:40:32 +00:00
void ensureState(State expected) const
2022-06-16 12:09:23 +00:00
{
2022-10-05 12:40:32 +00:00
State cur_state = state.load();
if (cur_state != expected)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid state transition, expected {}, got {}", expected, state.load());
2022-06-16 12:09:23 +00:00
}
TemporaryFileStream & left_file;
TemporaryFileStream & right_file;
2022-10-05 12:40:32 +00:00
mutable std::mutex left_file_mutex;
mutable std::mutex right_file_mutex;
std::atomic_bool is_empty = true;
2022-06-16 12:09:23 +00:00
std::atomic<State> state;
2022-10-04 08:20:13 +00:00
Poco::Logger * log;
2022-06-16 12:09:23 +00:00
};
namespace
{
2022-12-20 12:50:27 +00:00
2022-12-16 18:33:01 +00:00
template <JoinTableSide table_side>
2022-12-20 12:50:27 +00:00
void flushBlocksToBuckets(Blocks & blocks, const GraceHashJoin::Buckets & buckets, size_t except_index = 0)
2022-10-05 12:40:32 +00:00
{
chassert(blocks.size() == buckets.size());
2022-10-05 12:40:32 +00:00
retryForEach(
generateRandomPermutation(1, buckets.size()), // skipping 0 block, since we join it in memory w/o spilling on disk
2022-10-05 12:40:32 +00:00
[&](size_t i)
{
2022-12-20 12:50:27 +00:00
/// Skip empty and current bucket
if (!blocks[i].rows() || i == except_index)
2022-10-05 12:40:32 +00:00
return true;
bool flushed = false;
2022-12-16 18:33:01 +00:00
if constexpr (table_side == JoinTableSide::Left)
flushed = buckets[i]->tryAddLeftBlock(blocks[i]);
2022-12-16 18:33:01 +00:00
if constexpr (table_side == JoinTableSide::Right)
flushed = buckets[i]->tryAddRightBlock(blocks[i]);
2022-10-05 12:40:32 +00:00
if (flushed)
blocks[i].clear();
2022-10-05 12:40:32 +00:00
return flushed;
});
}
}
2022-06-16 12:09:23 +00:00
GraceHashJoin::GraceHashJoin(
ContextPtr context_, std::shared_ptr<TableJoin> table_join_,
const Block & left_sample_block_,
const Block & right_sample_block_,
TemporaryDataOnDiskScopePtr tmp_data_,
bool any_take_last_row_)
2022-06-16 12:09:23 +00:00
: log{&Poco::Logger::get("GraceHashJoin")}
, context{context_}
, table_join{std::move(table_join_)}
, left_sample_block{left_sample_block_}
2022-06-16 12:09:23 +00:00
, right_sample_block{right_sample_block_}
, any_take_last_row{any_take_last_row_}
, max_num_buckets{context->getSettingsRef().grace_hash_join_max_buckets}
, max_block_size{context->getSettingsRef().max_block_size}
2022-10-04 08:20:13 +00:00
, left_key_names(table_join->getOnlyClause().key_names_left)
, right_key_names(table_join->getOnlyClause().key_names_right)
, tmp_data(std::make_unique<TemporaryDataOnDisk>(tmp_data_, CurrentMetrics::TemporaryFilesForJoin))
2022-10-05 12:40:32 +00:00
, hash_join(makeInMemoryJoin())
2022-12-20 12:50:27 +00:00
, hash_join_sample_block(hash_join->savedBlockSample())
2022-06-16 12:09:23 +00:00
{
2022-09-15 12:14:27 +00:00
if (!GraceHashJoin::isSupported(table_join))
2022-09-07 16:38:47 +00:00
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "GraceHashJoin is not supported for this join type");
2022-10-06 14:26:56 +00:00
}
void GraceHashJoin::initBuckets()
{
if (!buckets.empty())
return;
const auto & settings = context->getSettingsRef();
2022-11-27 11:16:31 +00:00
size_t initial_num_buckets = roundUpToPowerOfTwoOrZero(std::clamp<size_t>(settings.grace_hash_join_initial_buckets, 1, settings.grace_hash_join_max_buckets));
2022-10-05 12:40:32 +00:00
2022-06-16 12:09:23 +00:00
for (size_t i = 0; i < initial_num_buckets; ++i)
{
2022-10-05 12:40:32 +00:00
addBucket(buckets);
2022-06-16 12:09:23 +00:00
}
2022-10-05 12:40:32 +00:00
if (buckets.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "No buckets created");
2022-12-20 12:50:27 +00:00
LOG_TRACE(log, "Initialize {} bucket{}", buckets.size(), buckets.size() > 1 ? "s" : "");
2022-10-05 12:40:32 +00:00
current_bucket = buckets.front().get();
current_bucket->startJoining();
2022-06-16 12:09:23 +00:00
}
2023-05-04 03:27:20 +00:00
bool GraceHashJoin::isSupported(const std::shared_ptr<TableJoin> & table_join)
2022-09-07 15:00:15 +00:00
{
2022-09-15 12:14:27 +00:00
bool is_asof = (table_join->strictness() == JoinStrictness::Asof);
auto kind = table_join->kind();
return !is_asof && (isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind)) && table_join->oneDisjunct();
}
2022-06-23 01:44:16 +00:00
GraceHashJoin::~GraceHashJoin() = default;
2022-06-16 12:09:23 +00:00
bool GraceHashJoin::addJoinedBlock(const Block & block, bool /*check_limits*/)
{
2022-10-06 14:26:56 +00:00
if (current_bucket == nullptr)
throw Exception(ErrorCodes::LOGICAL_ERROR, "GraceHashJoin is not initialized");
2022-06-17 21:17:05 +00:00
Block materialized = materializeBlock(block);
2022-12-20 12:50:27 +00:00
addJoinedBlockImpl(std::move(materialized));
2022-06-16 12:09:23 +00:00
return true;
}
2022-12-27 10:13:12 +00:00
bool GraceHashJoin::hasMemoryOverflow(size_t total_rows, size_t total_bytes) const
2022-06-16 12:09:23 +00:00
{
2022-09-27 13:39:06 +00:00
/// One row can't be split, avoid loop
2022-12-27 10:13:12 +00:00
if (total_rows < 2)
2022-12-20 12:50:27 +00:00
return false;
2022-12-27 10:13:12 +00:00
bool has_overflow = !table_join->sizeLimits().softCheck(total_rows, total_bytes);
2022-12-20 12:50:27 +00:00
if (has_overflow)
2022-12-27 10:13:12 +00:00
LOG_TRACE(log, "Memory overflow, size exceeded {} / {} bytes, {} / {} rows",
ReadableSize(total_bytes), ReadableSize(table_join->sizeLimits().max_bytes),
total_rows, table_join->sizeLimits().max_rows);
2022-12-20 12:50:27 +00:00
return has_overflow;
}
2022-12-27 10:13:12 +00:00
bool GraceHashJoin::hasMemoryOverflow(const BlocksList & blocks) const
2022-06-16 12:09:23 +00:00
{
2022-12-27 10:13:12 +00:00
size_t total_rows = 0;
size_t total_bytes = 0;
for (const auto & block : blocks)
{
total_rows += block.rows();
total_bytes += block.allocatedBytes();
}
return hasMemoryOverflow(total_rows, total_bytes);
}
2022-12-27 10:13:12 +00:00
bool GraceHashJoin::hasMemoryOverflow(const InMemoryJoinPtr & hash_join_) const
{
size_t total_rows = hash_join_->getTotalRowCount();
size_t total_bytes = hash_join_->getTotalByteCount();
2022-09-26 13:46:50 +00:00
2022-12-27 10:13:12 +00:00
return hasMemoryOverflow(total_rows, total_bytes);
2022-06-16 12:09:23 +00:00
}
2022-10-26 09:54:07 +00:00
GraceHashJoin::Buckets GraceHashJoin::rehashBuckets(size_t to_size)
2022-06-16 12:09:23 +00:00
{
2022-10-04 08:20:13 +00:00
std::unique_lock lock(rehash_mutex);
size_t current_size = buckets.size();
2022-10-05 12:40:32 +00:00
2022-10-26 09:54:07 +00:00
if (to_size <= current_size)
2022-10-05 12:40:32 +00:00
return buckets;
2022-12-20 12:50:27 +00:00
chassert(isPowerOf2(to_size));
2022-06-16 12:09:23 +00:00
2022-10-26 09:54:07 +00:00
if (to_size > max_num_buckets)
2022-06-16 12:09:23 +00:00
{
2022-09-15 12:14:27 +00:00
throw Exception(ErrorCodes::LIMIT_EXCEEDED,
"Too many grace hash join buckets ({} > {}), "
"consider increasing grace_hash_join_max_buckets or max_rows_in_join/max_bytes_in_join",
2022-10-26 09:54:07 +00:00
to_size, max_num_buckets);
2022-06-16 12:09:23 +00:00
}
2022-10-26 09:54:07 +00:00
LOG_TRACE(log, "Rehashing from {} to {}", current_size, to_size);
2022-10-26 09:54:07 +00:00
buckets.reserve(to_size);
for (size_t i = current_size; i < to_size; ++i)
2022-10-05 12:40:32 +00:00
addBucket(buckets);
2022-10-07 09:16:09 +00:00
2022-10-04 08:20:13 +00:00
return buckets;
2022-06-11 11:03:44 +00:00
}
2022-10-05 12:40:32 +00:00
void GraceHashJoin::addBucket(Buckets & destination)
2022-06-16 12:09:23 +00:00
{
2023-01-20 16:30:34 +00:00
auto & left_file = tmp_data->createStream(left_sample_block);
auto & right_file = tmp_data->createStream(prepareRightBlock(right_sample_block));
BucketPtr new_bucket = std::make_shared<FileBucket>(destination.size(), left_file, right_file, log);
2022-10-04 08:20:13 +00:00
destination.emplace_back(std::move(new_bucket));
2022-06-16 12:09:23 +00:00
}
void GraceHashJoin::checkTypesOfKeys(const Block & block) const
{
2023-01-23 12:34:36 +00:00
chassert(hash_join);
2022-10-05 12:40:32 +00:00
return hash_join->checkTypesOfKeys(block);
2022-06-11 11:03:44 +00:00
}
2022-11-29 11:46:11 +00:00
void GraceHashJoin::initialize(const Block & sample_block)
2022-06-16 12:09:23 +00:00
{
2022-11-29 11:46:11 +00:00
left_sample_block = sample_block.cloneEmpty();
output_sample_block = left_sample_block.cloneEmpty();
ExtraBlockPtr not_processed;
hash_join->joinBlock(output_sample_block, not_processed);
initBuckets();
}
2022-11-29 11:46:11 +00:00
void GraceHashJoin::joinBlock(Block & block, std::shared_ptr<ExtraBlock> & not_processed)
{
2022-06-17 17:36:24 +00:00
if (block.rows() == 0)
{
2022-10-05 12:40:32 +00:00
hash_join->joinBlock(block, not_processed);
2022-06-16 18:36:49 +00:00
return;
}
2022-06-16 12:09:23 +00:00
2022-06-17 21:17:05 +00:00
materializeBlockInplace(block);
2022-12-16 18:33:01 +00:00
/// number of buckets doesn't change after right table is split to buckets, i.e. read-only access to buckets
/// so, no need to copy buckets here
size_t num_buckets = getNumBuckets();
Blocks blocks = JoinCommon::scatterBlockByHash(left_key_names, block, num_buckets);
2022-06-16 12:09:23 +00:00
2022-10-05 12:40:32 +00:00
block = std::move(blocks[current_bucket->idx]);
hash_join->joinBlock(block, not_processed);
2022-06-16 12:09:23 +00:00
if (not_processed)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unhandled not processed block in GraceHashJoin");
2022-06-16 12:09:23 +00:00
2022-12-16 18:33:01 +00:00
flushBlocksToBuckets<JoinTableSide::Left>(blocks, buckets);
2022-06-16 12:09:23 +00:00
}
2022-06-23 05:41:45 +00:00
void GraceHashJoin::setTotals(const Block & block)
{
2022-10-31 09:53:30 +00:00
if (block.rows() > 0)
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Totals are not supported for GraceHashJoin, got '{}'", block.dumpStructure());
2022-06-23 05:41:45 +00:00
}
2022-06-16 12:09:23 +00:00
size_t GraceHashJoin::getTotalRowCount() const
{
2022-10-07 09:16:09 +00:00
std::lock_guard lock(hash_join_mutex);
2022-10-05 12:40:32 +00:00
assert(hash_join);
return hash_join->getTotalRowCount();
2022-06-16 12:09:23 +00:00
}
size_t GraceHashJoin::getTotalByteCount() const
{
2022-10-07 09:16:09 +00:00
std::lock_guard lock(hash_join_mutex);
2023-01-23 12:34:36 +00:00
chassert(hash_join);
2022-10-05 12:40:32 +00:00
return hash_join->getTotalByteCount();
2022-06-16 12:09:23 +00:00
}
bool GraceHashJoin::alwaysReturnsEmptySet() const
{
if (!isInnerOrRight(table_join->kind()))
return false;
2022-12-16 18:33:01 +00:00
bool file_buckets_are_empty = [this]()
{
std::shared_lock lock(rehash_mutex);
return std::all_of(buckets.begin(), buckets.end(), [](const auto & bucket) { return bucket->empty(); });
}();
2023-01-23 12:34:36 +00:00
if (!file_buckets_are_empty)
return false;
chassert(hash_join);
bool hash_join_is_empty = hash_join->alwaysReturnsEmptySet();
return hash_join_is_empty;
2022-06-16 12:09:23 +00:00
}
// This is only be called for bucket[0] at present. other buckets' non-joined blocks are generated in
2023-05-04 03:27:20 +00:00
// DelayedBlocks.
2023-05-08 03:35:48 +00:00
// There is a finished counter in JoiningTransform, only the last JoiningTransform could call this function.
2023-05-04 03:27:20 +00:00
IBlocksStreamPtr
GraceHashJoin::getNonJoinedBlocks(const Block & left_sample_block_, const Block & result_sample_block_, UInt64 max_block_size_) const
2022-06-16 12:09:23 +00:00
{
return hash_join->getNonJoinedBlocks(left_sample_block_, result_sample_block_, max_block_size_);
2022-06-16 12:09:23 +00:00
}
2022-10-04 08:21:02 +00:00
class GraceHashJoin::DelayedBlocks : public IBlocksStream
2022-06-16 12:09:23 +00:00
{
public:
explicit DelayedBlocks(
size_t current_bucket_,
Buckets buckets_,
InMemoryJoinPtr hash_join_,
const Names & left_key_names_,
const Names & right_key_names_,
const Block & left_sample_block_,
const Block & result_sample_block_,
size_t max_block_size_)
2022-10-05 12:40:32 +00:00
: current_bucket(current_bucket_)
, buckets(std::move(buckets_))
, hash_join(std::move(hash_join_))
, left_reader(buckets[current_bucket]->getLeftTableReader())
, left_key_names(left_key_names_)
, right_key_names(right_key_names_)
, left_sample_block(left_sample_block_)
, result_sample_block(result_sample_block_)
, max_block_size(max_block_size_)
2022-06-16 12:09:23 +00:00
{
}
// One DelayedBlocks is shared among multiple DelayedJoinedBlocksWorkerTransforms, need locks for
// - reading from left_reader. left_reader.read() has had a lock inside.
// - reading non-joined blocks from hash_join. Since iterate on non-joined blocks will has state
// changed inside.
2022-10-05 12:40:32 +00:00
Block nextImpl() override
{
// there is data race case wihthout this lock:
// 1. thread 1 read the last block from left_reader, but not finish the join
// 2. thread 2 try to read from non-joined blocks. Since thread 1 has not finished,
// the used flags in the hash_join is incomplete, then thread 2 return invalid mismatch rows.
std::lock_guard lock(non_joined_blocks_mutex);
2022-10-05 12:40:32 +00:00
Block block;
size_t num_buckets = buckets.size();
size_t current_idx = buckets[current_bucket]->idx;
2022-10-05 12:40:32 +00:00
do
{
2023-05-04 03:27:20 +00:00
// When left reader finish, return non-joined blocks.
// empty block means the end of this stream.
if (!is_left_reader_finished)
2022-10-19 10:20:41 +00:00
{
block = left_reader.read();
if (!block)
{
is_left_reader_finished = true;
non_joined_blocks = hash_join->getNonJoinedBlocks(left_sample_block, result_sample_block, max_block_size);
}
}
if (is_left_reader_finished)
{
if (non_joined_blocks)
{
return non_joined_blocks->next();
}
else
return {};
2022-10-19 10:20:41 +00:00
}
// block comes from left_reader, need to join with right table to get the result.
2022-10-05 12:40:32 +00:00
Blocks blocks = JoinCommon::scatterBlockByHash(left_key_names, block, num_buckets);
block = std::move(blocks[current_idx]);
/*
* We need to filter out blocks that were written to the current bucket `B_{n}`
* but then virtually moved to another bucket `B_{n+i}` on rehash.
2022-11-27 11:16:31 +00:00
* Bucket `B_{n+i}` is waiting for the buckets with smaller index to be processed,
2022-10-05 12:40:32 +00:00
* and rows can be moved only forward (because we increase hash modulo twice on each rehash),
* so it is safe to add blocks.
*/
for (size_t bucket_idx = 0; bucket_idx < num_buckets; ++bucket_idx)
{
if (blocks[bucket_idx].rows() == 0)
continue;
2022-10-05 12:40:32 +00:00
if (bucket_idx == current_idx) // Rows that are still in our bucket
continue;
2022-06-16 12:09:23 +00:00
2022-10-05 12:40:32 +00:00
buckets[bucket_idx]->addLeftBlock(blocks[bucket_idx]);
}
} while (block.rows() == 0);
2022-06-16 12:09:23 +00:00
2022-10-05 12:40:32 +00:00
ExtraBlockPtr not_processed;
hash_join->joinBlock(block, not_processed);
2022-10-04 08:21:02 +00:00
2022-10-05 12:40:32 +00:00
if (not_processed)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unsupported hash join type");
2022-06-17 17:36:24 +00:00
2022-10-05 12:40:32 +00:00
return block;
2022-06-16 12:09:23 +00:00
}
2022-10-05 12:40:32 +00:00
size_t current_bucket;
Buckets buckets;
InMemoryJoinPtr hash_join;
2022-10-04 08:20:13 +00:00
2022-10-05 12:40:32 +00:00
AccumulatedBlockReader left_reader;
2022-10-04 08:20:13 +00:00
2022-10-05 12:40:32 +00:00
Names left_key_names;
Names right_key_names;
Block left_sample_block;
Block result_sample_block;
size_t max_block_size = 0;
bool is_left_reader_finished = false;
IBlocksStreamPtr non_joined_blocks = nullptr;
std::mutex non_joined_blocks_mutex;
2022-10-05 12:40:32 +00:00
};
2022-10-04 08:20:13 +00:00
2022-10-18 11:43:01 +00:00
IBlocksStreamPtr GraceHashJoin::getDelayedBlocks()
2022-10-05 12:40:32 +00:00
{
std::lock_guard current_bucket_lock(current_bucket_mutex);
2022-06-17 17:36:24 +00:00
2022-10-05 12:40:32 +00:00
if (current_bucket == nullptr)
return nullptr;
2022-06-16 12:29:29 +00:00
2022-10-05 12:40:32 +00:00
size_t bucket_idx = current_bucket->idx;
2022-06-16 12:09:23 +00:00
2023-05-08 01:29:29 +00:00
if (hash_join)
{
auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false);
for (auto & block : right_blocks)
{
Blocks blocks = JoinCommon::scatterBlockByHash(right_key_names, block, buckets.size());
flushBlocksToBuckets<JoinTableSide::Right>(blocks, buckets, bucket_idx);
}
}
2022-10-18 11:43:01 +00:00
hash_join = makeInMemoryJoin();
2022-10-04 08:20:13 +00:00
2022-10-18 11:43:01 +00:00
for (bucket_idx = bucket_idx + 1; bucket_idx < buckets.size(); ++bucket_idx)
{
2022-10-05 12:40:32 +00:00
current_bucket = buckets[bucket_idx].get();
2022-10-18 11:43:01 +00:00
if (current_bucket->finished() || current_bucket->empty())
2022-06-16 18:36:49 +00:00
{
2022-10-18 11:43:01 +00:00
LOG_TRACE(log, "Skipping {} {} bucket {}",
current_bucket->finished() ? "finished" : "",
current_bucket->empty() ? "empty" : "",
bucket_idx);
2022-10-05 12:40:32 +00:00
continue;
}
2022-10-05 12:40:32 +00:00
auto right_reader = current_bucket->startJoining();
size_t num_rows = 0; /// count rows that were written and rehashed
while (Block block = right_reader.read())
{
num_rows += block.rows();
addJoinedBlockImpl(std::move(block));
2022-06-16 18:36:49 +00:00
}
2022-06-16 12:09:23 +00:00
2022-10-05 12:40:32 +00:00
LOG_TRACE(log, "Loaded bucket {} with {}(/{}) rows",
bucket_idx, hash_join->getTotalRowCount(), num_rows);
auto result_sample_block = left_sample_block;
ExtraBlockPtr tmp;
hash_join->joinBlock(result_sample_block, tmp);
return std::make_unique<DelayedBlocks>(
current_bucket->idx,
buckets,
hash_join,
left_key_names,
right_key_names,
left_sample_block,
result_sample_block,
max_block_size);
2022-10-05 12:40:32 +00:00
}
2022-06-16 12:09:23 +00:00
LOG_TRACE(log, "Finished loading all {} buckets", buckets.size());
2022-10-18 11:43:01 +00:00
current_bucket = nullptr;
return nullptr;
2022-06-16 12:09:23 +00:00
}
2022-09-26 13:46:50 +00:00
GraceHashJoin::InMemoryJoinPtr GraceHashJoin::makeInMemoryJoin()
2022-06-16 12:09:23 +00:00
{
return std::make_unique<InMemoryJoin>(table_join, right_sample_block, any_take_last_row);
2022-06-16 12:09:23 +00:00
}
2022-12-20 12:50:27 +00:00
Block GraceHashJoin::prepareRightBlock(const Block & block)
{
return HashJoin::prepareRightBlock(block, hash_join_sample_block);
}
2022-10-05 12:40:32 +00:00
void GraceHashJoin::addJoinedBlockImpl(Block block)
2022-06-16 12:09:23 +00:00
{
2022-10-04 08:20:13 +00:00
Buckets buckets_snapshot = getCurrentBuckets();
2022-10-05 12:40:32 +00:00
size_t bucket_index = current_bucket->idx;
2022-12-20 12:50:27 +00:00
Block current_block;
{
Blocks blocks = JoinCommon::scatterBlockByHash(right_key_names, block, buckets_snapshot.size());
flushBlocksToBuckets<JoinTableSide::Right>(blocks, buckets_snapshot, bucket_index);
current_block = std::move(blocks[bucket_index]);
}
2022-06-16 12:09:23 +00:00
// Add block to the in-memory join
2022-12-20 12:50:27 +00:00
if (current_block.rows() > 0)
2022-06-17 17:36:24 +00:00
{
2022-10-05 12:40:32 +00:00
std::lock_guard lock(hash_join_mutex);
2022-10-04 08:20:13 +00:00
2023-01-23 12:34:36 +00:00
if (!hash_join)
hash_join = makeInMemoryJoin();
2022-12-20 12:50:27 +00:00
hash_join->addJoinedBlock(current_block, /* check_limits = */ false);
2022-10-04 08:20:13 +00:00
2022-12-27 10:13:12 +00:00
if (!hasMemoryOverflow(hash_join))
2022-12-20 12:50:27 +00:00
return;
2022-10-05 16:50:16 +00:00
2022-12-20 12:50:27 +00:00
current_block = {};
2022-10-05 16:50:16 +00:00
2022-12-20 12:50:27 +00:00
auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false);
2023-01-23 12:34:36 +00:00
hash_join = nullptr;
2022-12-20 12:50:27 +00:00
2022-12-27 10:13:12 +00:00
buckets_snapshot = rehashBuckets(buckets_snapshot.size() * 2);
2022-06-16 12:09:23 +00:00
{
2023-01-17 12:21:35 +00:00
Blocks current_blocks;
current_blocks.reserve(right_blocks.size());
for (const auto & right_block : right_blocks)
{
Blocks blocks = JoinCommon::scatterBlockByHash(right_key_names, right_block, buckets_snapshot.size());
flushBlocksToBuckets<JoinTableSide::Right>(blocks, buckets_snapshot, bucket_index);
current_blocks.emplace_back(std::move(blocks[bucket_index]));
}
2022-10-04 08:20:13 +00:00
2023-01-20 16:30:34 +00:00
if (current_blocks.size() == 1)
current_block = std::move(current_blocks.front());
else
current_block = concatenateBlocks(current_blocks);
}
2022-12-20 12:50:27 +00:00
hash_join = makeInMemoryJoin();
if (current_block.rows() > 0)
hash_join->addJoinedBlock(current_block, /* check_limits = */ false);
}
2022-06-16 12:09:23 +00:00
}
size_t GraceHashJoin::getNumBuckets() const
{
std::shared_lock lock(rehash_mutex);
return buckets.size();
}
2022-10-04 08:20:13 +00:00
GraceHashJoin::Buckets GraceHashJoin::getCurrentBuckets() const
{
2022-10-04 08:20:13 +00:00
std::shared_lock lock(rehash_mutex);
return buckets;
}
2022-06-06 17:26:22 +00:00
}