2024-03-11 13:52:32 +00:00
|
|
|
#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>
|
2022-09-30 11:07:49 +00:00
|
|
|
#include <Interpreters/TemporaryDataOnDisk.h>
|
2024-03-11 13:52:32 +00:00
|
|
|
#include <base/FnTraits.h>
|
2023-12-23 13:46:21 +00:00
|
|
|
#include <Common/formatReadable.h>
|
2022-06-23 14:41:11 +00:00
|
|
|
#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>
|
2024-03-11 13:52:32 +00:00
|
|
|
#include <fmt/format.h>
|
2023-08-09 00:19:02 +00:00
|
|
|
|
|
|
|
|
2022-09-30 11:07:49 +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_)
|
2022-06-19 16:13:04 +00:00
|
|
|
{
|
2022-09-30 11:07:49 +00:00
|
|
|
if (!reader.isWriteFinished())
|
|
|
|
throw Exception(ErrorCodes::LOGICAL_ERROR, "Reading not finished file");
|
2022-06-19 16:13:04 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
Block read()
|
|
|
|
{
|
2023-03-02 13:36:47 +00:00
|
|
|
std::lock_guard lock(mutex);
|
2022-10-26 09:54:07 +00:00
|
|
|
|
2022-06-19 16:13:04 +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)
|
2022-06-19 16:13:04 +00:00
|
|
|
{
|
|
|
|
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-06-19 16:13:04 +00:00
|
|
|
}
|
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);
|
2022-06-19 16:13:04 +00:00
|
|
|
|
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-06-19 16:13:04 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
private:
|
2022-09-30 11:07:49 +00:00
|
|
|
TemporaryFileStream & reader;
|
2022-10-05 12:40:32 +00:00
|
|
|
std::mutex & mutex;
|
|
|
|
|
|
|
|
const size_t result_block_size;
|
2022-06-19 16:13:04 +00:00
|
|
|
bool eof = false;
|
|
|
|
};
|
|
|
|
|
2022-06-23 01:41:21 +00:00
|
|
|
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();
|
2022-06-23 01:41:21 +00:00
|
|
|
indices.pop_front();
|
|
|
|
|
2022-11-27 11:16:31 +00:00
|
|
|
if (!callback(bucket_index))
|
|
|
|
indices.push_back(bucket_index);
|
2022-06-23 01:41:21 +00:00
|
|
|
}
|
|
|
|
}
|
2022-06-11 11:03:44 +00:00
|
|
|
}
|
|
|
|
|
2022-09-30 11:54:40 +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_}
|
2022-09-30 11:07:49 +00:00
|
|
|
, 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
|
|
|
{
|
|
|
|
}
|
|
|
|
|
2022-09-30 11:07:49 +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-09-30 11:07:49 +00:00
|
|
|
}
|
|
|
|
|
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
|
|
|
}
|
|
|
|
|
2022-09-30 11:07:49 +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)
|
2022-06-23 01:41:21 +00:00
|
|
|
{
|
|
|
|
ensureState(State::WRITING_BLOCKS);
|
2022-10-07 09:16:09 +00:00
|
|
|
|
2022-06-23 01:41:21 +00:00
|
|
|
if (!lock.owns_lock())
|
|
|
|
return false;
|
2022-09-30 11:07:49 +00:00
|
|
|
|
2022-10-07 09:16:09 +00:00
|
|
|
if (block.rows())
|
|
|
|
is_empty = false;
|
|
|
|
|
2022-06-23 01:41:21 +00:00
|
|
|
writer.write(block);
|
2022-10-04 08:20:13 +00:00
|
|
|
return true;
|
2022-06-23 01:41:21 +00:00
|
|
|
}
|
|
|
|
|
2022-06-17 17:36:24 +00:00
|
|
|
void transition(State expected, State desired)
|
|
|
|
{
|
|
|
|
State prev = state.exchange(desired);
|
2022-09-30 11:07:49 +00:00
|
|
|
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
|
|
|
}
|
|
|
|
|
2022-09-30 11:07:49 +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;
|
2022-09-30 11:07:49 +00:00
|
|
|
|
|
|
|
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
|
|
|
};
|
|
|
|
|
2022-11-30 22:09:16 +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
|
|
|
{
|
2022-11-30 22:09:16 +00:00
|
|
|
chassert(blocks.size() == buckets.size());
|
2022-10-05 12:40:32 +00:00
|
|
|
retryForEach(
|
2022-11-30 22:09:16 +00:00
|
|
|
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;
|
2022-11-30 22:09:16 +00:00
|
|
|
|
|
|
|
bool flushed = false;
|
2022-12-16 18:33:01 +00:00
|
|
|
if constexpr (table_side == JoinTableSide::Left)
|
2022-11-30 22:09:16 +00:00
|
|
|
flushed = buckets[i]->tryAddLeftBlock(blocks[i]);
|
2022-12-16 18:33:01 +00:00
|
|
|
if constexpr (table_side == JoinTableSide::Right)
|
2022-11-30 22:09:16 +00:00
|
|
|
flushed = buckets[i]->tryAddRightBlock(blocks[i]);
|
|
|
|
|
2022-10-05 12:40:32 +00:00
|
|
|
if (flushed)
|
|
|
|
blocks[i].clear();
|
2022-11-30 22:09:16 +00:00
|
|
|
|
2022-10-05 12:40:32 +00:00
|
|
|
return flushed;
|
|
|
|
});
|
|
|
|
}
|
2022-11-30 22:09:16 +00:00
|
|
|
}
|
2022-06-23 01:41:21 +00:00
|
|
|
|
2022-06-16 12:09:23 +00:00
|
|
|
GraceHashJoin::GraceHashJoin(
|
2022-09-30 11:07:49 +00:00
|
|
|
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_)}
|
2022-09-30 11:07:49 +00:00
|
|
|
, 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}
|
2022-06-23 14:41:11 +00:00
|
|
|
, 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)
|
2022-09-30 11:07:49 +00:00
|
|
|
, 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()
|
|
|
|
{
|
2022-12-19 15:15:52 +00:00
|
|
|
if (!buckets.empty())
|
|
|
|
return;
|
|
|
|
|
2022-09-30 11:54:40 +00:00
|
|
|
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
|
|
|
|
2023-07-04 09:00:53 +00:00
|
|
|
addBuckets(initial_num_buckets);
|
2022-09-30 11:07:49 +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);
|
2023-06-15 00:05:47 +00:00
|
|
|
auto kind = table_join->kind();
|
|
|
|
return !is_asof && (isInner(kind) || isLeft(kind) || isRight(kind) || isFull(kind)) && table_join->oneDisjunct();
|
2022-06-23 13:55:15 +00:00
|
|
|
}
|
|
|
|
|
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-19 15:19:49 +00:00
|
|
|
|
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
|
|
|
}
|
|
|
|
|
2023-07-03 21:58:56 +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
|
|
|
|
2023-07-03 21:58:56 +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
|
|
|
|
2023-07-03 21:58:56 +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
|
|
|
{
|
2023-07-04 09:00:53 +00:00
|
|
|
throw Exception(
|
|
|
|
ErrorCodes::LIMIT_EXCEEDED,
|
2023-01-23 21:13:58 +00:00
|
|
|
"Too many grace hash join buckets ({} > {}), "
|
|
|
|
"consider increasing grace_hash_join_max_buckets or max_rows_in_join/max_bytes_in_join",
|
2023-07-04 09:00:53 +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-09-30 11:54:40 +00:00
|
|
|
|
2023-07-04 09:00:53 +00:00
|
|
|
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
|
|
|
}
|
|
|
|
|
2023-07-04 09:00:53 +00:00
|
|
|
void GraceHashJoin::addBuckets(const size_t bucket_count)
|
2022-06-16 12:09:23 +00:00
|
|
|
{
|
2023-07-04 09:00:53 +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
|
|
|
|
2023-07-04 09:00:53 +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"),
|
2023-07-04 09:00:53 +00:00
|
|
|
"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)
|
2023-07-04 09:00:53 +00:00
|
|
|
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-09-30 11:54:40 +00:00
|
|
|
|
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
|
|
|
|
{
|
2022-09-30 11:54:40 +00:00
|
|
|
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(); });
|
|
|
|
}();
|
2022-09-30 11:54:40 +00:00
|
|
|
|
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
|
|
|
}
|
2023-06-08 09:40:41 +00:00
|
|
|
|
2023-06-15 00:05:47 +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
|
|
|
{
|
2023-06-15 00:05:47 +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:
|
2023-06-15 00:05:47 +00:00
|
|
|
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
|
2022-06-23 14:41:11 +00:00
|
|
|
{
|
2023-11-21 13:36:53 +00:00
|
|
|
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)
|
|
|
|
{
|
2023-11-21 13:36:53 +00:00
|
|
|
Block block = std::move(not_processed->block);
|
2023-11-20 16:16:37 +00:00
|
|
|
hash_join->joinBlock(block, not_processed);
|
2023-11-21 13:36:53 +00:00
|
|
|
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-06-23 14:41:11 +00:00
|
|
|
|
2022-10-05 12:40:32 +00:00
|
|
|
do
|
2022-06-23 14:41:11 +00:00
|
|
|
{
|
2023-06-15 00:05:47 +00:00
|
|
|
// One DelayedBlocks is shared among multiple DelayedJoinedBlocksWorkerTransform.
|
|
|
|
// There is a lock inside left_reader.read() .
|
2023-05-09 03:17:09 +00:00
|
|
|
block = left_reader.read();
|
|
|
|
if (!block)
|
2023-05-04 02:17:04 +00:00
|
|
|
{
|
2023-05-09 03:17:09 +00:00
|
|
|
return {};
|
2022-10-19 10:20:41 +00:00
|
|
|
}
|
2022-06-23 14:41:11 +00:00
|
|
|
|
2023-06-15 00:05:47 +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-06-23 14:41:11 +00:00
|
|
|
|
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);
|
2023-11-21 13:36:53 +00:00
|
|
|
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
|
|
|
|
2023-11-21 13:36:53 +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
|
|
|
{
|
2023-03-03 16:28:41 +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);
|
2023-06-08 09:40:41 +00:00
|
|
|
|
2023-05-09 03:17:09 +00:00
|
|
|
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
|
|
|
|
2022-12-19 15:19:49 +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
|
|
|
{
|
2023-05-31 09:37:20 +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
|
|
|
|
2022-06-23 01:41:21 +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.
|
2023-07-03 21:58:56 +00:00
|
|
|
buckets_snapshot = rehashBuckets();
|
2023-05-17 06:41:33 +00:00
|
|
|
auto right_blocks = hash_join->releaseJoinedBlocks(/* restructure */ false);
|
2023-06-08 09:40:11 +00:00
|
|
|
hash_join = nullptr;
|
2022-12-20 12:50:27 +00:00
|
|
|
|
2022-06-23 01:41:21 +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-06-23 01:41:21 +00:00
|
|
|
}
|
|
|
|
|
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
|
|
|
}
|
|
|
|
|
2022-09-30 11:54:40 +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-09-30 11:54:40 +00:00
|
|
|
{
|
2022-10-04 08:20:13 +00:00
|
|
|
std::shared_lock lock(rehash_mutex);
|
2022-09-30 11:54:40 +00:00
|
|
|
return buckets;
|
|
|
|
}
|
|
|
|
|
2022-06-06 17:26:22 +00:00
|
|
|
}
|