ClickHouse/src/Interpreters/GraceHashJoin.cpp

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

750 lines
24 KiB
C++
Raw Normal View History

#include <Compression/CompressedWriteBuffer.h>
#include <Formats/NativeWriter.h>
#include <Formats/formatBlock.h>
#include <Interpreters/Context.h>
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/TemporaryDataOnDisk.h>
#include <base/FnTraits.h>
#include <Common/formatReadable.h>
#include <Common/logger_useful.h>
#include <Common/thread_local_rng.h>
2022-06-16 12:09:23 +00:00
2023-08-09 00:19:02 +00:00
#include <numeric>
#include <fmt/format.h>
2023-08-09 00:19:02 +00:00
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>;
2024-01-23 17:04:50 +00:00
explicit FileBucket(size_t bucket_index_, TemporaryFileStream & left_file_, TemporaryFileStream & right_file_, LoggerPtr 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
2024-01-23 17:04:50 +00:00
LoggerPtr 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_)
2024-01-23 17:04:50 +00:00
: log{getLogger("GraceHashJoin")}
2022-06-16 12:09:23 +00:00
, 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))
2023-11-28 10:28:11 +00:00
, hash_join(makeInMemoryJoin("grace0"))
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
addBuckets(initial_num_buckets);
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;
2023-07-05 17:03:18 +00:00
bool GraceHashJoin::addBlockToJoin(const Block & block, bool /*check_limits*/)
2022-06-16 12:09:23 +00:00
{
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);
2023-07-05 17:03:18 +00:00
addBlockToJoinImpl(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
}
GraceHashJoin::Buckets GraceHashJoin::rehashBuckets()
2022-06-16 12:09:23 +00:00
{
2022-10-04 08:20:13 +00:00
std::unique_lock lock(rehash_mutex);
2022-10-05 12:40:32 +00:00
if (!isPowerOf2(buckets.size())) [[unlikely]]
throw Exception(ErrorCodes::LOGICAL_ERROR, "Number of buckets should be power of 2 but it's {}", buckets.size());
2022-10-05 12:40:32 +00:00
const size_t to_size = buckets.size() * 2;
size_t current_size = buckets.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
{
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",
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);
addBuckets(to_size - current_size);
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
}
void GraceHashJoin::addBuckets(const size_t bucket_count)
2022-06-16 12:09:23 +00:00
{
// Exception can be thrown in number of cases:
// - during creation of temporary files for buckets
// - in CI tests, there is a certain probability of failure in allocating memory, see memory_tracker_fault_probability
// Therefore, new buckets are added only after all of them created successfully,
// otherwise we can end up having unexpected number of buckets
const size_t current_size = buckets.size();
Buckets tmp_buckets;
tmp_buckets.reserve(bucket_count);
for (size_t i = 0; i < bucket_count; ++i)
try
{
auto & left_file = tmp_data->createStream(left_sample_block);
auto & right_file = tmp_data->createStream(prepareRightBlock(right_sample_block));
2023-01-20 16:30:34 +00:00
BucketPtr new_bucket = std::make_shared<FileBucket>(current_size + i, left_file, right_file, log);
tmp_buckets.emplace_back(std::move(new_bucket));
}
catch (...)
{
LOG_ERROR(
2024-01-23 17:04:50 +00:00
getLogger("GraceHashJoin"),
"Can't create bucket {} due to error: {}",
current_size + i,
getCurrentExceptionMessage(false));
throw;
}
buckets.reserve(buckets.size() + bucket_count);
2023-07-04 15:46:40 +00:00
for (auto & bucket : tmp_buckets)
buckets.emplace_back(std::move(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();
2023-11-20 16:16:37 +00:00
ExtraBlockPtr not_processed = nullptr;
2022-11-29 11:46:11 +00:00
hash_join->joinBlock(output_sample_block, not_processed);
2023-11-20 16:16:37 +00:00
if (not_processed)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unhandled not processed block in GraceHashJoin");
2022-11-29 11:46:11 +00:00
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-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
}
/// Each bucket are handled by the following steps
/// 1. build hash_join by the right side blocks.
/// 2. join left side with the hash_join,
/// 3. read right non-joined blocks from hash_join.
/// buckets are handled one by one, each hash_join will not be release before the right non-joined blocks are emitted.
///
/// There is a finished counter in JoiningTransform/DelayedJoinedBlocksWorkerTransform,
/// only one processor could take the non-joined blocks from right stream, and ensure all rows from
/// left stream have been emitted before this.
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_)
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_)
2022-06-16 12:09:23 +00:00
{
}
2022-10-05 12:40:32 +00:00
Block nextImpl() override
{
ExtraBlockPtr not_processed = nullptr;
{
std::lock_guard lock(extra_block_mutex);
if (!not_processed_blocks.empty())
{
not_processed = std::move(not_processed_blocks.front());
not_processed_blocks.pop_front();
}
}
2023-11-20 16:16:37 +00:00
if (not_processed)
{
Block block = std::move(not_processed->block);
2023-11-20 16:16:37 +00:00
hash_join->joinBlock(block, not_processed);
if (not_processed)
{
std::lock_guard lock(extra_block_mutex);
not_processed_blocks.emplace_back(std::move(not_processed));
}
2023-11-20 16:16:37 +00:00
return block;
}
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
{
// One DelayedBlocks is shared among multiple DelayedJoinedBlocksWorkerTransform.
// There is a lock inside left_reader.read() .
block = left_reader.read();
if (!block)
{
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
hash_join->joinBlock(block, not_processed);
if (not_processed)
{
std::lock_guard lock(extra_block_mutex);
not_processed_blocks.emplace_back(std::move(not_processed));
}
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;
2023-11-20 16:16:37 +00:00
std::mutex extra_block_mutex;
std::list<ExtraBlockPtr> not_processed_blocks TSA_GUARDED_BY(extra_block_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-07-10 08:44:01 +00:00
size_t prev_keys_num = 0;
if (hash_join && buckets.size() > 1)
2023-05-12 06:40:17 +00:00
{
2023-07-10 08:44:01 +00:00
prev_keys_num = hash_join->getTotalRowCount();
2023-05-12 06:40:17 +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;
}
2023-05-17 06:41:33 +00:00
2023-11-28 10:28:11 +00:00
hash_join = makeInMemoryJoin(fmt::format("grace{}", bucket_idx), prev_keys_num);
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();
2023-07-05 17:03:18 +00:00
addBlockToJoinImpl(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);
return std::make_unique<DelayedBlocks>(current_bucket->idx, buckets, hash_join, left_key_names, right_key_names);
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
}
2023-11-28 10:28:11 +00:00
GraceHashJoin::InMemoryJoinPtr GraceHashJoin::makeInMemoryJoin(const String & bucket_id, size_t reserve_num)
2022-06-16 12:09:23 +00:00
{
2023-11-28 10:28:11 +00:00
return std::make_unique<HashJoin>(table_join, right_sample_block, any_take_last_row, reserve_num, bucket_id);
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);
}
2023-07-05 17:03:18 +00:00
void GraceHashJoin::addBlockToJoinImpl(Block block)
2022-06-16 12:09:23 +00:00
{
block = prepareRightBlock(block);
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);
2023-01-23 12:34:36 +00:00
if (!hash_join)
2023-11-28 10:28:11 +00:00
hash_join = makeInMemoryJoin(fmt::format("grace{}", bucket_index));
2023-01-23 12:34:36 +00:00
2023-07-10 08:44:01 +00:00
// buckets size has been changed in other threads. Need to scatter current_block again.
// rehash could only happen under hash_join_mutex's scope.
auto current_buckets = getCurrentBuckets();
if (buckets_snapshot.size() != current_buckets.size())
{
LOG_TRACE(log, "mismatch buckets size. previous:{}, current:{}", buckets_snapshot.size(), getCurrentBuckets().size());
Blocks blocks = JoinCommon::scatterBlockByHash(right_key_names, current_block, current_buckets.size());
flushBlocksToBuckets<JoinTableSide::Right>(blocks, current_buckets, bucket_index);
current_block = std::move(blocks[bucket_index]);
if (!current_block.rows())
return;
}
auto prev_keys_num = hash_join->getTotalRowCount();
2023-07-05 17:03:18 +00:00
hash_join->addBlockToJoin(current_block, /* check_limits = */ false);
2022-10-04 08:20:13 +00:00
2023-05-17 06:41:33 +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
2023-06-25 03:07:15 +00:00
// Must use the latest buckets snapshot in case that it has been rehashed by other threads.
buckets_snapshot = rehashBuckets();
2023-05-17 06:41:33 +00:00
auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false);
hash_join = nullptr;
2022-12-20 12:50:27 +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);
}
2023-11-28 10:28:11 +00:00
hash_join = makeInMemoryJoin(fmt::format("grace{}", bucket_index), prev_keys_num);
2022-12-20 12:50:27 +00:00
if (current_block.rows() > 0)
2023-07-05 17:03:18 +00:00
hash_join->addBlockToJoin(current_block, /* check_limits = */ false);
2022-12-20 12:50:27 +00:00
}
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
}