patch 4.0
This commit is contained in:
@ -19,32 +19,27 @@
|
||||
#include "lib/utility/ob_unify_serialize.h"
|
||||
#include "sql/engine/basic/ob_chunk_datum_store.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
namespace oceanbase
|
||||
{
|
||||
namespace sql
|
||||
{
|
||||
|
||||
class ObHashJoinBufMgr : public ObSqlMemoryCallback {
|
||||
class ObHashJoinBufMgr : public ObSqlMemoryCallback
|
||||
{
|
||||
public:
|
||||
ObHashJoinBufMgr()
|
||||
: reserve_memory_size_(0),
|
||||
pre_total_alloc_size_(0),
|
||||
total_alloc_size_(0),
|
||||
total_left_alloc_size_(0),
|
||||
page_size_(-1),
|
||||
dumped_size_(0)
|
||||
ObHashJoinBufMgr() :
|
||||
reserve_memory_size_(0), data_ratio_(1.0), pre_total_alloc_size_(0), total_alloc_size_(0),
|
||||
page_size_(-1), dumped_size_(0)
|
||||
{}
|
||||
virtual ~ObHashJoinBufMgr() {}
|
||||
|
||||
inline void set_page_size(int64_t page_size)
|
||||
{
|
||||
page_size_ = page_size;
|
||||
}
|
||||
inline int64_t get_page_size()
|
||||
{
|
||||
return page_size_;
|
||||
}
|
||||
inline void set_page_size(int64_t page_size) { page_size_ = page_size; }
|
||||
inline int64_t get_page_size() { return page_size_; }
|
||||
|
||||
void set_reserve_memory_size(int64_t reserve_memory_size)
|
||||
void set_reserve_memory_size(int64_t reserve_memory_size, double ratio)
|
||||
{
|
||||
reserve_memory_size_ = reserve_memory_size;
|
||||
data_ratio_ = ratio;
|
||||
}
|
||||
int64_t get_reserve_memory_size()
|
||||
{
|
||||
@ -67,6 +62,11 @@ public:
|
||||
return dumped_size_;
|
||||
}
|
||||
|
||||
double get_data_ratio()
|
||||
{
|
||||
return data_ratio_;
|
||||
}
|
||||
|
||||
void reuse()
|
||||
{
|
||||
dumped_size_ = 0;
|
||||
@ -88,220 +88,184 @@ public:
|
||||
return reserve_memory_size_ < page_size_ + total_alloc_size_;
|
||||
}
|
||||
|
||||
// OB_INLINE void finish_left_dump() { total_left_alloc_size_ = total_alloc_size_; }
|
||||
|
||||
OB_INLINE bool need_dump()
|
||||
{
|
||||
return reserve_memory_size_ * RATIO / 100 < total_alloc_size_;
|
||||
}
|
||||
|
||||
private:
|
||||
const static int64_t RATIO = 80;
|
||||
int64_t reserve_memory_size_;
|
||||
double data_ratio_;
|
||||
int64_t pre_total_alloc_size_;
|
||||
int64_t total_alloc_size_;
|
||||
int64_t total_left_alloc_size_;
|
||||
int64_t page_size_;
|
||||
int64_t dumped_size_;
|
||||
};
|
||||
|
||||
struct ObHashJoinStoredJoinRow : public sql::ObChunkDatumStore::StoredRow {
|
||||
struct ObHashJoinStoredJoinRow : public sql::ObChunkDatumStore::StoredRow
|
||||
{
|
||||
const static int64_t HASH_VAL_BIT = 63;
|
||||
const static int64_t STORED_ROW_PTR_BIT = 63;
|
||||
const static int64_t HASH_VAL_MASK = UINT64_MAX >> (64 - HASH_VAL_BIT);
|
||||
struct ExtraInfo {
|
||||
uint64_t hash_val_ : HASH_VAL_BIT;
|
||||
uint64_t is_match_ : 1;
|
||||
};
|
||||
struct ExtraInfo
|
||||
{
|
||||
union {
|
||||
// %hash_value_ is set when row is add to chunk datum store.
|
||||
struct {
|
||||
uint64_t hash_val_:HASH_VAL_BIT;
|
||||
uint64_t is_match_:1;
|
||||
};
|
||||
// %next_row_ is set when stored row added to hash table.
|
||||
struct {
|
||||
uint64_t next_row_:STORED_ROW_PTR_BIT;
|
||||
};
|
||||
uint64_t v_;
|
||||
};
|
||||
|
||||
ExtraInfo& get_extra_info()
|
||||
ObHashJoinStoredJoinRow *get_next() const
|
||||
{
|
||||
return reinterpret_cast<ObHashJoinStoredJoinRow *>(next_row_);
|
||||
}
|
||||
|
||||
void set_next(ObHashJoinStoredJoinRow *ptr)
|
||||
{
|
||||
next_row_ = reinterpret_cast<uint64_t>(ptr);
|
||||
}
|
||||
};
|
||||
STATIC_ASSERT(sizeof(uint64_t) == sizeof(ExtraInfo), "unexpected size");
|
||||
|
||||
ExtraInfo &get_extra_info()
|
||||
{
|
||||
static_assert(sizeof(ObHashJoinStoredJoinRow) == sizeof(sql::ObChunkDatumStore::StoredRow),
|
||||
"sizeof StoredJoinRow must be the save with StoredRow");
|
||||
return *reinterpret_cast<ExtraInfo*>(get_extra_payload());
|
||||
}
|
||||
const ExtraInfo& get_extra_info() const
|
||||
{
|
||||
return *reinterpret_cast<const ExtraInfo*>(get_extra_payload());
|
||||
return *reinterpret_cast<ExtraInfo *>(get_extra_payload());
|
||||
}
|
||||
const ExtraInfo &get_extra_info() const
|
||||
{ return *reinterpret_cast<const ExtraInfo *>(get_extra_payload()); }
|
||||
|
||||
uint64_t get_hash_value() const
|
||||
{
|
||||
return get_extra_info().hash_val_;
|
||||
}
|
||||
uint64_t get_hash_value() const { return get_extra_info().hash_val_; }
|
||||
void set_hash_value(const uint64_t hash_val)
|
||||
{
|
||||
get_extra_info().hash_val_ = hash_val & HASH_VAL_MASK;
|
||||
}
|
||||
bool is_match() const
|
||||
{
|
||||
return get_extra_info().is_match_;
|
||||
}
|
||||
void set_is_match(bool is_match)
|
||||
{
|
||||
get_extra_info().is_match_ = is_match;
|
||||
}
|
||||
{ get_extra_info().hash_val_ = hash_val & HASH_VAL_MASK; }
|
||||
bool is_match() const { return get_extra_info().is_match_; }
|
||||
void set_is_match(bool is_match) { get_extra_info().is_match_ = is_match; }
|
||||
|
||||
ObHashJoinStoredJoinRow *get_next() const { return get_extra_info().get_next(); }
|
||||
void set_next(ObHashJoinStoredJoinRow *ptr) { get_extra_info().set_next(ptr); }
|
||||
};
|
||||
|
||||
class ObHashJoinBatch {
|
||||
public:
|
||||
ObHashJoinBatch(
|
||||
common::ObIAllocator& alloc, ObHashJoinBufMgr* buf_mgr, uint64_t tenant_id, int32_t part_level, int32_t batchno)
|
||||
: alloc_(alloc),
|
||||
chunk_row_store_(&alloc),
|
||||
inner_callback_(nullptr),
|
||||
part_level_(part_level),
|
||||
batchno_(batchno),
|
||||
is_chunk_iter_(false),
|
||||
buf_mgr_(buf_mgr),
|
||||
tenant_id_(tenant_id),
|
||||
timeout_ms_(1000000),
|
||||
n_get_rows_(0),
|
||||
n_add_rows_(0),
|
||||
pre_total_size_(0),
|
||||
pre_bucket_number_(0),
|
||||
pre_part_count_(0)
|
||||
common::ObIAllocator &alloc,
|
||||
ObHashJoinBufMgr *buf_mgr,
|
||||
uint64_t tenant_id,
|
||||
int32_t part_level,
|
||||
int64_t part_shift,
|
||||
int64_t batchno)
|
||||
: chunk_row_store_(&alloc),
|
||||
inner_callback_(nullptr),
|
||||
part_level_(part_level),
|
||||
part_shift_(part_shift),
|
||||
batchno_(batchno),
|
||||
is_chunk_iter_(false),
|
||||
buf_mgr_(buf_mgr),
|
||||
tenant_id_(tenant_id),
|
||||
n_get_rows_(0),
|
||||
n_add_rows_(0),
|
||||
pre_total_size_(0),
|
||||
pre_bucket_number_(0),
|
||||
pre_part_count_(0)
|
||||
{}
|
||||
|
||||
virtual ~ObHashJoinBatch();
|
||||
int get_next_batch(const common::ObIArray<ObExpr*> &exprs,
|
||||
ObEvalCtx &ctx,
|
||||
const int64_t max_rows,
|
||||
int64_t &read_rows,
|
||||
const ObHashJoinStoredJoinRow **stored_row);
|
||||
int get_next_batch(const ObHashJoinStoredJoinRow **stored_row,
|
||||
const int64_t max_rows,
|
||||
int64_t &read_rows);
|
||||
int get_next_row(
|
||||
const common::ObIArray<ObExpr*>& exprs, ObEvalCtx& eval_ctx, const ObHashJoinStoredJoinRow*& stored_row);
|
||||
int get_next_row(const ObHashJoinStoredJoinRow*& stored_row);
|
||||
int get_next_block_row(const ObHashJoinStoredJoinRow*& stored_row);
|
||||
const common::ObIArray<ObExpr*> &exprs,
|
||||
ObEvalCtx &eval_ctx,
|
||||
const ObHashJoinStoredJoinRow *&stored_row);
|
||||
int get_next_row(const ObHashJoinStoredJoinRow *&stored_row);
|
||||
int get_next_block_row(const ObHashJoinStoredJoinRow *&stored_row);
|
||||
int convert_row(
|
||||
const ObHashJoinStoredJoinRow* stored_row, const common::ObIArray<ObExpr*>& exprs, ObEvalCtx& eval_ctx);
|
||||
int add_row(const common::ObIArray<ObExpr*>& exprs, ObEvalCtx* eval_ctx, ObHashJoinStoredJoinRow*& stored_row);
|
||||
int add_row(const ObHashJoinStoredJoinRow* src_stored_row, ObHashJoinStoredJoinRow*& stored_row);
|
||||
const ObHashJoinStoredJoinRow *stored_row,
|
||||
const common::ObIArray<ObExpr*> &exprs,
|
||||
ObEvalCtx &eval_ctx);
|
||||
int add_row(
|
||||
const common::ObIArray<ObExpr*> &exprs,
|
||||
ObEvalCtx *eval_ctx,
|
||||
ObHashJoinStoredJoinRow *&stored_row);
|
||||
int add_batch(const common::ObIArray<ObExpr *> &exprs, ObEvalCtx &ctx,
|
||||
const ObBitVector &skip, const int64_t batch_size,
|
||||
const uint16_t selector[], const int64_t size,
|
||||
ObHashJoinStoredJoinRow **stored_rows = nullptr);
|
||||
int add_row(const ObHashJoinStoredJoinRow *src_stored_row, ObHashJoinStoredJoinRow *&stored_row);
|
||||
int init();
|
||||
int open();
|
||||
int close();
|
||||
|
||||
int finish_dump(bool memory_need_dump);
|
||||
int dump(bool all_dump);
|
||||
int dump(bool all_dump, int64_t dumped_size);
|
||||
|
||||
bool has_next()
|
||||
{
|
||||
return chunk_iter_.has_next_chunk();
|
||||
}
|
||||
bool has_next() { return chunk_iter_.has_next_chunk(); }
|
||||
int set_iterator(bool is_chunk_iter);
|
||||
int init_progressive_iterator();
|
||||
|
||||
void set_part_level(int32_t part_level)
|
||||
{
|
||||
part_level_ = part_level;
|
||||
}
|
||||
void set_batchno(int32_t batchno)
|
||||
{
|
||||
batchno_ = batchno;
|
||||
}
|
||||
void set_part_level(int32_t part_level) { part_level_ = part_level; }
|
||||
void set_batchno(int64_t batchno) { batchno_ = batchno; }
|
||||
|
||||
int32_t get_part_level() const
|
||||
{
|
||||
return part_level_;
|
||||
}
|
||||
int32_t get_batchno() const
|
||||
{
|
||||
return batchno_;
|
||||
}
|
||||
int32_t get_part_level() const { return part_level_; }
|
||||
int64_t get_batchno() const { return batchno_; }
|
||||
int64_t get_part_shift() const { return part_shift_; }
|
||||
|
||||
int64_t get_cur_row_count()
|
||||
{
|
||||
return n_get_rows_;
|
||||
}
|
||||
int64_t get_cur_row_count() { return n_get_rows_; }
|
||||
|
||||
void set_callback(ObSqlMemoryCallback* callback)
|
||||
{
|
||||
inner_callback_ = callback;
|
||||
}
|
||||
void set_callback(ObSqlMemoryCallback *callback) { inner_callback_ = callback; }
|
||||
|
||||
int load_next_chunk();
|
||||
int rescan();
|
||||
|
||||
int64_t get_row_count_in_memory()
|
||||
{
|
||||
return chunk_row_store_.get_row_cnt_in_memory();
|
||||
}
|
||||
int64_t get_row_count_on_disk()
|
||||
{
|
||||
return chunk_row_store_.get_row_cnt_on_disk();
|
||||
}
|
||||
int64_t get_row_count_in_memory() { return chunk_row_store_.get_row_cnt_in_memory(); }
|
||||
int64_t get_row_count_on_disk() { return chunk_row_store_.get_row_cnt_on_disk(); }
|
||||
|
||||
int64_t get_size_in_memory()
|
||||
{
|
||||
return chunk_row_store_.get_mem_used();
|
||||
}
|
||||
int64_t get_size_on_disk()
|
||||
{
|
||||
return chunk_row_store_.get_file_size();
|
||||
}
|
||||
int64_t get_cur_chunk_row_cnt()
|
||||
{
|
||||
return chunk_iter_.get_cur_chunk_row_cnt();
|
||||
}
|
||||
int64_t get_cur_chunk_size()
|
||||
{
|
||||
return chunk_iter_.get_chunk_read_size();
|
||||
}
|
||||
int64_t get_last_buffer_mem_size() { return chunk_row_store_.get_last_buffer_mem_size(); }
|
||||
int64_t get_size_in_memory() { return chunk_row_store_.get_mem_used(); }
|
||||
int64_t get_size_on_disk() { return chunk_row_store_.get_file_size(); }
|
||||
int64_t get_cur_chunk_row_cnt() { return chunk_iter_.get_cur_chunk_row_cnt(); }
|
||||
int64_t get_cur_chunk_size() { return chunk_iter_.get_chunk_read_size();}
|
||||
|
||||
bool has_switch_block()
|
||||
{
|
||||
return chunk_row_store_.get_block_list_cnt() > 1;
|
||||
}
|
||||
bool has_switch_block() { return chunk_row_store_.get_block_list_cnt() > 1; }
|
||||
|
||||
void set_pre_total_size(int64_t pre_total_size)
|
||||
{
|
||||
pre_total_size_ = pre_total_size;
|
||||
}
|
||||
void set_pre_bucket_number(int64_t pre_bucket_number)
|
||||
{
|
||||
pre_bucket_number_ = pre_bucket_number;
|
||||
}
|
||||
void set_pre_part_count(int64_t pre_part_count)
|
||||
{
|
||||
pre_part_count_ = pre_part_count;
|
||||
}
|
||||
int64_t get_pre_total_size()
|
||||
{
|
||||
return pre_total_size_;
|
||||
}
|
||||
int64_t get_pre_bucket_number()
|
||||
{
|
||||
return pre_bucket_number_;
|
||||
}
|
||||
int64_t get_pre_part_count()
|
||||
{
|
||||
return pre_part_count_;
|
||||
}
|
||||
void set_pre_total_size(int64_t pre_total_size) { pre_total_size_ = pre_total_size; }
|
||||
void set_pre_bucket_number(int64_t pre_bucket_number) { pre_bucket_number_ = pre_bucket_number; }
|
||||
void set_pre_part_count(int64_t pre_part_count) { pre_part_count_ = pre_part_count; }
|
||||
int64_t get_pre_total_size() { return pre_total_size_; }
|
||||
int64_t get_pre_bucket_number() { return pre_bucket_number_; }
|
||||
int64_t get_pre_part_count() { return pre_part_count_; }
|
||||
|
||||
void set_memory_limit(int64_t limit)
|
||||
{
|
||||
chunk_row_store_.set_mem_limit(limit);
|
||||
}
|
||||
bool is_dumped()
|
||||
{
|
||||
return 0 < chunk_row_store_.get_file_size();
|
||||
}
|
||||
int64_t get_dump_size()
|
||||
{
|
||||
return chunk_row_store_.get_file_size();
|
||||
}
|
||||
sql::ObChunkDatumStore& get_chunk_row_store()
|
||||
void set_memory_limit(int64_t limit) { chunk_row_store_.set_mem_limit(limit); }
|
||||
bool is_dumped() { return 0 < chunk_row_store_.get_file_size(); }
|
||||
int64_t get_dump_size() { return chunk_row_store_.get_file_size(); }
|
||||
sql::ObChunkDatumStore &get_chunk_row_store()
|
||||
{
|
||||
return chunk_row_store_;
|
||||
}
|
||||
|
||||
private:
|
||||
const int64_t ROW_CNT_PER = 0x3FF;
|
||||
common::ObIAllocator& alloc_;
|
||||
sql::ObChunkDatumStore chunk_row_store_;
|
||||
sql::ObChunkDatumStore::RowIterator row_store_iter_;
|
||||
sql::ObChunkDatumStore::ChunkIterator chunk_iter_;
|
||||
sql::ObSqlMemoryCallback* inner_callback_;
|
||||
sql::ObSqlMemoryCallback *inner_callback_;
|
||||
int32_t part_level_;
|
||||
int32_t batchno_;
|
||||
int64_t part_shift_;
|
||||
int64_t batchno_; // high: batch_round low: part_id
|
||||
bool is_chunk_iter_;
|
||||
ObHashJoinBufMgr* buf_mgr_;
|
||||
ObHashJoinBufMgr *buf_mgr_;
|
||||
uint64_t tenant_id_;
|
||||
int64_t timeout_ms_;
|
||||
int64_t n_get_rows_;
|
||||
int64_t n_add_rows_;
|
||||
|
||||
@ -310,24 +274,24 @@ private:
|
||||
int64_t pre_part_count_;
|
||||
};
|
||||
|
||||
struct ObHashJoinBatchPair {
|
||||
ObHashJoinBatch* left_;
|
||||
ObHashJoinBatch* right_;
|
||||
struct ObHashJoinBatchPair
|
||||
{
|
||||
ObHashJoinBatch *left_;
|
||||
ObHashJoinBatch *right_;
|
||||
|
||||
ObHashJoinBatchPair() : left_(NULL), right_(NULL)
|
||||
{}
|
||||
ObHashJoinBatchPair() : left_(NULL), right_(NULL) {}
|
||||
};
|
||||
|
||||
class ObHashJoinBatchMgr {
|
||||
public:
|
||||
ObHashJoinBatchMgr(common::ObIAllocator& alloc, ObHashJoinBufMgr* buf_mgr, uint64_t tenant_id)
|
||||
: total_dump_count_(0),
|
||||
total_dump_size_(0),
|
||||
batch_count_(0),
|
||||
tenant_id_(tenant_id),
|
||||
alloc_(alloc),
|
||||
batch_list_(alloc),
|
||||
buf_mgr_(buf_mgr)
|
||||
ObHashJoinBatchMgr(common::ObIAllocator &alloc, ObHashJoinBufMgr *buf_mgr, uint64_t tenant_id) :
|
||||
total_dump_count_(0),
|
||||
total_dump_size_(0),
|
||||
batch_count_(0),
|
||||
tenant_id_(tenant_id),
|
||||
alloc_(alloc),
|
||||
batch_list_(alloc),
|
||||
buf_mgr_(buf_mgr)
|
||||
{}
|
||||
|
||||
virtual ~ObHashJoinBatchMgr();
|
||||
@ -335,22 +299,24 @@ public:
|
||||
|
||||
typedef common::ObList<ObHashJoinBatchPair, common::ObIAllocator> hj_batch_pair_list_type;
|
||||
|
||||
int next_batch(ObHashJoinBatchPair& batch_pair);
|
||||
int next_batch(ObHashJoinBatchPair &batch_pair);
|
||||
|
||||
int64_t get_batch_list_size()
|
||||
{
|
||||
return batch_list_.size();
|
||||
}
|
||||
int remove_undumped_batch();
|
||||
int get_or_create_batch(int32_t level, int32_t batchno, bool is_left, ObHashJoinBatch*& batch, bool only_get = false);
|
||||
int64_t get_batch_list_size() { return batch_list_.size(); }
|
||||
int remove_undumped_batch(int64_t cur_dumped_partition,
|
||||
int32_t batch_round);
|
||||
int get_or_create_batch(int32_t level,
|
||||
int64_t part_shift,
|
||||
int64_t batchno,
|
||||
bool is_left,
|
||||
ObHashJoinBatch *&batch,
|
||||
bool only_get = false);
|
||||
|
||||
void free(ObHashJoinBatch* batch)
|
||||
{
|
||||
void free(ObHashJoinBatch *batch) {
|
||||
if (NULL != batch) {
|
||||
batch->~ObHashJoinBatch();
|
||||
alloc_.free(batch);
|
||||
batch = NULL;
|
||||
batch_count_--;
|
||||
batch_count_ --;
|
||||
}
|
||||
}
|
||||
|
||||
@ -360,115 +326,120 @@ public:
|
||||
int64_t batch_count_;
|
||||
|
||||
private:
|
||||
static const int64_t PARTITION_IDX_MASK = 0x00000000FFFFFFFF;
|
||||
uint64_t tenant_id_;
|
||||
common::ObIAllocator& alloc_;
|
||||
common::ObIAllocator &alloc_;
|
||||
hj_batch_pair_list_type batch_list_;
|
||||
ObHashJoinBufMgr* buf_mgr_;
|
||||
ObHashJoinBufMgr *buf_mgr_;
|
||||
};
|
||||
|
||||
class ObHashJoinPartition {
|
||||
class ObHashJoinPartition
|
||||
{
|
||||
public:
|
||||
ObHashJoinPartition() : buf_mgr_(nullptr), batch_mgr_(nullptr), batch_(nullptr), part_level_(-1), part_id_(-1)
|
||||
{}
|
||||
ObHashJoinPartition() :
|
||||
buf_mgr_(nullptr),
|
||||
batch_mgr_(nullptr),
|
||||
batch_(nullptr),
|
||||
part_level_(-1),
|
||||
part_id_(-1)
|
||||
{}
|
||||
|
||||
ObHashJoinPartition(ObHashJoinBufMgr* buf_mgr, ObHashJoinBatchMgr* batch_mgr)
|
||||
: buf_mgr_(buf_mgr), batch_mgr_(batch_mgr), batch_(nullptr), part_level_(-1), part_id_(-1)
|
||||
{}
|
||||
ObHashJoinPartition(ObHashJoinBufMgr *buf_mgr, ObHashJoinBatchMgr *batch_mgr) :
|
||||
buf_mgr_(buf_mgr),
|
||||
batch_mgr_(batch_mgr),
|
||||
batch_(nullptr),
|
||||
part_level_(-1),
|
||||
part_id_(-1)
|
||||
{}
|
||||
|
||||
virtual ~ObHashJoinPartition()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
void set_hj_buf_mgr(ObHashJoinBufMgr* buf_mgr)
|
||||
{
|
||||
void set_hj_buf_mgr(ObHashJoinBufMgr *buf_mgr) {
|
||||
buf_mgr_ = buf_mgr;
|
||||
}
|
||||
|
||||
void set_hj_batch_mgr(ObHashJoinBatchMgr* batch_mgr)
|
||||
{
|
||||
void set_hj_batch_mgr(ObHashJoinBatchMgr *batch_mgr) {
|
||||
batch_mgr_ = batch_mgr;
|
||||
}
|
||||
|
||||
int init(int32_t part_level, int32_t part_id, bool is_left, ObHashJoinBufMgr* buf_mgr, ObHashJoinBatchMgr* batch_mgr,
|
||||
ObHashJoinBatch* pre_batch, ObOperator* child_op, ObSqlMemoryCallback* callback, int64_t dir_id);
|
||||
int init(
|
||||
int32_t part_level,
|
||||
int64_t part_shift,
|
||||
int32_t part_id,
|
||||
int32_t batch_round,
|
||||
bool is_left,
|
||||
ObHashJoinBufMgr *buf_mgr,
|
||||
ObHashJoinBatchMgr *batch_mgr,
|
||||
ObHashJoinBatch* pre_batch,
|
||||
ObOperator *child_op,
|
||||
ObSqlMemoryCallback *callback,
|
||||
int64_t dir_id,
|
||||
ObIOEventObserver *io_event_observer);
|
||||
|
||||
int get_next_row(const ObHashJoinStoredJoinRow*& stored_row);
|
||||
int get_next_block_row(const ObHashJoinStoredJoinRow*& stored_row);
|
||||
int get_next_row(const ObHashJoinStoredJoinRow *&stored_row);
|
||||
inline int get_next_batch(const ObHashJoinStoredJoinRow **stored_row,
|
||||
const int64_t max_rows,
|
||||
int64_t &read_rows);
|
||||
int get_next_block_row(const ObHashJoinStoredJoinRow *&stored_row);
|
||||
|
||||
// payload of ObRowStore::StoredRow will be set after added.
|
||||
int add_row(const ObIArray<ObExpr*>& exprs, ObEvalCtx* eval_ctx, ObHashJoinStoredJoinRow*& stored_row);
|
||||
int add_row(const ObHashJoinStoredJoinRow* src_stored_row, ObHashJoinStoredJoinRow*& stored_row);
|
||||
int add_row(
|
||||
const ObIArray<ObExpr*> &exprs, ObEvalCtx *eval_ctx, ObHashJoinStoredJoinRow *&stored_row);
|
||||
int add_row(const ObHashJoinStoredJoinRow *src_stored_row, ObHashJoinStoredJoinRow *&stored_row);
|
||||
int add_batch(const common::ObIArray<ObExpr *> &exprs, ObEvalCtx &ctx,
|
||||
const ObBitVector &skip, const int64_t batch_size,
|
||||
const uint16_t selector[], const int64_t size,
|
||||
ObHashJoinStoredJoinRow **stored_rows = nullptr);
|
||||
int finish_dump(bool memory_need_dump);
|
||||
int dump(bool all_dump);
|
||||
int dump(bool all_dump, int64_t dumped_size);
|
||||
|
||||
int init_iterator(bool is_chunk_iter);
|
||||
int init_progressive_iterator();
|
||||
|
||||
void set_part_level(int32_t part_level)
|
||||
{
|
||||
part_level_ = part_level;
|
||||
}
|
||||
void set_part_id(int32_t part_id)
|
||||
{
|
||||
part_id_ = part_id;
|
||||
}
|
||||
void set_part_level(int32_t part_level) { part_level_ = part_level; }
|
||||
void set_part_id(int32_t part_id) { part_id_ = part_id; }
|
||||
|
||||
int32_t get_part_level()
|
||||
{
|
||||
return part_level_;
|
||||
}
|
||||
int32_t get_part_id()
|
||||
{
|
||||
return part_id_;
|
||||
}
|
||||
bool is_dumped()
|
||||
{
|
||||
return batch_->is_dumped();
|
||||
}
|
||||
int32_t get_part_level() { return part_level_; }
|
||||
int32_t get_part_id() { return part_id_; }
|
||||
bool is_dumped() { return batch_->is_dumped(); } //需要实现
|
||||
|
||||
int check();
|
||||
void reset();
|
||||
|
||||
ObHashJoinBatch* get_batch()
|
||||
{
|
||||
return batch_;
|
||||
}
|
||||
ObHashJoinBatch *get_batch() { return batch_; }
|
||||
|
||||
bool has_switch_block()
|
||||
{
|
||||
return batch_->has_switch_block();
|
||||
}
|
||||
bool has_switch_block() { return batch_->has_switch_block(); }
|
||||
|
||||
int64_t get_row_count_in_memory()
|
||||
{
|
||||
return batch_->get_row_count_in_memory();
|
||||
}
|
||||
int64_t get_row_count_on_disk()
|
||||
{
|
||||
return batch_->get_row_count_on_disk();
|
||||
}
|
||||
int64_t get_row_count_in_memory() { return batch_->get_row_count_in_memory(); }
|
||||
int64_t get_row_count_on_disk() { return batch_->get_row_count_on_disk(); }
|
||||
|
||||
int64_t get_size_in_memory()
|
||||
{
|
||||
return batch_->get_size_in_memory();
|
||||
}
|
||||
int64_t get_size_on_disk()
|
||||
{
|
||||
return batch_->get_size_on_disk();
|
||||
}
|
||||
int64_t get_size_in_memory() { return batch_->get_size_in_memory(); }
|
||||
int64_t get_size_on_disk() { return batch_->get_size_on_disk(); }
|
||||
|
||||
int64_t get_last_buffer_mem_size() { return batch_->get_last_buffer_mem_size(); }
|
||||
|
||||
int record_pre_batch_info(int64_t pre_part_count, int64_t pre_bucket_number, int64_t total_size);
|
||||
|
||||
private:
|
||||
ObHashJoinBufMgr* buf_mgr_;
|
||||
ObHashJoinBatchMgr* batch_mgr_;
|
||||
ObHashJoinBatch* batch_;
|
||||
ObHashJoinBufMgr *buf_mgr_;
|
||||
ObHashJoinBatchMgr *batch_mgr_;
|
||||
ObHashJoinBatch *batch_;
|
||||
int32_t part_level_;
|
||||
int32_t part_id_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
int ObHashJoinPartition::get_next_batch(const ObHashJoinStoredJoinRow **stored_row,
|
||||
const int64_t max_rows,
|
||||
int64_t &read_rows)
|
||||
{
|
||||
return batch_->get_next_batch(stored_row, max_rows, read_rows);
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* SRC_SQL_ENGINE_JOIN_OB_HASH_JOIN_BASIC_H_ */
|
||||
|
||||
|
||||
|
||||
Reference in New Issue
Block a user