init push
This commit is contained in:
2145
src/sql/engine/basic/ob_chunk_datum_store.cpp
Normal file
2145
src/sql/engine/basic/ob_chunk_datum_store.cpp
Normal file
File diff suppressed because it is too large
Load Diff
1007
src/sql/engine/basic/ob_chunk_datum_store.h
Normal file
1007
src/sql/engine/basic/ob_chunk_datum_store.h
Normal file
File diff suppressed because it is too large
Load Diff
1808
src/sql/engine/basic/ob_chunk_row_store.cpp
Normal file
1808
src/sql/engine/basic/ob_chunk_row_store.cpp
Normal file
File diff suppressed because it is too large
Load Diff
733
src/sql/engine/basic/ob_chunk_row_store.h
Normal file
733
src/sql/engine/basic/ob_chunk_row_store.h
Normal file
@ -0,0 +1,733 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_ROW_STORE2_H_
|
||||
#define OCEANBASE_BASIC_OB_ROW_STORE2_H_
|
||||
|
||||
#include "share/ob_define.h"
|
||||
#include "lib/container/ob_se_array.h"
|
||||
#include "lib/allocator/page_arena.h"
|
||||
#include "lib/utility/ob_print_utils.h"
|
||||
#include "lib/list/ob_dlist.h"
|
||||
#include "common/row/ob_row.h"
|
||||
#include "common/row/ob_row_iterator.h"
|
||||
#include "storage/blocksstable/ob_tmp_file.h"
|
||||
#include "sql/engine/basic/ob_sql_mem_callback.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
// Random access row store, support disk store.
|
||||
// All row must have same cell count and projector.
|
||||
class ObChunkRowStore {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
enum STORE_MODE { /* e.g. [0, 1, 2] with projector (0, 2) */
|
||||
WITHOUT_PROJECTOR = 0, /* store [0, 2] get [0, 2] */
|
||||
FULL /* store [0, 1, 2] get [0, 1, 2] with(0, 2) */
|
||||
};
|
||||
struct StoredRow {
|
||||
StoredRow() : cnt_(0), row_size_(0)
|
||||
{}
|
||||
|
||||
static inline int64_t row_copy_size(const common::ObNewRow& row)
|
||||
{
|
||||
int64_t size = sizeof(common::ObObj) * row.get_count();
|
||||
for (int64_t i = 0; i < row.get_count(); ++i) {
|
||||
size += row.get_cell(i).get_deep_copy_size();
|
||||
}
|
||||
return size;
|
||||
}
|
||||
inline int copy_row(const common::ObNewRow& row, char* buf, const int64_t size, const int64_t row_size,
|
||||
const uint32_t row_extend_size);
|
||||
inline common::ObObj* cells()
|
||||
{
|
||||
return reinterpret_cast<common::ObObj*>(payload_);
|
||||
}
|
||||
inline const common::ObObj* cells() const
|
||||
{
|
||||
return reinterpret_cast<const common::ObObj*>(payload_);
|
||||
}
|
||||
inline void* get_extra_payload() const
|
||||
{
|
||||
return static_cast<void*>(const_cast<char*>(payload_ + sizeof(ObObj) * cnt_));
|
||||
}
|
||||
int assign(const StoredRow* sr);
|
||||
inline void unswizzling(char* base = NULL);
|
||||
inline void swizzling();
|
||||
|
||||
TO_STRING_KV(K_(cnt), K_(row_size), K(cells()[0]));
|
||||
|
||||
uint32_t cnt_;
|
||||
uint32_t row_size_;
|
||||
char payload_[0];
|
||||
} __attribute__((packed));
|
||||
|
||||
typedef int (*copy_row_fun_t_)(const StoredRow* stored_row);
|
||||
|
||||
class BlockBuffer;
|
||||
struct Block {
|
||||
static const int64_t MAGIC = 0xbc054e02d8536315;
|
||||
static const int32_t ROW_HEAD_SIZE = sizeof(StoredRow);
|
||||
Block() : magic_(0), blk_size_(0), rows_(0)
|
||||
{}
|
||||
|
||||
static int64_t inline min_buf_size(const common::ObNewRow& row)
|
||||
{
|
||||
return BlockBuffer::HEAD_SIZE + sizeof(BlockBuffer) + row_store_size(row);
|
||||
}
|
||||
static int64_t inline min_buf_size(const int64_t row_store_size)
|
||||
{
|
||||
return BlockBuffer::HEAD_SIZE + sizeof(BlockBuffer) + row_store_size;
|
||||
}
|
||||
|
||||
static int64_t inline row_store_size(const common::ObNewRow& row, uint32_t row_extend_size = 0)
|
||||
{
|
||||
return ROW_HEAD_SIZE + row_extend_size + StoredRow::row_copy_size(row);
|
||||
}
|
||||
int add_row(
|
||||
const common::ObNewRow& row, const int64_t row_size, uint32_t row_extend_size, StoredRow** stored_row = NULL);
|
||||
int copy_row(const StoredRow* stored_row);
|
||||
int gen_unswizzling_payload(char* unswizzling_payload, uint32 size);
|
||||
int unswizzling();
|
||||
int swizzling(int64_t* col_cnt);
|
||||
inline bool magic_check()
|
||||
{
|
||||
return MAGIC == magic_;
|
||||
}
|
||||
int get_store_row(int64_t& cur_pos, const StoredRow*& sr);
|
||||
inline Block* get_next() const
|
||||
{
|
||||
return next_;
|
||||
}
|
||||
inline bool is_empty()
|
||||
{
|
||||
return get_buffer()->is_empty();
|
||||
}
|
||||
inline void set_block_size(uint32 blk_size)
|
||||
{
|
||||
blk_size_ = blk_size;
|
||||
}
|
||||
inline BlockBuffer* get_buffer()
|
||||
{
|
||||
return static_cast<BlockBuffer*>(static_cast<void*>(payload_ + blk_size_ - BlockBuffer::HEAD_SIZE));
|
||||
}
|
||||
inline int64_t data_size()
|
||||
{
|
||||
return get_buffer()->data_size();
|
||||
}
|
||||
inline uint32_t rows()
|
||||
{
|
||||
return rows_;
|
||||
}
|
||||
inline int64_t remain()
|
||||
{
|
||||
return get_buffer()->remain();
|
||||
}
|
||||
friend class BlockBuffer;
|
||||
TO_STRING_KV(K_(magic), K_(blk_size), K_(rows));
|
||||
union {
|
||||
int64_t magic_; // for dump
|
||||
Block* next_; // for block list in mem
|
||||
};
|
||||
uint32 blk_size_; /* current blk's size, for dump/read */
|
||||
uint32 rows_;
|
||||
char payload_[0];
|
||||
} __attribute__((packed));
|
||||
|
||||
struct BlockList {
|
||||
public:
|
||||
BlockList() : head_(NULL), last_(NULL), size_(0)
|
||||
{}
|
||||
inline int64_t get_size() const
|
||||
{
|
||||
return size_;
|
||||
}
|
||||
inline bool is_empty()
|
||||
{
|
||||
return size_ == 0;
|
||||
}
|
||||
inline Block* get_first() const
|
||||
{
|
||||
return head_;
|
||||
}
|
||||
inline void reset()
|
||||
{
|
||||
size_ = 0;
|
||||
head_ = NULL;
|
||||
last_ = NULL;
|
||||
}
|
||||
|
||||
inline void add_last(Block* blk)
|
||||
{
|
||||
if (NULL == head_) {
|
||||
head_ = blk;
|
||||
last_ = blk;
|
||||
blk->next_ = NULL;
|
||||
} else {
|
||||
last_->next_ = blk;
|
||||
blk->next_ = NULL;
|
||||
last_ = blk;
|
||||
}
|
||||
size_++;
|
||||
}
|
||||
|
||||
inline Block* remove_first()
|
||||
{
|
||||
Block* cur = head_;
|
||||
if (NULL != head_) {
|
||||
head_ = head_->next_;
|
||||
cur->next_ = NULL;
|
||||
size_--;
|
||||
if (0 == size_) {
|
||||
last_ = NULL;
|
||||
}
|
||||
}
|
||||
return cur;
|
||||
}
|
||||
TO_STRING_KV(K_(size), K_(head), K_(last), K_(*head), K_(last));
|
||||
|
||||
private:
|
||||
Block* head_;
|
||||
Block* last_;
|
||||
int64_t size_;
|
||||
};
|
||||
|
||||
/* contiguous memory:
|
||||
* |----------------|---Block
|
||||
* |next_ |-|-------------|
|
||||
* |cnt_ | |--HEAD_SIZE |
|
||||
* |block_size_ |-| |--block_size
|
||||
* |payload[] | |
|
||||
* | |---------------|
|
||||
* |----------------|--BlockBuffer
|
||||
* |data->BlockHead |-|
|
||||
* |cur_pos | |--TAIL_SIZE
|
||||
* |cap_=block_size |-|
|
||||
* |----------------|
|
||||
* */
|
||||
class BlockBuffer {
|
||||
public:
|
||||
static const int64_t HEAD_SIZE = sizeof(Block); /* n_rows, check_sum */
|
||||
BlockBuffer() : data_(NULL), cur_pos_(0), cap_(0)
|
||||
{}
|
||||
|
||||
int init(char* buf, const int64_t buf_size);
|
||||
inline int64_t remain() const
|
||||
{
|
||||
return cap_ - cur_pos_;
|
||||
}
|
||||
inline char* data()
|
||||
{
|
||||
return data_;
|
||||
}
|
||||
inline Block* get_block()
|
||||
{
|
||||
return block;
|
||||
}
|
||||
inline char* head() const
|
||||
{
|
||||
return data_ + cur_pos_;
|
||||
}
|
||||
inline int64_t capacity() const
|
||||
{
|
||||
return cap_;
|
||||
}
|
||||
inline int64_t mem_size() const
|
||||
{
|
||||
return cap_ + sizeof(BlockBuffer);
|
||||
}
|
||||
inline int64_t data_size() const
|
||||
{
|
||||
return cur_pos_;
|
||||
}
|
||||
inline bool is_inited() const
|
||||
{
|
||||
return NULL != data_;
|
||||
}
|
||||
inline bool is_empty() const
|
||||
{
|
||||
return HEAD_SIZE >= cur_pos_;
|
||||
}
|
||||
|
||||
inline void reset()
|
||||
{
|
||||
cur_pos_ = 0;
|
||||
cap_ = 0;
|
||||
data_ = NULL;
|
||||
}
|
||||
inline void reuse()
|
||||
{
|
||||
cur_pos_ = 0;
|
||||
advance(HEAD_SIZE);
|
||||
block->rows_ = 0;
|
||||
}
|
||||
inline int advance(int64_t size);
|
||||
|
||||
TO_STRING_KV(KP_(data), K_(cur_pos), K_(cap));
|
||||
|
||||
friend ObChunkRowStore;
|
||||
friend Block;
|
||||
|
||||
private:
|
||||
union {
|
||||
char* data_;
|
||||
Block* block;
|
||||
};
|
||||
int64_t cur_pos_;
|
||||
int64_t cap_;
|
||||
};
|
||||
|
||||
class ChunkIterator;
|
||||
class RowIterator {
|
||||
public:
|
||||
friend class ObChunkRowStore;
|
||||
RowIterator();
|
||||
virtual ~RowIterator()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
int init(ChunkIterator* chunk_it);
|
||||
/* from StoredRow to NewRow */
|
||||
int get_next_row(const StoredRow*& sr);
|
||||
int convert_to_row(common::ObNewRow& row, const StoredRow* sr);
|
||||
int convert_to_row(common::ObNewRow*& row, const StoredRow* sr);
|
||||
int convert_to_row_full(common::ObNewRow& row, const StoredRow* sr);
|
||||
int convert_to_row_full(common::ObNewRow*& row, const StoredRow* sr);
|
||||
|
||||
void reset()
|
||||
{
|
||||
reset_cursor();
|
||||
}
|
||||
bool is_valid() const
|
||||
{
|
||||
return store_ != NULL && cur_iter_blk_ != NULL;
|
||||
}
|
||||
inline bool cur_blk_has_next() const
|
||||
{
|
||||
return (cur_iter_blk_ != NULL && cur_row_in_blk_ < cur_iter_blk_->rows_);
|
||||
}
|
||||
inline bool has_next() const
|
||||
{
|
||||
return cur_iter_blk_ != NULL && (cur_iter_blk_->get_next() != NULL || cur_row_in_blk_ < cur_iter_blk_->rows_);
|
||||
}
|
||||
|
||||
TO_STRING_KV(KP_(store), K_(*store), K_(cur_iter_blk), K_(cur_row_in_blk), K_(cur_pos_in_blk), K_(n_blocks),
|
||||
K_(cur_nth_block));
|
||||
|
||||
private:
|
||||
explicit RowIterator(ObChunkRowStore* row_store);
|
||||
void reset_cursor()
|
||||
{
|
||||
cur_iter_blk_ = NULL;
|
||||
cur_row_in_blk_ = 0;
|
||||
cur_pos_in_blk_ = 0;
|
||||
n_blocks_ = 0;
|
||||
cur_nth_block_ = 0;
|
||||
}
|
||||
|
||||
protected:
|
||||
ObChunkRowStore* store_;
|
||||
Block* cur_iter_blk_;
|
||||
common::ObNewRow row_;
|
||||
int64_t cur_row_in_blk_; // cur nth row in cur block for in-mem debug
|
||||
int64_t cur_pos_in_blk_; // cur nth row in cur block
|
||||
int64_t n_blocks_;
|
||||
int64_t cur_nth_block_;
|
||||
};
|
||||
|
||||
class ChunkIterator {
|
||||
public:
|
||||
enum IterEndState { PROCESSING = 0x00, MEM_ITER_END = 0x01, DISK_ITER_END = 0x02 };
|
||||
|
||||
public:
|
||||
friend class ObChunkRowStore;
|
||||
ChunkIterator();
|
||||
virtual ~ChunkIterator();
|
||||
int init(ObChunkRowStore* row_store, int64_t chunk_read_size);
|
||||
int load_next_chunk(RowIterator& it);
|
||||
inline bool has_next_chunk()
|
||||
{
|
||||
return store_->n_blocks_ > 0 && (cur_nth_blk_ < store_->n_blocks_ - 1);
|
||||
}
|
||||
void set_chunk_read_size(int64_t chunk_read_size)
|
||||
{
|
||||
chunk_read_size_ = chunk_read_size;
|
||||
}
|
||||
inline int64_t get_chunk_read_size()
|
||||
{
|
||||
return chunk_read_size_;
|
||||
}
|
||||
inline int64_t get_row_cnt() const
|
||||
{
|
||||
return store_->get_row_cnt();
|
||||
}
|
||||
inline int64_t get_cur_chunk_row_cnt() const
|
||||
{
|
||||
return chunk_n_rows_;
|
||||
}
|
||||
inline int64_t get_chunk_read_size() const
|
||||
{
|
||||
return chunk_read_size_;
|
||||
}
|
||||
void reset();
|
||||
inline bool is_valid()
|
||||
{
|
||||
return store_ != NULL;
|
||||
}
|
||||
inline bool read_file_iter_end()
|
||||
{
|
||||
return iter_end_flag_ & DISK_ITER_END;
|
||||
}
|
||||
inline void set_read_file_iter_end()
|
||||
{
|
||||
iter_end_flag_ |= DISK_ITER_END;
|
||||
}
|
||||
inline bool read_mem_iter_end()
|
||||
{
|
||||
return iter_end_flag_ & MEM_ITER_END;
|
||||
}
|
||||
inline void set_read_mem_iter_end()
|
||||
{
|
||||
iter_end_flag_ |= MEM_ITER_END;
|
||||
}
|
||||
|
||||
TO_STRING_KV(KP_(store), KP_(cur_iter_blk), KP_(cur_iter_blk_buf), K_(cur_chunk_n_blocks), K_(cur_iter_pos),
|
||||
K_(file_size), K_(chunk_read_size), KP_(chunk_mem));
|
||||
|
||||
private:
|
||||
void reset_cursor(const int64_t file_size);
|
||||
|
||||
protected:
|
||||
ObChunkRowStore* store_;
|
||||
Block* cur_iter_blk_;
|
||||
BlockBuffer* cur_iter_blk_buf_; /*for reuse of cur_iter_blk_;
|
||||
cause Block::get_buffer() depends on blk_size_
|
||||
but blk_size_ will change with block reusing
|
||||
*/
|
||||
int64_t cur_nth_blk_; // reading nth blk start from 1
|
||||
int64_t cur_chunk_n_blocks_; // the number of blocks of current chunk
|
||||
int64_t cur_iter_pos_; // pos in file
|
||||
int64_t file_size_;
|
||||
int64_t chunk_read_size_;
|
||||
char* chunk_mem_;
|
||||
int64_t chunk_n_rows_;
|
||||
int32_t iter_end_flag_;
|
||||
};
|
||||
|
||||
#define CONVERT_FUN(C, W, T) \
|
||||
inline int C##W(RowIterator& it, T row, const StoredRow* sr) \
|
||||
{ \
|
||||
return it.C(row, sr); \
|
||||
}
|
||||
|
||||
class Iterator : public common::ObOuterRowIterator {
|
||||
public:
|
||||
friend class ObChunkRowStore;
|
||||
Iterator() : start_iter_(false), convert_row_with_obj_fun_(NULL), convert_row_fun_(NULL)
|
||||
{}
|
||||
virtual ~Iterator()
|
||||
{}
|
||||
int init(ObChunkRowStore* row_store, int64_t chunk_read_size);
|
||||
void set_chunk_read_size(int64_t chunk_read_size)
|
||||
{
|
||||
chunk_it_.set_chunk_read_size(chunk_read_size);
|
||||
}
|
||||
/// @param row [in/out] row.size_ is used to verify the data
|
||||
int get_next_row(common::ObNewRow& row);
|
||||
int get_next_row(common::ObNewRow*& row);
|
||||
int get_next_row(const StoredRow*& sr);
|
||||
void reset()
|
||||
{
|
||||
row_it_.reset();
|
||||
chunk_it_.reset();
|
||||
start_iter_ = false;
|
||||
}
|
||||
inline bool has_next()
|
||||
{
|
||||
return chunk_it_.has_next_chunk() || (row_it_.is_valid() && row_it_.has_next());
|
||||
}
|
||||
bool is_valid()
|
||||
{
|
||||
return chunk_it_.is_valid();
|
||||
}
|
||||
inline int64_t get_chunk_read_size()
|
||||
{
|
||||
return chunk_it_.get_chunk_read_size();
|
||||
}
|
||||
int convert_to_row_with_obj(const StoredRow* sr, common::ObNewRow& row)
|
||||
{
|
||||
return (this->*convert_row_with_obj_fun_)(row_it_, row, sr);
|
||||
}
|
||||
int convert_to_row(const StoredRow* sr, common::ObNewRow*& row)
|
||||
{
|
||||
return (this->*convert_row_fun_)(row_it_, row, sr);
|
||||
}
|
||||
|
||||
private:
|
||||
explicit Iterator(ObChunkRowStore* row_store);
|
||||
|
||||
CONVERT_FUN(convert_to_row, , common::ObNewRow*&)
|
||||
CONVERT_FUN(convert_to_row, _with_obj, common::ObNewRow&)
|
||||
CONVERT_FUN(convert_to_row_full, , common::ObNewRow*&)
|
||||
CONVERT_FUN(convert_to_row_full, _with_obj, common::ObNewRow&)
|
||||
protected:
|
||||
bool start_iter_;
|
||||
ChunkIterator chunk_it_;
|
||||
RowIterator row_it_;
|
||||
int (Iterator::*convert_row_with_obj_fun_)(RowIterator& it, common::ObNewRow& row, const StoredRow* sr);
|
||||
int (Iterator::*convert_row_fun_)(RowIterator& it, ObNewRow*& row, const StoredRow* sr);
|
||||
};
|
||||
|
||||
public:
|
||||
const static int64_t BLOCK_SIZE = (64L << 10); //+ BlockBuffer::HEAD_SIZE;
|
||||
|
||||
explicit ObChunkRowStore(common::ObIAllocator* alloc = NULL);
|
||||
|
||||
virtual ~ObChunkRowStore()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
int init(int64_t mem_limit, uint64_t tenant_id = common::OB_SERVER_TENANT_ID,
|
||||
int64_t mem_ctx_id = common::ObCtxIds::DEFAULT_CTX_ID,
|
||||
const char* label = common::ObModIds::OB_SQL_CHUNK_ROW_STORE, bool enable_dump = true,
|
||||
STORE_MODE row_store_mode = WITHOUT_PROJECTOR, uint32_t row_extra_size = 0);
|
||||
|
||||
void set_allocator(common::ObIAllocator& alloc)
|
||||
{
|
||||
allocator_ = &alloc;
|
||||
}
|
||||
|
||||
void reset();
|
||||
// Keeping one memory block, reader must call reuse() too.
|
||||
void reuse();
|
||||
|
||||
/// begin iterator
|
||||
int begin(ChunkIterator& it, int64_t chunk_read_size = 0)
|
||||
{
|
||||
return it.init(this, chunk_read_size);
|
||||
}
|
||||
|
||||
int begin(Iterator& it, int64_t chunk_read_size = 0)
|
||||
{
|
||||
return it.init(this, chunk_read_size);
|
||||
}
|
||||
|
||||
// template<typename T = ObChunkStoredRowType::StoredRow>
|
||||
int add_row(const common::ObNewRow& row, StoredRow** stored_row = NULL);
|
||||
int copy_row(const StoredRow* stored_row, ObChunkRowStore* crs = NULL);
|
||||
int finish_add_row(bool need_dump = true);
|
||||
OB_INLINE bool is_inited() const
|
||||
{
|
||||
return inited_;
|
||||
}
|
||||
bool is_file_open() const
|
||||
{
|
||||
return io_.fd_ >= 0;
|
||||
}
|
||||
|
||||
// void set_tenant_id(const uint64_t tenant_id) { tenant_id_ = tenant_id; }
|
||||
// void set_mem_ctx_id(const int64_t ctx_id) { ctx_id_ = ctx_id; }
|
||||
void set_mem_limit(const int64_t limit)
|
||||
{
|
||||
mem_limit_ = limit;
|
||||
}
|
||||
inline int64_t get_mem_limit()
|
||||
{
|
||||
return mem_limit_;
|
||||
}
|
||||
void set_block_size(const int64_t size)
|
||||
{
|
||||
default_block_size_ = size;
|
||||
}
|
||||
inline int64_t get_block_cnt() const
|
||||
{
|
||||
return n_blocks_;
|
||||
}
|
||||
inline int64_t get_block_list_cnt()
|
||||
{
|
||||
return blocks_.get_size();
|
||||
}
|
||||
inline int64_t get_row_cnt() const
|
||||
{
|
||||
return row_cnt_;
|
||||
}
|
||||
inline int64_t get_col_cnt() const
|
||||
{
|
||||
return col_count_;
|
||||
}
|
||||
inline int64_t get_mem_hold() const
|
||||
{
|
||||
return mem_hold_;
|
||||
}
|
||||
inline int64_t get_mem_used() const
|
||||
{
|
||||
return mem_used_;
|
||||
}
|
||||
inline int64_t get_file_size() const
|
||||
{
|
||||
return file_size_;
|
||||
}
|
||||
inline int64_t min_blk_size(const int64_t row_store_size)
|
||||
{
|
||||
int64_t size = std::max(std::max(static_cast<int64_t>(BLOCK_SIZE), default_block_size_), row_store_size);
|
||||
size = next_pow2(size);
|
||||
return size;
|
||||
}
|
||||
int init_block_buffer(void* mem, const int64_t size, Block*& block);
|
||||
int add_block(Block* block, bool need_swizzling, bool* added = nullptr);
|
||||
int append_block(char* buf, int size, bool need_swizzling);
|
||||
void remove_added_blocks();
|
||||
bool has_dumped()
|
||||
{
|
||||
return has_dumped_;
|
||||
}
|
||||
inline int64_t get_row_cnt_in_memory() const
|
||||
{
|
||||
return row_cnt_ - dumped_row_cnt_;
|
||||
}
|
||||
inline int64_t get_row_cnt_on_disk() const
|
||||
{
|
||||
return dumped_row_cnt_;
|
||||
}
|
||||
void set_callback(ObSqlMemoryCallback* callback)
|
||||
{
|
||||
callback_ = callback;
|
||||
}
|
||||
int dump(bool reuse, bool all_dump);
|
||||
void set_dir_id(int64_t dir_id)
|
||||
{
|
||||
io_.dir_id_ = dir_id;
|
||||
}
|
||||
int alloc_dir_id();
|
||||
TO_STRING_KV(K_(tenant_id), K_(label), K_(ctx_id), K_(mem_limit), K_(row_cnt), K_(file_size));
|
||||
|
||||
private:
|
||||
static int get_timeout(int64_t& timeout_ms);
|
||||
void* alloc_blk_mem(const int64_t size, const bool for_iterator);
|
||||
void free_blk_mem(void* mem, const int64_t size = 0);
|
||||
void free_block(Block* item);
|
||||
void free_blk_list();
|
||||
bool shrink_block(int64_t size);
|
||||
int alloc_block_buffer(Block*& block, const int64_t data_size, const bool for_iterator);
|
||||
int alloc_block_buffer(Block*& block, const int64_t data_size, const int64_t min_size, const bool for_iterator);
|
||||
// new block is not needed if %min_size is zero. (finish add row)
|
||||
int switch_block(const int64_t min_size);
|
||||
int clean_memory_data(bool reuse);
|
||||
|
||||
inline void use_block(Block* item)
|
||||
{
|
||||
cur_blk_ = item;
|
||||
int64_t used = item->get_buffer()->capacity() + sizeof(BlockBuffer);
|
||||
mem_used_ += used;
|
||||
}
|
||||
inline int dump_one_block(BlockBuffer* item);
|
||||
|
||||
int write_file(void* buf, int64_t size);
|
||||
int read_file(void* buf, const int64_t size, const int64_t offset);
|
||||
|
||||
bool need_dump(int64_t extra_size);
|
||||
BlockBuffer* new_block();
|
||||
void set_io(int64_t size, char* buf)
|
||||
{
|
||||
io_.size_ = size;
|
||||
io_.buf_ = buf;
|
||||
}
|
||||
bool find_block_can_hold(const int64_t size, bool& need_shrink);
|
||||
int get_store_row(RowIterator& it, const StoredRow*& sr);
|
||||
int load_next_block(ChunkIterator& it);
|
||||
int load_next_chunk_blocks(ChunkIterator& it);
|
||||
inline void callback_alloc(int64_t size)
|
||||
{
|
||||
if (callback_ != nullptr) {
|
||||
alloc_size_ += size;
|
||||
callback_->alloc(size);
|
||||
}
|
||||
}
|
||||
inline void callback_free(int64_t size)
|
||||
{
|
||||
if (callback_ != nullptr) {
|
||||
free_size_ += size;
|
||||
callback_->free(size);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
bool inited_;
|
||||
uint64_t tenant_id_;
|
||||
const char* label_;
|
||||
int64_t ctx_id_;
|
||||
int64_t mem_limit_;
|
||||
|
||||
Block* cur_blk_;
|
||||
BlockList blocks_; // ASSERT: all linked blocks has at least one row stored
|
||||
BlockList free_list_; // empty blocks
|
||||
int64_t max_blk_size_; // max block ever allocated
|
||||
int64_t min_blk_size_; // min block ever allocated
|
||||
int64_t default_block_size_; // default(min) block size; blocks larger then this will not be reused
|
||||
int64_t n_blocks_;
|
||||
int64_t col_count_;
|
||||
int64_t row_cnt_;
|
||||
|
||||
bool enable_dump_;
|
||||
bool has_dumped_;
|
||||
int64_t dumped_row_cnt_;
|
||||
|
||||
// int fd_;
|
||||
blocksstable::ObTmpFileIOInfo io_;
|
||||
int64_t file_size_;
|
||||
int64_t n_block_in_file_;
|
||||
|
||||
// BlockList blocks_; // ASSERT: all linked blocks has at least one row stored
|
||||
int64_t mem_hold_;
|
||||
int64_t mem_used_;
|
||||
common::DefaultPageAllocator inner_allocator_;
|
||||
common::ObIAllocator* allocator_;
|
||||
STORE_MODE row_store_mode_;
|
||||
int32_t* projector_;
|
||||
int64_t projector_size_;
|
||||
|
||||
uint32_t row_extend_size_;
|
||||
int64_t alloc_size_;
|
||||
int64_t free_size_;
|
||||
ObSqlMemoryCallback* callback_;
|
||||
|
||||
DISALLOW_COPY_AND_ASSIGN(ObChunkRowStore);
|
||||
};
|
||||
|
||||
inline int ObChunkRowStore::BlockBuffer::advance(int64_t size)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
if (size < -cur_pos_) {
|
||||
// overflow
|
||||
ret = common::OB_INVALID_ARGUMENT;
|
||||
SQL_ENG_LOG(WARN, "invalid argument", K(size), K_(cur_pos));
|
||||
} else if (size > remain()) {
|
||||
ret = common::OB_BUF_NOT_ENOUGH;
|
||||
SQL_ENG_LOG(WARN, "buffer not enough", K(size), "remain", remain());
|
||||
} else {
|
||||
cur_pos_ += size;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
class ObChunkStoreUtil {
|
||||
public:
|
||||
static int alloc_dir_id(int64_t& dir_id);
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_BASIC_OB_RA_ROW_STORE_H_
|
||||
304
src/sql/engine/basic/ob_count.cpp
Normal file
304
src/sql/engine/basic/ob_count.cpp
Normal file
@ -0,0 +1,304 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/basic/ob_count.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
ObCount::ObCount(ObIAllocator& alloc)
|
||||
: ObSingleChildPhyOperator(alloc), rownum_limit_expr_(NULL), anti_monotone_filter_exprs_()
|
||||
{}
|
||||
|
||||
ObCount::~ObCount()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
void ObCount::reset()
|
||||
{
|
||||
rownum_limit_expr_ = NULL;
|
||||
anti_monotone_filter_exprs_.reset();
|
||||
ObSingleChildPhyOperator::reset();
|
||||
}
|
||||
|
||||
void ObCount::reuse()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
int ObCount::rescan(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObCountCtx* count_ctx = NULL;
|
||||
if (OB_FAIL(ObSingleChildPhyOperator::rescan(ctx))) {
|
||||
LOG_WARN("rescan single child phy operator failed", K(ret));
|
||||
} else if (OB_ISNULL(count_ctx = GET_PHY_OPERATOR_CTX(ObCountCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("count ctx is null", K(ret));
|
||||
} else {
|
||||
count_ctx->cur_rownum_ = 1;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCount::add_anti_monotone_filter_exprs(ObSqlExpression* expr)
|
||||
{
|
||||
return ObSqlExpressionUtil::add_expr_to_list(anti_monotone_filter_exprs_, expr);
|
||||
}
|
||||
|
||||
int ObCount::get_rownum_limit_value(ObExecContext& ctx, int64_t& rownum_limit_value) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(get_int_value(ctx, rownum_limit_expr_, rownum_limit_value))) {
|
||||
LOG_WARN("failed to get rownum value from rownum expr", K(ret));
|
||||
} else { /*do nothing*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCount::get_int_value(ObExecContext& ctx, const ObSqlExpression* in_val, int64_t& out_val) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObNewRow input_row;
|
||||
ObObj result;
|
||||
ObExprCtx expr_ctx;
|
||||
if (in_val != NULL && !in_val->is_empty()) {
|
||||
if (OB_FAIL(wrap_expr_ctx(ctx, expr_ctx))) {
|
||||
LOG_WARN("wrap expr context failed", K(ret));
|
||||
} else if (OB_FAIL(in_val->calc(expr_ctx, input_row, result))) {
|
||||
LOG_WARN("Failed to calculate expression", K(ret));
|
||||
} else if (OB_LIKELY(result.is_int())) {
|
||||
if (OB_FAIL(result.get_int(out_val))) {
|
||||
LOG_WARN("get_int error", K(ret), K(result));
|
||||
}
|
||||
} else if (result.is_null()) {
|
||||
out_val = 0;
|
||||
} else {
|
||||
EXPR_DEFINE_CAST_CTX(expr_ctx, CM_NONE);
|
||||
EXPR_GET_INT64_V2(result, out_val);
|
||||
if (OB_FAIL(ret)) {
|
||||
LOG_WARN("get_int error", K(ret), K(result));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObCount::is_valid() const
|
||||
{
|
||||
return (get_column_count() > 0 && child_op_ != NULL && child_op_->get_column_count() > 0);
|
||||
}
|
||||
|
||||
int ObCount::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhyOperatorCtx* op_ctx = NULL;
|
||||
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObCountCtx, ctx, get_id(), get_type(), op_ctx))) {
|
||||
LOG_WARN("failed to create LimitCtx", K(ret));
|
||||
} else if (OB_ISNULL(op_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("op_ctx is null");
|
||||
} else if (OB_FAIL(init_cur_row(*op_ctx, true))) {
|
||||
LOG_WARN("init current row failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCount::get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObCountCtx* count_ctx = NULL;
|
||||
ObSQLSessionInfo* my_session = NULL;
|
||||
if (OB_UNLIKELY(!is_valid())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("count operator is invalid", K(ret));
|
||||
} else if (OB_ISNULL(count_ctx = GET_PHY_OPERATOR_CTX(ObCountCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator ctx failed");
|
||||
} else if (OB_ISNULL(my_session = GET_MY_SESSION(ctx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get my session", K(ret));
|
||||
} else if (count_ctx->has_rownum_limit_ &&
|
||||
(count_ctx->rownum_limit_value_ <= 0 || count_ctx->cur_rownum_ - 1 == count_ctx->rownum_limit_value_)) {
|
||||
ret = OB_ITER_END;
|
||||
} else if (OB_FAIL(ObSingleChildPhyOperator::get_next_row(ctx, row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("fail to get next row from child phy operator", K(ret));
|
||||
}
|
||||
} else {
|
||||
++count_ctx->cur_rownum_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCount::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObCountCtx* count_ctx = NULL;
|
||||
if (OB_UNLIKELY(!is_valid())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("count operator is invalid", K(ret));
|
||||
} else if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("initialize operator context failed", K(ret));
|
||||
} else if (OB_ISNULL(count_ctx = GET_PHY_OPERATOR_CTX(ObCountCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K_(id));
|
||||
} else {
|
||||
if (NULL != rownum_limit_expr_ && !rownum_limit_expr_->is_empty()) {
|
||||
count_ctx->has_rownum_limit_ = true;
|
||||
if (OB_FAIL(get_rownum_limit_value(ctx, count_ctx->rownum_limit_value_))) {
|
||||
LOG_WARN("fail to instantiate rownum limit", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCount::switch_iterator(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObCountCtx* count_ctx = NULL;
|
||||
if (OB_FAIL(ObSingleChildPhyOperator::switch_iterator(ctx))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("switch parent iterator failed", K(ret));
|
||||
}
|
||||
} else if (OB_ISNULL(count_ctx = GET_PHY_OPERATOR_CTX(ObCountCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K_(id));
|
||||
} else {
|
||||
if (NULL != rownum_limit_expr_ && !rownum_limit_expr_->is_empty()) {
|
||||
count_ctx->has_rownum_limit_ = true;
|
||||
if (OB_FAIL(get_rownum_limit_value(ctx, count_ctx->rownum_limit_value_))) {
|
||||
LOG_WARN("fail to instantiate rownum limit", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCount::inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObCountCtx* count_ctx = NULL;
|
||||
ObSQLSessionInfo* my_session = NULL;
|
||||
const ObNewRow* input_row = NULL;
|
||||
bool is_filtered = false;
|
||||
if (OB_UNLIKELY(!is_valid())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("count operator is invalid", K(ret));
|
||||
} else if (OB_ISNULL(count_ctx = GET_PHY_OPERATOR_CTX(ObCountCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator ctx failed");
|
||||
} else if (OB_ISNULL(my_session = GET_MY_SESSION(ctx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get my session", K(ret));
|
||||
} else if (OB_FAIL(child_op_->get_next_row(ctx, input_row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("child_op failed to get next row", K(ret));
|
||||
}
|
||||
} else if (OB_ISNULL(input_row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(input_row), K(ret));
|
||||
} else if (OB_FAIL(ObPhyOperator::filter_row(
|
||||
count_ctx->expr_ctx_, *input_row, anti_monotone_filter_exprs_, is_filtered))) {
|
||||
LOG_WARN("failed to evaluate anti-monotone filter exprs", K(ret));
|
||||
} else if (is_filtered) {
|
||||
row = NULL;
|
||||
ret = OB_ITER_END;
|
||||
LOG_DEBUG("row is filtered by anti-monotone filters, running is stopped");
|
||||
} else {
|
||||
row = input_row;
|
||||
if (OB_FAIL(copy_cur_row_by_projector(*count_ctx, row))) {
|
||||
LOG_WARN("copy current row failed", K(ret));
|
||||
} else {
|
||||
LOG_DEBUG("succeed to copy cur row by projector", K(*row));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCount::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
UNUSED(ctx);
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
int64_t ObCount::to_string_kv(char* buf, const int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
if (rownum_limit_expr_) {
|
||||
J_KV(N_LIMIT, rownum_limit_expr_);
|
||||
}
|
||||
return pos;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE(ObCount)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool has_rownum_limit = (rownum_limit_expr_ != NULL && !rownum_limit_expr_->is_empty());
|
||||
OB_UNIS_ENCODE(has_rownum_limit);
|
||||
if (OB_SUCC(ret) && has_rownum_limit) {
|
||||
OB_UNIS_ENCODE(*rownum_limit_expr_);
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(ObSingleChildPhyOperator::serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("serialize child operator failed", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(serialize_dlist(anti_monotone_filter_exprs_, buf, buf_len, pos))) {
|
||||
LOG_WARN("failed to serialize filter_exprs_", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE_SIZE(ObCount)
|
||||
{
|
||||
bool has_rownum_limit = (rownum_limit_expr_ != NULL && !rownum_limit_expr_->is_empty());
|
||||
int64_t len = 0;
|
||||
OB_UNIS_ADD_LEN(has_rownum_limit);
|
||||
if (has_rownum_limit) {
|
||||
OB_UNIS_ADD_LEN(*rownum_limit_expr_);
|
||||
}
|
||||
len += ObSingleChildPhyOperator::get_serialize_size();
|
||||
len += get_dlist_serialize_size(anti_monotone_filter_exprs_);
|
||||
return len;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObCount)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool has_rownum_limit = false;
|
||||
OB_UNIS_DECODE(has_rownum_limit);
|
||||
if (OB_SUCC(ret) && has_rownum_limit) {
|
||||
if (OB_FAIL(ObSqlExpressionUtil::make_sql_expr(my_phy_plan_, rownum_limit_expr_))) {
|
||||
LOG_WARN("make sql expression failed", K(ret));
|
||||
} else if (OB_LIKELY(rownum_limit_expr_ != NULL)) {
|
||||
OB_UNIS_DECODE(*rownum_limit_expr_);
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
ret = ObSingleChildPhyOperator::deserialize(buf, data_len, pos);
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
OB_UNIS_DECODE_EXPR_DLIST(ObSqlExpression, anti_monotone_filter_exprs_, my_phy_plan_);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} /* namespace sql */
|
||||
} /* namespace oceanbase */
|
||||
99
src/sql/engine/basic/ob_count.h
Normal file
99
src/sql/engine/basic/ob_count.h
Normal file
@ -0,0 +1,99 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef SRC_SQL_ENGINE_BASIC_OB_COUNT_H_
|
||||
#define SRC_SQL_ENGINE_BASIC_OB_COUNT_H_
|
||||
#include "sql/engine/ob_single_child_phy_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObSqlExpression;
|
||||
|
||||
class ObCount : public ObSingleChildPhyOperator {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
class ObCountCtx : ObPhyOperatorCtx {
|
||||
friend class ObCount;
|
||||
|
||||
public:
|
||||
explicit ObCountCtx(ObExecContext& ctx)
|
||||
: ObPhyOperatorCtx(ctx), cur_rownum_(1), has_rownum_limit_(false), rownum_limit_value_(-1)
|
||||
{}
|
||||
virtual void destroy()
|
||||
{
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
|
||||
public:
|
||||
int64_t cur_rownum_;
|
||||
bool has_rownum_limit_;
|
||||
int64_t rownum_limit_value_;
|
||||
};
|
||||
|
||||
public:
|
||||
explicit ObCount(common::ObIAllocator& alloc);
|
||||
virtual ~ObCount();
|
||||
virtual int get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
virtual void reset();
|
||||
virtual void reuse();
|
||||
virtual int rescan(ObExecContext& ctx) const;
|
||||
inline void set_rownum_limit_expr(ObSqlExpression* rownum_limit_expr)
|
||||
{
|
||||
rownum_limit_expr_ = rownum_limit_expr;
|
||||
}
|
||||
int add_anti_monotone_filter_exprs(ObSqlExpression* expr);
|
||||
virtual int switch_iterator(ObExecContext& ctx) const;
|
||||
|
||||
private:
|
||||
bool is_valid() const;
|
||||
int get_rownum_limit_value(ObExecContext& ctx, int64_t& rownum_limit_value) const;
|
||||
int get_int_value(ObExecContext& ctx, const ObSqlExpression* in_val, int64_t& out_val) const;
|
||||
/**
|
||||
* @brief init operator context, will create a physical operator context (and a current row space)
|
||||
* @param ctx[in], execute context
|
||||
* @return if success, return OB_SUCCESS, otherwise, return errno
|
||||
*/
|
||||
int init_op_ctx(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief: called by get_next_row(), get a row from the child operator or row_store
|
||||
* @param: ctx[in], execute context
|
||||
* @param: row[out], ObSqlRow an obj array and row_size
|
||||
*/
|
||||
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
/**
|
||||
* @brief open operator, not including children operators.
|
||||
* called by open.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_open(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief close operator, not including children operators.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_close(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief for specified phy operator to print it's member variable with json key-value format
|
||||
* @param buf[in] to string buffer
|
||||
* @param buf_len[in] buffer length
|
||||
* @return if success, return the length used by print string, otherwise return 0
|
||||
*/
|
||||
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
|
||||
|
||||
private:
|
||||
ObSqlExpression* rownum_limit_expr_;
|
||||
common::ObDList<ObSqlExpression> anti_monotone_filter_exprs_;
|
||||
};
|
||||
} /* namespace sql */
|
||||
} /* namespace oceanbase */
|
||||
|
||||
#endif /* SRC_SQL_ENGINE_BASIC_OB_COUNT_H_ */
|
||||
143
src/sql/engine/basic/ob_count_op.cpp
Normal file
143
src/sql/engine/basic/ob_count_op.cpp
Normal file
@ -0,0 +1,143 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "ob_count_op.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
ObCountSpec::ObCountSpec(ObIAllocator& alloc, const ObPhyOperatorType type)
|
||||
: ObOpSpec(alloc, type), rownum_limit_(NULL), anti_monotone_filters_(alloc)
|
||||
|
||||
{}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObCountSpec, ObOpSpec), rownum_limit_, anti_monotone_filters_);
|
||||
|
||||
ObCountOp::ObCountOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObOperator(exec_ctx, spec, input)
|
||||
{
|
||||
reset_default();
|
||||
}
|
||||
|
||||
void ObCountOp::reset_default()
|
||||
{
|
||||
cur_rownum_ = 0;
|
||||
rownum_limit_ = INT64_MAX;
|
||||
}
|
||||
|
||||
int ObCountOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(child_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("count operator has no child", K(ret));
|
||||
} else if (OB_FAIL(get_rownum_limit())) {
|
||||
LOG_WARN("get rownum limit failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCountOp::get_rownum_limit()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObExpr* expr = MY_SPEC.rownum_limit_;
|
||||
if (NULL != expr) {
|
||||
ObDatum* datum = NULL;
|
||||
if (OB_FAIL(expr->eval(eval_ctx_, datum))) {
|
||||
LOG_WARN("limit expr evaluate failed", K(ret));
|
||||
} else {
|
||||
if (datum->null_) {
|
||||
rownum_limit_ = 0;
|
||||
} else {
|
||||
OB_ASSERT(ob_is_int_tc(expr->datum_meta_.type_));
|
||||
rownum_limit_ = *datum->int_;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCountOp::rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
if (OB_FAIL(ObOperator::rescan())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("operator rescan fail", K(ret));
|
||||
}
|
||||
} else {
|
||||
reset_default();
|
||||
OZ(get_rownum_limit());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCountOp::switch_iterator()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObOperator::switch_iterator())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("operator switch iterator fail", K(ret));
|
||||
}
|
||||
} else {
|
||||
reset_default();
|
||||
OZ(get_rownum_limit());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCountOp::get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (cur_rownum_ >= rownum_limit_) {
|
||||
ret = OB_ITER_END;
|
||||
} else {
|
||||
cur_rownum_ += 1;
|
||||
if (OB_FAIL(ObOperator::get_next_row())) {
|
||||
cur_rownum_ -= 1;
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("operator get next row failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObCountOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(child_->get_next_row())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get row from child failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
clear_evaluated_flag();
|
||||
if (!MY_SPEC.anti_monotone_filters_.empty()) {
|
||||
bool filtered = false;
|
||||
if (OB_FAIL(filter(MY_SPEC.anti_monotone_filters_, filtered))) {
|
||||
LOG_WARN("filter row failed", K(ret));
|
||||
} else if (filtered) {
|
||||
// Set rownum limit to zero make it return OB_ITER_END directly on next call.
|
||||
rownum_limit_ = 0;
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
68
src/sql/engine/basic/ob_count_op.h
Normal file
68
src/sql/engine/basic/ob_count_op.h
Normal file
@ -0,0 +1,68 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_COUNT_OP_H_
|
||||
#define OCEANBASE_BASIC_OB_COUNT_OP_H_
|
||||
|
||||
#include "sql/engine/ob_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObCountSpec : public ObOpSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObCountSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
INHERIT_TO_STRING_KV("op_spec", ObOpSpec, K_(rownum_limit), K_(anti_monotone_filters));
|
||||
|
||||
ObExpr* rownum_limit_;
|
||||
ExprFixedArray anti_monotone_filters_;
|
||||
};
|
||||
|
||||
class ObCountOp : public ObOperator {
|
||||
public:
|
||||
ObCountOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int rescan() override;
|
||||
virtual int switch_iterator() override;
|
||||
|
||||
// Increment %cur_rownum_ in get_next_row()
|
||||
virtual int get_next_row() override;
|
||||
|
||||
virtual int inner_get_next_row() override;
|
||||
|
||||
virtual void destroy() override
|
||||
{
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
int64_t get_cur_rownum() const
|
||||
{
|
||||
return cur_rownum_;
|
||||
}
|
||||
|
||||
private:
|
||||
// reset default value of %cur_rownum_ && %rownum_limit_
|
||||
void reset_default();
|
||||
int get_rownum_limit();
|
||||
|
||||
private:
|
||||
int64_t cur_rownum_;
|
||||
int64_t rownum_limit_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_BASIC_OB_COUNT_OP_H_
|
||||
650
src/sql/engine/basic/ob_expr_values.cpp
Normal file
650
src/sql/engine/basic/ob_expr_values.cpp
Normal file
@ -0,0 +1,650 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/basic/ob_expr_values.h"
|
||||
#include "share/object/ob_obj_cast.h"
|
||||
#include "common/sql_mode/ob_sql_mode_utils.h"
|
||||
#include "sql/ob_sql_utils.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/engine/expr/ob_iter_expr_range_param.h"
|
||||
#include "sql/engine/px/ob_dfo.h"
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
using namespace share;
|
||||
namespace sql {
|
||||
OB_SERIALIZE_MEMBER(ObExprValuesInput, partition_id_values_);
|
||||
|
||||
class ObExprValues::ObExprValuesCtx : public ObPhyOperatorCtx {
|
||||
public:
|
||||
explicit ObExprValuesCtx(ObExecContext& ctx)
|
||||
: ObPhyOperatorCtx(ctx),
|
||||
node_idx_(0),
|
||||
iter_expr_ctx_(ctx, ctx.get_allocator()),
|
||||
value_count_(0),
|
||||
switch_value_(false)
|
||||
{}
|
||||
~ObExprValuesCtx()
|
||||
{}
|
||||
int get_value_count(ObExecContext& ctx, bool is_range_param, int64_t partition_id_values, int64_t origin_value_count,
|
||||
int64_t col_num);
|
||||
virtual void destroy()
|
||||
{
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
|
||||
private:
|
||||
int64_t node_idx_;
|
||||
ObIterExprCtx iter_expr_ctx_;
|
||||
int64_t value_count_;
|
||||
bool switch_value_;
|
||||
friend class ObExprValues;
|
||||
};
|
||||
|
||||
int ObExprValues::ObExprValuesCtx::get_value_count(
|
||||
ObExecContext& ctx, bool is_range_param, int64_t partition_id_values, int64_t origin_value_count, int64_t col_num)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (partition_id_values != 0) {
|
||||
common::ObIArray<ObPxSqcMeta::PartitionIdValue>* pid_values =
|
||||
reinterpret_cast<common::ObIArray<ObPxSqcMeta::PartitionIdValue>*>(partition_id_values);
|
||||
int64_t partition_id = ctx.get_expr_partition_id();
|
||||
bool find = false;
|
||||
CK(partition_id != OB_INVALID_ID);
|
||||
if (is_range_param) {
|
||||
col_num = 1;
|
||||
}
|
||||
for (int i = 0; OB_SUCC(ret) && i < pid_values->count() && !find; ++i) {
|
||||
if (partition_id == pid_values->at(i).partition_id_) {
|
||||
node_idx_ = pid_values->at(i).value_begin_idx_ * col_num;
|
||||
value_count_ = node_idx_ + pid_values->at(i).value_count_ * col_num;
|
||||
if (OB_FAIL(value_count_ > origin_value_count)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected value count", K(ret), K(value_count_), K(origin_value_count));
|
||||
}
|
||||
find = true;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (!find) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected partition id", K(partition_id));
|
||||
} else {
|
||||
switch_value_ = false;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
value_count_ = origin_value_count;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
ObExprValues::ObExprValues(ObIAllocator& alloc) : ObNoChildrenPhyOperator(alloc), values_(alloc), range_params_(alloc)
|
||||
{}
|
||||
|
||||
ObExprValues::~ObExprValues()
|
||||
{}
|
||||
|
||||
int ObExprValues::init_value_count(int64_t value_cnt)
|
||||
{
|
||||
return values_.init(value_cnt);
|
||||
}
|
||||
|
||||
int ObExprValues::add_value(ObSqlExpression* value)
|
||||
{
|
||||
return values_.push_back(value);
|
||||
}
|
||||
|
||||
int ObExprValues::init_range_params(int64_t range_param_count)
|
||||
{
|
||||
return range_params_.init(range_param_count);
|
||||
}
|
||||
|
||||
int ObExprValues::add_range_param(ObIterExprRangeParam* range_param_expr)
|
||||
{
|
||||
return range_params_.push_back(range_param_expr);
|
||||
}
|
||||
|
||||
int ObExprValues::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObExprValuesCtx* op_ctx = NULL;
|
||||
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObExprValuesCtx, ctx, get_id(), get_type(), op_ctx))) {
|
||||
LOG_WARN("create physical operator context failed", K(ret));
|
||||
} else if (OB_ISNULL(op_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("op_ctx is null");
|
||||
} else if (OB_FAIL(op_ctx->create_cur_row(get_column_count(), projector_, projector_size_))) {
|
||||
LOG_WARN("create current row failed", K(ret));
|
||||
} else {
|
||||
op_ctx->iter_expr_ctx_.set_cur_row(&op_ctx->get_cur_row());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValues::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObExprValuesCtx* values_ctx = NULL;
|
||||
|
||||
if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("init operator context failed", K(ret));
|
||||
} else if (OB_ISNULL(values_ctx = GET_PHY_OPERATOR_CTX(ObExprValuesCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K(ret), K_(id));
|
||||
} else {
|
||||
values_ctx->node_idx_ = 0;
|
||||
values_ctx->value_count_ = (values_.empty() ? range_params_.count() : values_.count());
|
||||
values_ctx->switch_value_ = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValues::rescan(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObExprValuesCtx* values_ctx = NULL;
|
||||
ObExprValuesInput* op_input = GET_PHY_OP_INPUT(ObExprValuesInput, ctx, get_id());
|
||||
if (OB_ISNULL(values_ctx = GET_PHY_OPERATOR_CTX(ObExprValuesCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("values_ctx is null");
|
||||
} else if (OB_NOT_NULL(op_input) && 0 != op_input->partition_id_values_ && ctx.is_gi_restart()) {
|
||||
values_ctx->switch_value_ = true;
|
||||
} else {
|
||||
values_ctx->node_idx_ = 0;
|
||||
}
|
||||
OZ(ObNoChildrenPhyOperator::rescan(ctx));
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValues::switch_iterator(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObExprValuesCtx* values_ctx = NULL;
|
||||
ObPhysicalPlanCtx* plan_ctx = NULL;
|
||||
if (OB_ISNULL(values_ctx = GET_PHY_OPERATOR_CTX(ObExprValuesCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("values_ctx is null");
|
||||
} else if (OB_ISNULL(plan_ctx = GET_PHY_PLAN_CTX(ctx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("plan ctx is null", K(ret));
|
||||
} else {
|
||||
values_ctx->node_idx_ = 0;
|
||||
if (values_ctx->expr_ctx_.cur_array_index_ >= plan_ctx->get_bind_array_count() - 1) {
|
||||
ret = OB_ITER_END;
|
||||
} else {
|
||||
++values_ctx->expr_ctx_.cur_array_index_;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValues::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObExprValuesCtx* values_ctx = NULL;
|
||||
if (OB_ISNULL(values_ctx = GET_PHY_OPERATOR_CTX(ObExprValuesCtx, ctx, get_id()))) {
|
||||
LOG_DEBUG("The operator has not been opened.", K(ret), K_(id), "op_type", ob_phy_operator_type_str(get_type()));
|
||||
} else {
|
||||
values_ctx->node_idx_ = 0;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValues::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
NG_TRACE_TIMES(2, value_start_next_row);
|
||||
ObPhysicalPlanCtx* plan_ctx = GET_PHY_PLAN_CTX(ctx);
|
||||
ObExprValuesCtx* values_ctx = GET_PHY_OPERATOR_CTX(ObExprValuesCtx, ctx, id_);
|
||||
ObExprValuesInput* op_input = GET_PHY_OP_INPUT(ObExprValuesInput, ctx, get_id());
|
||||
// reset temp auto-increment value before calculate a row
|
||||
#if !defined(NDEBUG)
|
||||
if (OB_ISNULL(plan_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("plan_ctx is null");
|
||||
} else if (OB_ISNULL(values_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("values_cts is null");
|
||||
}
|
||||
#endif
|
||||
if (OB_SUCC(ret)) {
|
||||
plan_ctx->set_autoinc_id_tmp(0);
|
||||
if (get_rows() > 10) {
|
||||
if (OB_FAIL(THIS_WORKER.check_status())) {
|
||||
LOG_WARN("check physical plan status faild", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && OB_NOT_NULL(op_input)) {
|
||||
if (values_ctx->switch_value_ && OB_FAIL(values_ctx->get_value_count(ctx,
|
||||
!range_params_.empty(),
|
||||
op_input->partition_id_values_,
|
||||
range_params_.empty() ? values_.count() : range_params_.count(),
|
||||
get_column_count()))) {
|
||||
LOG_WARN("fail to get value count", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (!range_params_.empty()) {
|
||||
if (OB_FAIL(calc_next_row_by_range_param(*values_ctx, row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("calc next row by range param failed", K(ret));
|
||||
}
|
||||
}
|
||||
} else if (OB_FAIL(calc_next_row(ctx))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get next row from row store failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
row = &(values_ctx->get_cur_row());
|
||||
NG_TRACE_TIMES(2, value_end_next_row);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_INLINE int ObExprValues::calc_next_row_by_range_param(ObExprValuesCtx& values_ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObIterExprRangeParam* iter_range_param = NULL;
|
||||
if (OB_UNLIKELY(values_ctx.node_idx_ >= values_ctx.value_count_)) {
|
||||
ret = OB_ITER_END;
|
||||
LOG_DEBUG("get next row from values iterator end", K(ret), K(values_ctx.node_idx_), K(range_params_.count()));
|
||||
} else if (OB_ISNULL(iter_range_param = range_params_.at(values_ctx.node_idx_++))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("iter range param is null", K(ret), K(values_ctx.node_idx_), K(range_params_));
|
||||
} else if (OB_FAIL(iter_range_param->get_next_row(values_ctx.iter_expr_ctx_, row))) {
|
||||
LOG_WARN("get next row from range params failed", K(ret), KPC(iter_range_param));
|
||||
} else {
|
||||
LOG_DEBUG("calc next row by range param", KPC(row), KPC(iter_range_param));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_INLINE int ObExprValues::calc_next_row(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObExprValuesCtx* value_ctx = GET_PHY_OPERATOR_CTX(ObExprValuesCtx, ctx, id_);
|
||||
ObSQLSessionInfo* session = ctx.get_my_session();
|
||||
int64_t col_num = get_column_count();
|
||||
int64_t col_index = 0;
|
||||
NG_TRACE_TIMES(2, value_start_calc_row);
|
||||
#if !defined(NDEBUG)
|
||||
if (OB_UNLIKELY(get_size() <= 0 || get_column_count() <= 0 || get_size() % get_column_count() != 0 ||
|
||||
OB_ISNULL(my_phy_plan_))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("expr values status is unexpected", K(values_), K(get_column_count()), K_(my_phy_plan));
|
||||
} else if (OB_ISNULL(value_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("value_ctx is null");
|
||||
}
|
||||
#endif //! NDEBUG
|
||||
if (OB_SUCC(ret)) {
|
||||
ObExprCtx& expr_ctx = value_ctx->expr_ctx_;
|
||||
ObNewRow& cur_row = value_ctx->get_cur_row();
|
||||
if (OB_UNLIKELY(value_ctx->node_idx_ == value_ctx->value_count_)) {
|
||||
// there is no values any more
|
||||
ret = OB_ITER_END;
|
||||
} else if (OB_UNLIKELY(cur_row.is_invalid())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("cur row is not init", K(cur_row));
|
||||
} else {
|
||||
bool is_break = false;
|
||||
while (OB_SUCC(ret) && value_ctx->node_idx_ < value_ctx->value_count_ && !is_break) {
|
||||
if (OB_ISNULL(values_.at(value_ctx->node_idx_))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("value exprs is null", K(values_.at(value_ctx->node_idx_)));
|
||||
} else if (OB_FAIL(values_.at(value_ctx->node_idx_)->calc(expr_ctx, cur_row, cur_row.cells_[col_index]))) {
|
||||
LOG_WARN("failed to calc expr", K(ret));
|
||||
} else {
|
||||
++value_ctx->node_idx_;
|
||||
if (col_index == col_num - 1) {
|
||||
// last cell values resolved, output row now
|
||||
is_break = true;
|
||||
} else {
|
||||
col_index = (col_index + 1) % col_num;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
NG_TRACE_TIMES(2, value_after_calc_row);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValues::create_operator_input(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObIPhyOperatorInput* input = NULL;
|
||||
if (OB_FAIL(CREATE_PHY_OP_INPUT(ObExprValuesInput, ctx, get_id(), get_type(), input))) {
|
||||
LOG_WARN("failed to create phy op input.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
UNUSED(input);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObExprValues::to_string_kv(char* buf, const int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
J_KV(K_(values));
|
||||
return pos;
|
||||
}
|
||||
|
||||
int ObExprValues::serialize(char* buf, int64_t buf_len, int64_t& pos, ObPhyOpSeriCtx& seri_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t len = get_serialize_size_(seri_ctx);
|
||||
int64_t value_count = 0;
|
||||
bool use_range_param = !range_params_.empty();
|
||||
int64_t range_param_count = 0;
|
||||
int64_t col_num = get_column_count();
|
||||
const ObIArray<int64_t>* row_id_list = static_cast<const ObIArray<int64_t>*>(seri_ctx.row_id_list_);
|
||||
if (OB_SUCC(ret)) {
|
||||
if (use_range_param) {
|
||||
value_count = 0;
|
||||
if (row_id_list != NULL) {
|
||||
range_param_count = row_id_list->count();
|
||||
} else if (OB_NOT_NULL(seri_ctx.exec_ctx_) && !seri_ctx.exec_ctx_->get_row_id_list_array().empty()) {
|
||||
range_param_count = seri_ctx.exec_ctx_->get_row_id_list_total_count();
|
||||
} else {
|
||||
range_param_count = range_params_.count();
|
||||
}
|
||||
} else {
|
||||
range_param_count = 0;
|
||||
if (row_id_list != NULL) {
|
||||
value_count = col_num * row_id_list->count();
|
||||
} else if (OB_NOT_NULL(seri_ctx.exec_ctx_) && !seri_ctx.exec_ctx_->get_row_id_list_array().empty()) {
|
||||
value_count = seri_ctx.exec_ctx_->get_row_id_list_total_count() * col_num;
|
||||
} else {
|
||||
value_count = values_.count();
|
||||
}
|
||||
}
|
||||
}
|
||||
OB_UNIS_ENCODE(UNIS_VERSION);
|
||||
OB_UNIS_ENCODE(len);
|
||||
OB_UNIS_ENCODE(value_count);
|
||||
if (OB_SUCC(ret) && !use_range_param) {
|
||||
if (row_id_list != NULL) {
|
||||
ARRAY_FOREACH(*row_id_list, idx)
|
||||
{
|
||||
int64_t start_idx = row_id_list->at(idx) * col_num;
|
||||
int64_t end_idx = start_idx + col_num;
|
||||
for (int64_t i = start_idx; OB_SUCC(ret) && i < end_idx; ++i) {
|
||||
if (OB_ISNULL(values_.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("node is null");
|
||||
} else if (OB_FAIL(values_.at(i)->serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("serialize expr node failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
} else if (OB_NOT_NULL(seri_ctx.exec_ctx_) && !seri_ctx.exec_ctx_->get_row_id_list_array().empty()) {
|
||||
const ObIArray<int64_t>* new_row_id_list = NULL;
|
||||
for (int array_idx = 0; OB_SUCC(ret) && array_idx < seri_ctx.exec_ctx_->get_row_id_list_array().count();
|
||||
++array_idx) {
|
||||
if (OB_ISNULL(new_row_id_list = seri_ctx.exec_ctx_->get_row_id_list_array().at(array_idx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("row id list is null", K(ret));
|
||||
} else {
|
||||
ARRAY_FOREACH(*new_row_id_list, idx)
|
||||
{
|
||||
int64_t start_idx = new_row_id_list->at(idx) * col_num;
|
||||
int64_t end_idx = start_idx + col_num;
|
||||
for (int64_t i = start_idx; OB_SUCC(ret) && i < end_idx; ++i) {
|
||||
if (OB_ISNULL(values_.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("node is null");
|
||||
} else if (OB_FAIL(values_.at(i)->serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("serialize expr node failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ARRAY_FOREACH(values_, i)
|
||||
{
|
||||
if (OB_ISNULL(values_.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("expr node is null");
|
||||
} else if (OB_FAIL(values_.at(i)->serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("serialize expr node failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(ObNoChildrenPhyOperator::serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("serialize physical operator failed", K(ret));
|
||||
}
|
||||
}
|
||||
OB_UNIS_ENCODE(range_param_count);
|
||||
if (OB_SUCC(ret) && use_range_param) {
|
||||
if (row_id_list != NULL) {
|
||||
ARRAY_FOREACH(*row_id_list, idx)
|
||||
{
|
||||
int64_t row_index = row_id_list->at(idx);
|
||||
if (OB_ISNULL(range_params_.at(row_index))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("range params is invalid", K(ret), K(row_index));
|
||||
} else if (OB_FAIL(range_params_.at(row_index)->serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("serialize range params failed", K(ret), K(row_index));
|
||||
}
|
||||
}
|
||||
} else if (OB_NOT_NULL(seri_ctx.exec_ctx_) && !seri_ctx.exec_ctx_->get_row_id_list_array().empty()) {
|
||||
const ObIArray<int64_t>* new_row_id_list = NULL;
|
||||
for (int array_idx = 0; OB_SUCC(ret) && array_idx < seri_ctx.exec_ctx_->get_row_id_list_array().count();
|
||||
++array_idx) {
|
||||
if (OB_ISNULL(new_row_id_list = seri_ctx.exec_ctx_->get_row_id_list_array().at(array_idx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("row id list is null", K(ret));
|
||||
} else {
|
||||
ARRAY_FOREACH(*new_row_id_list, idx)
|
||||
{
|
||||
int64_t row_index = new_row_id_list->at(idx);
|
||||
if (OB_ISNULL(range_params_.at(row_index))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("range params is invalid", K(ret), K(row_index));
|
||||
} else if (OB_FAIL(range_params_.at(row_index)->serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("serialize range params failed", K(ret), K(row_index));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ARRAY_FOREACH(range_params_, i)
|
||||
{
|
||||
if (OB_ISNULL(range_params_.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("range param node is null");
|
||||
} else if (OB_FAIL(range_params_.at(i)->serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("serialize range param failed", K(ret), K(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObExprValues::get_serialize_size(const ObPhyOpSeriCtx& seri_ctx) const
|
||||
{
|
||||
int64_t len = get_serialize_size_(seri_ctx);
|
||||
OB_UNIS_ADD_LEN(len);
|
||||
OB_UNIS_ADD_LEN(UNIS_VERSION);
|
||||
return len;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObExprValues)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int32_t value_size = 0;
|
||||
int64_t range_param_count = 0;
|
||||
ObSqlExpression* expr = NULL;
|
||||
ObExprOperatorFactory expr_factory(my_phy_plan_->get_allocator());
|
||||
if (OB_ISNULL(my_phy_plan_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("phy_plan is null");
|
||||
}
|
||||
OB_UNIS_DECODE(value_size);
|
||||
if (OB_FAIL(init_value_count(value_size))) {
|
||||
LOG_WARN("init value count failed", K(ret));
|
||||
}
|
||||
for (int32_t i = 0; OB_SUCC(ret) && i < value_size; ++i) {
|
||||
if (OB_FAIL(ObSqlExpressionUtil::make_sql_expr(my_phy_plan_, expr))) {
|
||||
LOG_WARN("make sql expression failed", K(ret));
|
||||
} else if (OB_ISNULL(expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("expr is null");
|
||||
} else if (OB_FAIL(expr->deserialize(buf, data_len, pos))) {
|
||||
LOG_WARN("deserialize expression failed", K(ret), K(i));
|
||||
} else if (OB_FAIL(values_.push_back(expr))) {
|
||||
LOG_WARN("add expr node value failed", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(ObNoChildrenPhyOperator::deserialize(buf, data_len, pos))) {
|
||||
LOG_WARN("deserialize physical operator failed", K(ret));
|
||||
}
|
||||
}
|
||||
OB_UNIS_DECODE(range_param_count);
|
||||
if (OB_SUCC(ret) && range_param_count > 0) {
|
||||
if (OB_FAIL(range_params_.init(range_param_count))) {
|
||||
LOG_WARN("init range param store failed", K(ret));
|
||||
}
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < range_param_count; ++i) {
|
||||
ObIterExprRangeParam* range_param_expr = NULL;
|
||||
if (OB_FAIL(expr_factory.alloc(T_OP_RANGE_PARAM, range_param_expr))) {
|
||||
LOG_WARN("allocate range param expr failed", K(ret));
|
||||
} else if (OB_FAIL(range_param_expr->deserialize(buf, data_len, pos))) {
|
||||
LOG_WARN("deserialize range param expr failed", K(ret));
|
||||
} else if (OB_FAIL(range_params_.push_back(range_param_expr))) {
|
||||
LOG_WARN("store range param expr failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObExprValues::get_serialize_size_(const ObPhyOpSeriCtx& seri_ctx) const
|
||||
{
|
||||
int64_t len = 0;
|
||||
int64_t value_count = 0;
|
||||
bool use_range_param = !range_params_.empty();
|
||||
int64_t range_param_count = 0;
|
||||
int64_t col_num = get_column_count();
|
||||
const ObIArray<int64_t>* row_id_list = static_cast<const ObIArray<int64_t>*>(seri_ctx.row_id_list_);
|
||||
if (use_range_param) {
|
||||
value_count = 0;
|
||||
if (row_id_list != NULL) {
|
||||
range_param_count = row_id_list->count();
|
||||
} else if (OB_NOT_NULL(seri_ctx.exec_ctx_) && !seri_ctx.exec_ctx_->get_row_id_list_array().empty()) {
|
||||
range_param_count = seri_ctx.exec_ctx_->get_row_id_list_total_count();
|
||||
} else {
|
||||
range_param_count = range_params_.count();
|
||||
}
|
||||
} else {
|
||||
range_param_count = 0;
|
||||
if (row_id_list != NULL) {
|
||||
value_count = col_num * row_id_list->count();
|
||||
} else if (OB_NOT_NULL(seri_ctx.exec_ctx_) && !seri_ctx.exec_ctx_->get_row_id_list_array().empty()) {
|
||||
value_count = seri_ctx.exec_ctx_->get_row_id_list_total_count() * col_num;
|
||||
} else {
|
||||
value_count = values_.count();
|
||||
}
|
||||
}
|
||||
if (!use_range_param) {
|
||||
if (row_id_list != NULL) {
|
||||
OB_UNIS_ADD_LEN(value_count);
|
||||
ARRAY_FOREACH_NORET(*row_id_list, idx)
|
||||
{
|
||||
int64_t start_idx = row_id_list->at(idx) * col_num;
|
||||
int64_t end_idx = start_idx + col_num;
|
||||
for (int64_t i = start_idx; i < end_idx; ++i) {
|
||||
if (OB_NOT_NULL(values_.at(i))) {
|
||||
OB_UNIS_ADD_LEN(*values_.at(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
} else if (OB_NOT_NULL(seri_ctx.exec_ctx_) && !seri_ctx.exec_ctx_->get_row_id_list_array().empty()) {
|
||||
int64_t real_value_count = 0;
|
||||
len += serialization::encoded_length_i64(real_value_count);
|
||||
const ObIArray<int64_t>* new_row_id_list = NULL;
|
||||
for (int array_idx = 0; array_idx < seri_ctx.exec_ctx_->get_row_id_list_array().count(); ++array_idx) {
|
||||
if (OB_ISNULL(new_row_id_list = seri_ctx.exec_ctx_->get_row_id_list_array().at(array_idx))) {
|
||||
LOG_WARN("row id list is null");
|
||||
} else {
|
||||
for (int idx = 0; idx < new_row_id_list->count(); ++idx) {
|
||||
int64_t start_idx = new_row_id_list->at(idx) * col_num;
|
||||
int64_t end_idx = start_idx + col_num;
|
||||
for (int64_t i = start_idx; i < end_idx; ++i) {
|
||||
OB_UNIS_ADD_LEN(*values_.at(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
OB_UNIS_ADD_LEN(value_count);
|
||||
ARRAY_FOREACH_NORET(values_, i)
|
||||
{
|
||||
if (OB_NOT_NULL(values_.at(i))) {
|
||||
OB_UNIS_ADD_LEN(*values_.at(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
OB_UNIS_ADD_LEN(value_count);
|
||||
}
|
||||
len += ObNoChildrenPhyOperator::get_serialize_size();
|
||||
OB_UNIS_ADD_LEN(range_param_count);
|
||||
if (use_range_param) {
|
||||
if (row_id_list != NULL) {
|
||||
ARRAY_FOREACH_NORET(*row_id_list, idx)
|
||||
{
|
||||
int64_t row_index = row_id_list->at(idx);
|
||||
if (OB_NOT_NULL(range_params_.at(row_index))) {
|
||||
OB_UNIS_ADD_LEN(*range_params_.at(row_index));
|
||||
}
|
||||
}
|
||||
} else if (OB_NOT_NULL(seri_ctx.exec_ctx_) && !seri_ctx.exec_ctx_->get_row_id_list_array().empty()) {
|
||||
const ObIArray<int64_t>* new_row_id_list = NULL;
|
||||
for (int array_idx = 0; array_idx < seri_ctx.exec_ctx_->get_row_id_list_array().count(); ++array_idx) {
|
||||
if (OB_ISNULL(new_row_id_list = seri_ctx.exec_ctx_->get_row_id_list_array().at(array_idx))) {
|
||||
LOG_WARN("row id list is null");
|
||||
} else {
|
||||
ARRAY_FOREACH_NORET(*new_row_id_list, idx)
|
||||
{
|
||||
int64_t row_index = new_row_id_list->at(idx);
|
||||
if (OB_ISNULL(range_params_.at(row_index))) {
|
||||
LOG_WARN("range params is invalid", K(row_index));
|
||||
} else {
|
||||
OB_UNIS_ADD_LEN(*range_params_.at(row_index));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ARRAY_FOREACH_NORET(range_params_, i)
|
||||
{
|
||||
if (OB_NOT_NULL(range_params_.at(i))) {
|
||||
OB_UNIS_ADD_LEN(*range_params_.at(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return len;
|
||||
}
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
116
src/sql/engine/basic/ob_expr_values.h
Normal file
116
src/sql/engine/basic/ob_expr_values.h
Normal file
@ -0,0 +1,116 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SQL_ENGINE_DML_EXPR_VALUES_H
|
||||
#define OCEANBASE_SQL_ENGINE_DML_EXPR_VALUES_H
|
||||
#include "lib/charset/ob_charset.h"
|
||||
#include "sql/engine/ob_no_children_phy_operator.h"
|
||||
#include "sql/engine/expr/ob_sql_expression.h"
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObIterExprRangeParam;
|
||||
|
||||
class ObExprValuesInput : public ObIPhyOperatorInput {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObExprValuesInput() : partition_id_values_(0)
|
||||
{}
|
||||
virtual ~ObExprValuesInput()
|
||||
{}
|
||||
virtual int init(ObExecContext&, ObTaskInfo&, const ObPhyOperator&)
|
||||
{
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
virtual void reset() override
|
||||
{}
|
||||
virtual ObPhyOperatorType get_phy_op_type() const
|
||||
{
|
||||
return PHY_EXPR_VALUES;
|
||||
}
|
||||
|
||||
public:
|
||||
int64_t partition_id_values_;
|
||||
};
|
||||
|
||||
class ObExprValues : public ObNoChildrenPhyOperator {
|
||||
class ObExprValuesCtx;
|
||||
|
||||
public:
|
||||
explicit ObExprValues(common::ObIAllocator& alloc);
|
||||
virtual ~ObExprValues();
|
||||
|
||||
virtual int rescan(ObExecContext& ctx) const;
|
||||
virtual int switch_iterator(ObExecContext& ctx) const;
|
||||
int init_value_count(int64_t value_cnt);
|
||||
int add_value(ObSqlExpression* value);
|
||||
int init_range_params(int64_t range_param_count);
|
||||
int add_range_param(ObIterExprRangeParam* range_param_expr);
|
||||
int64_t get_size() const
|
||||
{
|
||||
return values_.count();
|
||||
}
|
||||
virtual int create_operator_input(ObExecContext& ctx) const override;
|
||||
virtual int serialize(char* buf, int64_t buf_len, int64_t& pos, ObPhyOpSeriCtx& seri_ctx) const;
|
||||
virtual int64_t get_serialize_size(const ObPhyOpSeriCtx& seri_ctx) const;
|
||||
int64_t get_serialize_size_(const ObPhyOpSeriCtx& seri_ctx) const;
|
||||
int deserialize(const char* buf, const int64_t data_len, int64_t& pos);
|
||||
int deserialize_(const char* buf, const int64_t data_len, int64_t& pos);
|
||||
int deserialize_dispatch_(const char* buf, const int64_t data_len, int64_t& pos, std::false_type)
|
||||
{
|
||||
return deserialize_(buf, data_len, pos);
|
||||
}
|
||||
|
||||
private:
|
||||
const static int64_t UNIS_VERSION = 1;
|
||||
|
||||
private:
|
||||
// function members
|
||||
/**
|
||||
* @brief init operator context, will create a physical operator context (and a current row space)
|
||||
* @param ctx[in], execute context
|
||||
* @return if success, return OB_SUCCESS, otherwise, return errno
|
||||
*/
|
||||
virtual int init_op_ctx(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief called by get_next_row(), get a row from the child operator or row_store
|
||||
* @param ctx[in], execute context
|
||||
* @param row[out], ObSqlRow an obj array and row_size
|
||||
*/
|
||||
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
/**
|
||||
* @brief open operator, not including children operators.
|
||||
* called by open.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_open(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief close operator, not including children operators.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_close(ObExecContext& ctx) const;
|
||||
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
|
||||
int calc_next_row(ObExecContext& ctx) const;
|
||||
int calc_next_row_by_range_param(ObExprValuesCtx& value_ctx, const common::ObNewRow*& row) const;
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObExprValues);
|
||||
|
||||
private:
|
||||
// data members
|
||||
common::ObFixedArray<ObSqlExpression*, common::ObIAllocator> values_;
|
||||
common::ObFixedArray<ObIterExprRangeParam*, common::ObIAllocator> range_params_;
|
||||
};
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SQL_ENGINE_DML_EXPR_VALUES_H */
|
||||
400
src/sql/engine/basic/ob_expr_values_op.cpp
Normal file
400
src/sql/engine/basic/ob_expr_values_op.cpp
Normal file
@ -0,0 +1,400 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "sql/engine/basic/ob_expr_values_op.h"
|
||||
#include "sql/engine/expr/ob_expr_column_conv.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/engine/ob_phy_operator.h"
|
||||
#include "sql/resolver/expr/ob_raw_expr_util.h"
|
||||
#include "sql/engine/expr/ob_expr_type_to_str.h"
|
||||
#include "sql/engine/px/ob_dfo.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
OB_SERIALIZE_MEMBER(ObExprValuesOpInput, partition_id_values_);
|
||||
|
||||
int ObExprValuesSpec::serialize(char* buf, int64_t buf_len, int64_t& pos, ObPhyOpSeriCtx& seri_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t len = get_serialize_size_(seri_ctx);
|
||||
OB_UNIS_ENCODE(UNIS_VERSION);
|
||||
OB_UNIS_ENCODE(len);
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(ObOpSpec::serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("serialize physical operator failed", K(ret));
|
||||
} else {
|
||||
const ObIArray<int64_t>* row_id_list = static_cast<const ObIArray<int64_t>*>(seri_ctx.row_id_list_);
|
||||
int64_t col_num = get_output_count();
|
||||
if (row_id_list != NULL) {
|
||||
int64_t value_count = col_num * row_id_list->count();
|
||||
OB_UNIS_ENCODE(value_count);
|
||||
if (OB_SUCC(ret)) {
|
||||
ARRAY_FOREACH(*row_id_list, idx)
|
||||
{
|
||||
int64_t start_idx = row_id_list->at(idx) * col_num;
|
||||
int64_t end_idx = start_idx + col_num;
|
||||
for (int64_t i = start_idx; OB_SUCC(ret) && i < end_idx; ++i) {
|
||||
OB_UNIS_ENCODE(values_.at(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
} else if (OB_NOT_NULL(seri_ctx.exec_ctx_) && !seri_ctx.exec_ctx_->get_row_id_list_array().empty()) {
|
||||
int64_t value_count = seri_ctx.exec_ctx_->get_row_id_list_total_count() * col_num;
|
||||
OB_UNIS_ENCODE(value_count);
|
||||
for (int array_idx = 0; OB_SUCC(ret) && array_idx < seri_ctx.exec_ctx_->get_row_id_list_array().count();
|
||||
++array_idx) {
|
||||
if (OB_ISNULL(row_id_list = seri_ctx.exec_ctx_->get_row_id_list_array().at(array_idx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("row id list is null", K(ret));
|
||||
} else {
|
||||
ARRAY_FOREACH(*row_id_list, idx)
|
||||
{
|
||||
int64_t start_idx = row_id_list->at(idx) * col_num;
|
||||
int64_t end_idx = start_idx + col_num;
|
||||
for (int64_t i = start_idx; OB_SUCC(ret) && i < end_idx; ++i) {
|
||||
OB_UNIS_ENCODE(values_.at(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
OB_UNIS_ENCODE(values_);
|
||||
}
|
||||
OB_UNIS_ENCODE(str_values_array_);
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObExprValuesSpec::get_serialize_size(const ObPhyOpSeriCtx& seri_ctx) const
|
||||
{
|
||||
int64_t len = get_serialize_size_(seri_ctx);
|
||||
OB_UNIS_ADD_LEN(len);
|
||||
OB_UNIS_ADD_LEN(UNIS_VERSION);
|
||||
return len;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE_SIZE(ObExprValuesSpec)
|
||||
{
|
||||
int64_t len = 0;
|
||||
BASE_ADD_LEN((ObExprValuesSpec, ObOpSpec));
|
||||
OB_UNIS_ADD_LEN(values_);
|
||||
OB_UNIS_ADD_LEN(str_values_array_);
|
||||
return len;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE(ObExprValuesSpec)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
BASE_SER((ObExprValuesSpec, ObOpSpec));
|
||||
OB_UNIS_ENCODE(values_);
|
||||
OB_UNIS_ENCODE(str_values_array_);
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObExprValuesSpec)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
BASE_DESER((ObExprValuesSpec, ObOpSpec));
|
||||
OB_UNIS_DECODE(values_);
|
||||
OB_UNIS_DECODE(str_values_array_);
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObExprValuesSpec::get_serialize_size_(const ObPhyOpSeriCtx& seri_ctx) const
|
||||
{
|
||||
int64_t len = 0;
|
||||
const ObIArray<int64_t>* row_id_list = static_cast<const ObIArray<int64_t>*>(seri_ctx.row_id_list_);
|
||||
len += ObOpSpec::get_serialize_size();
|
||||
int64_t col_num = get_output_count();
|
||||
if (row_id_list != NULL) {
|
||||
int64_t value_size = col_num * row_id_list->count();
|
||||
OB_UNIS_ADD_LEN(value_size);
|
||||
ARRAY_FOREACH_NORET(*row_id_list, idx)
|
||||
{
|
||||
int64_t start_idx = row_id_list->at(idx) * col_num;
|
||||
int64_t end_idx = start_idx + col_num;
|
||||
for (int64_t i = start_idx; i < end_idx; ++i) {
|
||||
OB_UNIS_ADD_LEN(values_.at(i));
|
||||
}
|
||||
}
|
||||
} else if (OB_NOT_NULL(seri_ctx.exec_ctx_) && !seri_ctx.exec_ctx_->get_row_id_list_array().empty()) {
|
||||
int64_t value_count = seri_ctx.exec_ctx_->get_row_id_list_total_count() * col_num;
|
||||
OB_UNIS_ADD_LEN(value_count);
|
||||
for (int array_idx = 0; array_idx < seri_ctx.exec_ctx_->get_row_id_list_array().count(); ++array_idx) {
|
||||
if (OB_ISNULL(row_id_list = seri_ctx.exec_ctx_->get_row_id_list_array().at(array_idx))) {
|
||||
LOG_WARN("row id list is null");
|
||||
} else {
|
||||
for (int idx = 0; idx < row_id_list->count(); ++idx) {
|
||||
int64_t start_idx = row_id_list->at(idx) * col_num;
|
||||
int64_t end_idx = start_idx + col_num;
|
||||
for (int64_t i = start_idx; i < end_idx; ++i) {
|
||||
OB_UNIS_ADD_LEN(values_.at(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
OB_UNIS_ADD_LEN(values_);
|
||||
}
|
||||
OB_UNIS_ADD_LEN(str_values_array_);
|
||||
|
||||
return len;
|
||||
}
|
||||
|
||||
ObExprValuesOp::ObExprValuesOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObOperator(exec_ctx, spec, input),
|
||||
node_idx_(0),
|
||||
vector_index_(0),
|
||||
datum_caster_(),
|
||||
cm_(CM_NONE),
|
||||
value_count_(0),
|
||||
switch_value_(false)
|
||||
{}
|
||||
|
||||
int ObExprValuesOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
node_idx_ = 0;
|
||||
const bool is_explicit_cast = false;
|
||||
const int32_t result_flag = 0;
|
||||
if (OB_FAIL(datum_caster_.init(eval_ctx_.exec_ctx_))) {
|
||||
LOG_WARN("fail to init datum_caster", K(ret));
|
||||
} else if (OB_FAIL(ObSQLUtils::get_default_cast_mode(is_explicit_cast, result_flag, ctx_.get_my_session(), cm_))) {
|
||||
LOG_WARN("fail to get_default_cast_mode", K(ret));
|
||||
} else {
|
||||
switch_value_ = true;
|
||||
// see ObSQLUtils::wrap_column_convert_ctx(), add CM_WARN_ON_FAIL for INSERT IGNORE.
|
||||
ObPhysicalPlanCtx* plan_ctx = GET_PHY_PLAN_CTX(ctx_);
|
||||
cm_ = cm_ | CM_COLUMN_CONVERT;
|
||||
if (plan_ctx->is_ignore_stmt()) {
|
||||
// CM_CHARSET_CONVERT_IGNORE_ERR is will give '?' when do string_string convert.
|
||||
// eg: insert into t(gbk_col) values('𐐀');
|
||||
cm_ = cm_ | CM_WARN_ON_FAIL | CM_CHARSET_CONVERT_IGNORE_ERR;
|
||||
LOG_TRACE("is ignore, set CM_WARN_ON_FAIL and CM_CHARSET_CONVERT_IGNORE_ERR", K(cm_));
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValuesOp::get_value_count()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (MY_INPUT.partition_id_values_ != 0) {
|
||||
common::ObIArray<ObPxSqcMeta::PartitionIdValue>* pid_values =
|
||||
reinterpret_cast<common::ObIArray<ObPxSqcMeta::PartitionIdValue>*>(MY_INPUT.partition_id_values_);
|
||||
int64_t partition_id = ctx_.get_expr_partition_id();
|
||||
bool find = false;
|
||||
int64_t col_num = MY_SPEC.get_output_count();
|
||||
CK(partition_id != OB_INVALID_ID);
|
||||
for (int i = 0; OB_SUCC(ret) && i < pid_values->count() && !find; ++i) {
|
||||
if (partition_id == pid_values->at(i).partition_id_) {
|
||||
node_idx_ = pid_values->at(i).value_begin_idx_ * col_num;
|
||||
value_count_ = node_idx_ + pid_values->at(i).value_count_ * col_num;
|
||||
if (OB_FAIL(value_count_ > MY_SPEC.values_.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected value count", K(ret), K(value_count_), K(MY_SPEC.values_.count()));
|
||||
}
|
||||
find = true;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (!find) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected partition id", K(partition_id));
|
||||
} else {
|
||||
switch_value_ = false;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
value_count_ = MY_SPEC.values_.count();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValuesOp::rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (MY_INPUT.partition_id_values_ != 0 && ctx_.is_gi_restart()) {
|
||||
switch_value_ = true;
|
||||
} else {
|
||||
node_idx_ = 0;
|
||||
}
|
||||
OZ(ObOperator::rescan());
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValuesOp::switch_iterator()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhysicalPlanCtx* plan_ctx = GET_PHY_PLAN_CTX(ctx_);
|
||||
node_idx_ = 0;
|
||||
if (plan_ctx->get_bind_array_idx() >= plan_ctx->get_bind_array_count() - 1) {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValuesOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
clear_evaluated_flag();
|
||||
ObPhysicalPlanCtx* plan_ctx = GET_PHY_PLAN_CTX(ctx_);
|
||||
if (OB_SUCC(ret)) {
|
||||
plan_ctx->set_autoinc_id_tmp(0);
|
||||
if (OB_FAIL(try_check_status())) {
|
||||
LOG_WARN("check physical plan status faild", K(ret));
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
if (switch_value_ && OB_FAIL(get_value_count())) {
|
||||
LOG_WARN("fail to get value count", K(ret));
|
||||
} else if (OB_FAIL(calc_next_row())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get next row from row store failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
LOG_DEBUG("output row", "row", ROWEXPR2STR(eval_ctx_, MY_SPEC.output_));
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_INLINE int ObExprValuesOp::calc_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
NG_TRACE_TIMES(2, value_start_calc_row);
|
||||
ObSQLSessionInfo* session = ctx_.get_my_session();
|
||||
int64_t col_num = MY_SPEC.get_output_count();
|
||||
int64_t col_idx = 0;
|
||||
if (node_idx_ == value_count_) {
|
||||
// there is no values any more
|
||||
ret = OB_ITER_END;
|
||||
} else {
|
||||
bool is_break = false;
|
||||
ObDatum* datum = NULL;
|
||||
ObPhysicalPlanCtx* plan_ctx = GET_PHY_PLAN_CTX(ctx_);
|
||||
while (OB_SUCC(ret) && node_idx_ < value_count_ && !is_break) {
|
||||
ObExpr* src_expr = MY_SPEC.values_.at(node_idx_);
|
||||
ObExpr* dst_expr = MY_SPEC.output_.at(col_idx);
|
||||
ObDatumMeta src_meta = src_expr->datum_meta_;
|
||||
ObObjMeta src_obj_meta = src_expr->obj_meta_;
|
||||
if (T_QUESTIONMARK == src_expr->type_ &&
|
||||
(src_expr->frame_idx_ < spec_.plan_->get_expr_frame_info().const_frame_.count() +
|
||||
spec_.plan_->get_expr_frame_info().param_frame_.count())) {
|
||||
/*
|
||||
* the 2nd condition with frame_idx is used to support subquery in values,
|
||||
* in this case the subquery expr will be replaced to question mark, we can
|
||||
* get its meta info from expr directly, not from param_store.
|
||||
*/
|
||||
int64_t param_idx = src_expr->extra_;
|
||||
ObPhysicalPlanCtx* plan_ctx = GET_PHY_PLAN_CTX(ctx_);
|
||||
if (param_idx < 0 || param_idx >= plan_ctx->get_param_store().count()) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid param idx", K(ret), K(param_idx));
|
||||
} else {
|
||||
src_obj_meta = plan_ctx->get_param_store().at(param_idx).meta_;
|
||||
src_meta.type_ = src_obj_meta.get_type();
|
||||
src_meta.cs_type_ = src_obj_meta.get_collation_type();
|
||||
const ObAccuracy& src_obj_acc = plan_ctx->get_param_store().at(param_idx).get_accuracy();
|
||||
src_meta.scale_ = src_obj_acc.get_scale();
|
||||
src_meta.precision_ = src_obj_acc.get_precision();
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
// do nothing
|
||||
} else if (src_expr == dst_expr) {
|
||||
// do nothing
|
||||
} else if (src_meta.type_ == dst_expr->datum_meta_.type_ && src_meta.cs_type_ == dst_expr->datum_meta_.cs_type_) {
|
||||
if (OB_FAIL(src_expr->eval(eval_ctx_, datum))) {
|
||||
LOG_WARN("fail to cast values", K(ret), K(*src_expr));
|
||||
} else {
|
||||
dst_expr->locate_datum_for_write(eval_ctx_) = *datum;
|
||||
dst_expr->get_eval_info(eval_ctx_).evaluated_ = true;
|
||||
}
|
||||
} else {
|
||||
ObExpr real_src_expr = *src_expr;
|
||||
real_src_expr.datum_meta_ = src_meta;
|
||||
real_src_expr.obj_meta_ = src_obj_meta;
|
||||
real_src_expr.obj_datum_map_ = ObDatum::get_obj_datum_map_type(src_meta.type_);
|
||||
if (dst_expr->obj_meta_.is_enum_or_set()) {
|
||||
if (OB_UNLIKELY(col_idx < 0 || col_idx >= MY_SPEC.str_values_array_.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("col_idx is out of size", K(ret), K(col_idx), K(MY_SPEC.str_values_array_.count()));
|
||||
} else {
|
||||
const ObIArray<ObString>& str_values = MY_SPEC.str_values_array_.at(col_idx);
|
||||
if (OB_FAIL(datum_caster_.to_type(dst_expr->datum_meta_, str_values, real_src_expr, cm_, datum))) {
|
||||
LOG_WARN("fail to do to_type", K(ret), K(*dst_expr), K(real_src_expr));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(datum_caster_.to_type(dst_expr->datum_meta_, real_src_expr, cm_, datum))) {
|
||||
LOG_WARN("fail to do to_type", K(ret), K(*dst_expr), K(real_src_expr));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
ObDatum& dst_datum = dst_expr->locate_datum_for_write(eval_ctx_);
|
||||
if (ObObjDatumMapType::OBJ_DATUM_STRING == dst_expr->obj_datum_map_ ||
|
||||
OBJ_DATUM_LOB_LOCATOR == dst_expr->obj_datum_map_) {
|
||||
ObExprStrResAlloc res_alloc(*dst_expr, eval_ctx_);
|
||||
if (OB_FAIL(dst_datum.deep_copy(*datum, res_alloc))) {
|
||||
LOG_WARN("fail to deep copy datum from cast res datum", K(ret), K(*datum));
|
||||
}
|
||||
} else {
|
||||
ObDataBuffer res_alloc(const_cast<char*>(dst_datum.ptr_), dst_expr->res_buf_len_);
|
||||
if (OB_FAIL(dst_datum.deep_copy(*datum, res_alloc))) {
|
||||
LOG_WARN("fail to deep copy datum from cast res datum", K(ret), K(*datum));
|
||||
}
|
||||
}
|
||||
dst_expr->get_eval_info(eval_ctx_).evaluated_ = true;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
LOG_DEBUG("expr values row columns", K(node_idx_), K(col_idx), K(*datum));
|
||||
++node_idx_;
|
||||
if (col_idx == col_num - 1) {
|
||||
// last cell values resolved, output row now
|
||||
is_break = true;
|
||||
} else {
|
||||
col_idx = (col_idx + 1) % col_num;
|
||||
}
|
||||
}
|
||||
} // while end
|
||||
}
|
||||
NG_TRACE_TIMES(2, value_after_calc_row);
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValuesOp::inner_close()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
node_idx_ = 0;
|
||||
if (OB_FAIL(datum_caster_.destroy())) {
|
||||
LOG_WARN("fail to destroy datum_caster", K(ret));
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
103
src/sql/engine/basic/ob_expr_values_op.h
Normal file
103
src/sql/engine/basic/ob_expr_values_op.h
Normal file
@ -0,0 +1,103 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SQL_ENGINE_BASIC_OB_EXPR_VALUES_OP_
|
||||
#define OCEANBASE_SQL_ENGINE_BASIC_OB_EXPR_VALUES_OP_
|
||||
|
||||
#include "sql/engine/ob_operator.h"
|
||||
#include "sql/engine/expr/ob_datum_cast.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObPhyOpSeriCtx;
|
||||
|
||||
class ObExprValuesOpInput : public ObOpInput {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObExprValuesOpInput(ObExecContext& ctx, const ObOpSpec& spec) : ObOpInput(ctx, spec), partition_id_values_(0)
|
||||
{}
|
||||
virtual ~ObExprValuesOpInput()
|
||||
{}
|
||||
virtual void reset() override
|
||||
{}
|
||||
virtual int init(ObTaskInfo& task_info)
|
||||
{
|
||||
UNUSED(task_info);
|
||||
return common::OB_SUCCESS;
|
||||
}
|
||||
|
||||
public:
|
||||
int64_t partition_id_values_;
|
||||
};
|
||||
|
||||
class ObExprValuesSpec : public ObOpSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObExprValuesSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type)
|
||||
: ObOpSpec(alloc, type), values_(alloc), str_values_array_(alloc)
|
||||
{}
|
||||
|
||||
int64_t get_value_count() const
|
||||
{
|
||||
return values_.count();
|
||||
}
|
||||
|
||||
virtual int serialize(char* buf, int64_t buf_len, int64_t& pos, ObPhyOpSeriCtx& seri_ctx) const override;
|
||||
virtual int64_t get_serialize_size(const ObPhyOpSeriCtx& seri_ctx) const override;
|
||||
int64_t get_serialize_size_(const ObPhyOpSeriCtx& seri_ctx) const;
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObExprValuesSpec);
|
||||
|
||||
public:
|
||||
common::ObFixedArray<ObExpr*, common::ObIAllocator> values_;
|
||||
common::ObFixedArray<ObStrValues, common::ObIAllocator> str_values_array_;
|
||||
};
|
||||
|
||||
class ObExprValuesOp : public ObOperator {
|
||||
public:
|
||||
ObExprValuesOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
virtual int inner_open() override;
|
||||
virtual int rescan() override;
|
||||
|
||||
virtual int switch_iterator() override;
|
||||
|
||||
virtual int inner_get_next_row() override;
|
||||
|
||||
virtual int inner_close() override;
|
||||
|
||||
virtual void destroy() override
|
||||
{
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
private:
|
||||
int calc_next_row();
|
||||
int get_value_count();
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObExprValuesOp);
|
||||
|
||||
private:
|
||||
int64_t node_idx_;
|
||||
int64_t vector_index_;
|
||||
ObDatumCaster datum_caster_;
|
||||
common::ObCastMode cm_;
|
||||
int64_t value_count_;
|
||||
bool switch_value_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
#endif
|
||||
343
src/sql/engine/basic/ob_expr_values_with_child.cpp
Normal file
343
src/sql/engine/basic/ob_expr_values_with_child.cpp
Normal file
@ -0,0 +1,343 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/basic/ob_expr_values_with_child.h"
|
||||
#include "share/object/ob_obj_cast.h"
|
||||
#include "common/sql_mode/ob_sql_mode_utils.h"
|
||||
#include "sql/ob_sql_utils.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
using namespace share;
|
||||
namespace sql {
|
||||
class ObExprValuesWithChild::ObExprValuesWithChildCtx : public ObPhyOperatorCtx {
|
||||
public:
|
||||
explicit ObExprValuesWithChildCtx(ObExecContext& ctx) : ObPhyOperatorCtx(ctx), node_idx_(0)
|
||||
{}
|
||||
~ObExprValuesWithChildCtx()
|
||||
{}
|
||||
virtual void destroy()
|
||||
{
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
|
||||
private:
|
||||
int64_t node_idx_;
|
||||
friend class ObExprValuesWithChild;
|
||||
};
|
||||
|
||||
ObExprValuesWithChild::ObExprValuesWithChild(ObIAllocator& alloc) : ObSingleChildPhyOperator(alloc), values_(alloc)
|
||||
{}
|
||||
|
||||
ObExprValuesWithChild::~ObExprValuesWithChild()
|
||||
{}
|
||||
|
||||
int ObExprValuesWithChild::init_value_count(int64_t value_cnt)
|
||||
{
|
||||
return values_.init(value_cnt);
|
||||
}
|
||||
|
||||
int ObExprValuesWithChild::add_value(ObSqlExpression* value)
|
||||
{
|
||||
return values_.push_back(value);
|
||||
}
|
||||
|
||||
int ObExprValuesWithChild::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhyOperatorCtx* op_ctx = NULL;
|
||||
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObExprValuesWithChildCtx, ctx, get_id(), get_type(), op_ctx))) {
|
||||
LOG_WARN("create physical operator context failed", K(ret));
|
||||
} else if (OB_ISNULL(op_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("op_ctx is null");
|
||||
} else if (OB_FAIL(op_ctx->create_cur_row(get_column_count(), projector_, projector_size_))) {
|
||||
LOG_WARN("create current row failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValuesWithChild::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObExprValuesWithChildCtx* values_ctx = NULL;
|
||||
|
||||
if (OB_ISNULL(child_op_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("child op is not set", K(ret));
|
||||
} else if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("init operator context failed", K(ret));
|
||||
} else if (OB_ISNULL(values_ctx = GET_PHY_OPERATOR_CTX(ObExprValuesWithChildCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K(ret), K_(id));
|
||||
} else {
|
||||
values_ctx->node_idx_ = 0;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValuesWithChild::rescan(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObExprValuesWithChildCtx* values_ctx = NULL;
|
||||
if (OB_FAIL(ObSingleChildPhyOperator::rescan(ctx))) {
|
||||
LOG_WARN("rescan ObSingleChildPhyOperator failed", K(ret));
|
||||
} else if (OB_ISNULL(values_ctx = GET_PHY_OPERATOR_CTX(ObExprValuesWithChildCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("values_ctx is null");
|
||||
} else {
|
||||
values_ctx->node_idx_ = 0;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValuesWithChild::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObExprValuesWithChildCtx* values_ctx = NULL;
|
||||
if (OB_ISNULL(values_ctx = GET_PHY_OPERATOR_CTX(ObExprValuesWithChildCtx, ctx, get_id()))) {
|
||||
LOG_DEBUG("The operator has not been opened.", K(ret), K_(id), "op_type", ob_phy_operator_type_str(get_type()));
|
||||
} else {
|
||||
values_ctx->node_idx_ = 0;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValuesWithChild::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
NG_TRACE_TIMES(2, value_start_next_row);
|
||||
ObPhysicalPlanCtx* plan_ctx = NULL;
|
||||
ObExprValuesWithChildCtx* values_ctx = NULL;
|
||||
// reset temp auto-increment value before calculate a row
|
||||
|
||||
if (OB_ISNULL(plan_ctx = GET_PHY_PLAN_CTX(ctx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("plan_ctx is null");
|
||||
} else {
|
||||
plan_ctx->set_autoinc_id_tmp(0);
|
||||
}
|
||||
if (!OB_SUCC(ret)) {
|
||||
// do nothing
|
||||
} else if (OB_ISNULL(values_ctx = GET_PHY_OPERATOR_CTX(ObExprValuesWithChildCtx, ctx, id_))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K(ret), K_(id));
|
||||
} else if (OB_FAIL(THIS_WORKER.check_status())) {
|
||||
LOG_WARN("check physical plan status faild", K(ret));
|
||||
} else if (OB_FAIL(calc_next_row(ctx))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get next row from row store failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
row = &(values_ctx->get_cur_row());
|
||||
LOG_DEBUG("get next row from expr values", K(*row));
|
||||
NG_TRACE_TIMES(2, value_end_next_row);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValuesWithChild::calc_next_row(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObExprValuesWithChildCtx* value_ctx = NULL;
|
||||
ObSQLSessionInfo* session = ctx.get_my_session();
|
||||
int64_t col_num = get_column_count();
|
||||
int64_t col_index = 0;
|
||||
NG_TRACE_TIMES(2, value_start_calc_row);
|
||||
if (OB_UNLIKELY(get_size() <= 0 || get_column_count() <= 0 || get_size() % get_column_count() != 0 ||
|
||||
OB_ISNULL(my_phy_plan_))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("expr values status is unexpected", K(get_size()), K(get_column_count()), K_(my_phy_plan));
|
||||
} else if (OB_ISNULL(value_ctx = GET_PHY_OPERATOR_CTX(ObExprValuesWithChildCtx, ctx, id_))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("value_ctx is null");
|
||||
} else if (OB_FAIL(ObSQLUtils::get_default_cast_mode(my_phy_plan_, session, value_ctx->expr_ctx_.cast_mode_))) {
|
||||
LOG_WARN("get cast mode failed", K(ret));
|
||||
} else {
|
||||
ObExprCtx& expr_ctx = value_ctx->expr_ctx_;
|
||||
ObNewRow& cur_row = value_ctx->get_cur_row();
|
||||
if (OB_UNLIKELY(value_ctx->node_idx_ == get_size())) {
|
||||
// there is no values any more
|
||||
ret = OB_ITER_END;
|
||||
} else if (OB_UNLIKELY(cur_row.is_invalid())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("cur row is not init", K(cur_row));
|
||||
} else if (OB_FAIL(try_get_next_row(ctx))) {
|
||||
LOG_WARN("fail get next row from child", K(ret));
|
||||
} else {
|
||||
bool is_break = false;
|
||||
while (OB_SUCC(ret) && value_ctx->node_idx_ < get_size() && !is_break) {
|
||||
if (OB_ISNULL(values_.at(value_ctx->node_idx_))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("value exprs is null", K(values_.at(value_ctx->node_idx_)));
|
||||
} else if (OB_FAIL(values_.at(value_ctx->node_idx_)->calc(expr_ctx, cur_row, cur_row.cells_[col_index]))) {
|
||||
LOG_WARN("failed to calc expr", K(ret));
|
||||
} else {
|
||||
++value_ctx->node_idx_;
|
||||
if (col_index == col_num - 1) {
|
||||
// last cell values resolved, output row now
|
||||
is_break = true;
|
||||
} else {
|
||||
col_index = (col_index + 1) % col_num;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
NG_TRACE_TIMES(2, value_after_calc_row);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObExprValuesWithChild::try_get_next_row(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ObNewRow* sequence_row = nullptr;
|
||||
if (OB_FAIL(child_op_->get_next_row(ctx, sequence_row))) {
|
||||
LOG_WARN("fail get row from child", KR(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObExprValuesWithChild::to_string_kv(char* buf, const int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
J_KV(K_(values));
|
||||
return pos;
|
||||
}
|
||||
|
||||
int ObExprValuesWithChild::serialize(char* buf, int64_t buf_len, int64_t& pos, ObPhyOpSeriCtx& seri_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t len = get_serialize_size_(seri_ctx);
|
||||
int64_t value_count = get_size();
|
||||
OB_UNIS_ENCODE(UNIS_VERSION);
|
||||
OB_UNIS_ENCODE(len);
|
||||
int64_t col_num = get_column_count();
|
||||
const ObIArray<int64_t>* row_id_list = static_cast<const ObIArray<int64_t>*>(seri_ctx.row_id_list_);
|
||||
if (OB_SUCC(ret) && row_id_list != NULL) {
|
||||
value_count = col_num * row_id_list->count();
|
||||
}
|
||||
OB_UNIS_ENCODE(value_count);
|
||||
if (OB_SUCC(ret) && row_id_list != NULL) {
|
||||
ARRAY_FOREACH(*row_id_list, idx)
|
||||
{
|
||||
int64_t start_idx = row_id_list->at(idx) * col_num;
|
||||
int64_t end_idx = start_idx + col_num;
|
||||
for (int64_t i = start_idx; OB_SUCC(ret) && i < end_idx; ++i) {
|
||||
if (OB_ISNULL(values_.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("node is null");
|
||||
} else if (OB_FAIL(values_.at(i)->serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("serialize expr node failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && row_id_list == NULL) {
|
||||
ARRAY_FOREACH(values_, i)
|
||||
{
|
||||
if (OB_ISNULL(values_.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("expr node is null");
|
||||
} else if (OB_FAIL(values_.at(i)->serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("serialize expr node failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(ObSingleChildPhyOperator::serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("serialize physical operator failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObExprValuesWithChild::get_serialize_size(const ObPhyOpSeriCtx& seri_ctx) const
|
||||
{
|
||||
int64_t len = get_serialize_size_(seri_ctx);
|
||||
OB_UNIS_ADD_LEN(len);
|
||||
OB_UNIS_ADD_LEN(UNIS_VERSION);
|
||||
return len;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObExprValuesWithChild)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int32_t value_size = 0;
|
||||
ObSqlExpression* expr = NULL;
|
||||
if (OB_ISNULL(my_phy_plan_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("phy_plan is null");
|
||||
}
|
||||
OB_UNIS_DECODE(value_size);
|
||||
if (OB_FAIL(init_value_count(value_size))) {
|
||||
LOG_WARN("init value count failed", K(ret));
|
||||
}
|
||||
for (int32_t i = 0; OB_SUCC(ret) && i < value_size; ++i) {
|
||||
if (OB_FAIL(ObSqlExpressionUtil::make_sql_expr(my_phy_plan_, expr))) {
|
||||
LOG_WARN("make sql expression failed", K(ret));
|
||||
} else if (OB_ISNULL(expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("expr is null");
|
||||
} else if (OB_FAIL(expr->deserialize(buf, data_len, pos))) {
|
||||
LOG_WARN("deserialize expression failed", K(ret), K(i));
|
||||
} else if (OB_FAIL(values_.push_back(expr))) {
|
||||
LOG_WARN("add expr node value failed", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(ObSingleChildPhyOperator::deserialize(buf, data_len, pos))) {
|
||||
LOG_WARN("deserialize physical operator failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObExprValuesWithChild::get_serialize_size_(const ObPhyOpSeriCtx& seri_ctx) const
|
||||
{
|
||||
int64_t len = 0;
|
||||
int64_t col_num = get_column_count();
|
||||
int64_t value_size = get_size();
|
||||
const ObIArray<int64_t>* row_id_list = static_cast<const ObIArray<int64_t>*>(seri_ctx.row_id_list_);
|
||||
if (row_id_list != NULL) {
|
||||
value_size = col_num * row_id_list->count();
|
||||
}
|
||||
OB_UNIS_ADD_LEN(value_size);
|
||||
if (row_id_list != NULL) {
|
||||
ARRAY_FOREACH_NORET(*row_id_list, idx)
|
||||
{
|
||||
int64_t start_idx = row_id_list->at(idx) * col_num;
|
||||
int64_t end_idx = start_idx + col_num;
|
||||
for (int64_t i = start_idx; i < end_idx; ++i) {
|
||||
if (values_.at(i) != NULL) {
|
||||
OB_UNIS_ADD_LEN(*values_.at(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ARRAY_FOREACH_NORET(values_, i)
|
||||
{
|
||||
if (values_.at(i) != NULL) {
|
||||
OB_UNIS_ADD_LEN(*values_.at(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
len += ObSingleChildPhyOperator::get_serialize_size();
|
||||
return len;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
103
src/sql/engine/basic/ob_expr_values_with_child.h
Normal file
103
src/sql/engine/basic/ob_expr_values_with_child.h
Normal file
@ -0,0 +1,103 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SQL_ENGINE_DML_EXPR_VALUES_WITH_CHILD_H
|
||||
#define OCEANBASE_SQL_ENGINE_DML_EXPR_VALUES_WITH_CHILD_H
|
||||
#include "lib/charset/ob_charset.h"
|
||||
#include "sql/engine/ob_single_child_phy_operator.h"
|
||||
#include "sql/engine/expr/ob_sql_expression.h"
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObIterExprRangeParam;
|
||||
class ObExprValuesWithChild : public ObSingleChildPhyOperator {
|
||||
class ObExprValuesWithChildCtx;
|
||||
|
||||
public:
|
||||
explicit ObExprValuesWithChild(common::ObIAllocator& alloc);
|
||||
virtual ~ObExprValuesWithChild();
|
||||
|
||||
virtual int rescan(ObExecContext& ctx) const;
|
||||
int init_value_count(int64_t value_cnt);
|
||||
int add_value(ObSqlExpression* value);
|
||||
int init_range_params(int64_t range_param_count)
|
||||
{
|
||||
UNUSED(range_param_count);
|
||||
return common::OB_SUCCESS;
|
||||
}
|
||||
int add_range_param(ObIterExprRangeParam* range_param_expr)
|
||||
{
|
||||
UNUSED(range_param_expr);
|
||||
return common::OB_SUCCESS;
|
||||
}
|
||||
int64_t get_size() const
|
||||
{
|
||||
return values_.count();
|
||||
}
|
||||
virtual int serialize(char* buf, int64_t buf_len, int64_t& pos, ObPhyOpSeriCtx& seri_ctx) const;
|
||||
virtual int64_t get_serialize_size(const ObPhyOpSeriCtx& seri_ctx) const;
|
||||
int deserialize(const char* buf, const int64_t data_len, int64_t& pos);
|
||||
int deserialize_(const char* buf, const int64_t data_len, int64_t& pos);
|
||||
int deserialize_dispatch_(const char* buf, const int64_t data_len, int64_t& pos, std::false_type)
|
||||
{
|
||||
return deserialize_(buf, data_len, pos);
|
||||
}
|
||||
|
||||
private:
|
||||
const static int64_t UNIS_VERSION = 1;
|
||||
|
||||
private:
|
||||
// function members
|
||||
/**
|
||||
* @brief init operator context, will create a physical operator context (and a current row space)
|
||||
* @param ctx[in], execute context
|
||||
* @return if success, return OB_SUCCESS, otherwise, return errno
|
||||
*/
|
||||
virtual int init_op_ctx(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief called by get_next_row(), get a row from the child operator or row_store
|
||||
* @param ctx[in], execute context
|
||||
* @param row[out], ObSqlRow an obj array and row_size
|
||||
*/
|
||||
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
/**
|
||||
* @brief open operator, not including children operators.
|
||||
* called by open.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_open(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief close operator, not including children operators.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_close(ObExecContext& ctx) const;
|
||||
/*
|
||||
* ObExprValuesWithChild's child is ObSequence for insert stmt with sequence.
|
||||
*/
|
||||
int try_get_next_row(ObExecContext& ctx) const;
|
||||
/*
|
||||
* misc
|
||||
*/
|
||||
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
|
||||
int calc_next_row(ObExecContext& ctx) const;
|
||||
int64_t get_serialize_size_(const ObPhyOpSeriCtx& seri_ctx) const;
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObExprValuesWithChild);
|
||||
|
||||
private:
|
||||
// data members
|
||||
common::ObFixedArray<ObSqlExpression*, common::ObIAllocator> values_;
|
||||
};
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SQL_ENGINE_DML_EXPR_VALUES_WITH_CHILD_H */
|
||||
72
src/sql/engine/basic/ob_hash_partitioning_basic.h
Normal file
72
src/sql/engine/basic/ob_hash_partitioning_basic.h
Normal file
@ -0,0 +1,72 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OB_HASH_PARTITIONING_BASIC_H_
|
||||
#define OB_HASH_PARTITIONING_BASIC_H_
|
||||
|
||||
#include "share/datum/ob_datum_funcs.h"
|
||||
#include "sql/engine/sort/ob_sort_basic_info.h"
|
||||
#include "sql/engine/basic/ob_chunk_datum_store.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
static const uint64_t DEFAULT_PART_HASH_VALUE = 99194853094755497L;
|
||||
struct ObHashPartStoredRow : public sql::ObChunkDatumStore::StoredRow {
|
||||
const static int64_t HASH_VAL_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;
|
||||
};
|
||||
ExtraInfo& get_extra_info()
|
||||
{
|
||||
static_assert(sizeof(ObHashPartStoredRow) == 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());
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
static uint64_t get_hash_mask()
|
||||
{
|
||||
return HASH_VAL_MASK;
|
||||
}
|
||||
};
|
||||
|
||||
enum InputSide {
|
||||
LEFT = 0,
|
||||
RIGHT = 1,
|
||||
};
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
|
||||
#endif /* OB_HASH_PARTITIONING_BASIC_H_ */
|
||||
77
src/sql/engine/basic/ob_hash_partitioning_infrastructure.cpp
Normal file
77
src/sql/engine/basic/ob_hash_partitioning_infrastructure.cpp
Normal file
@ -0,0 +1,77 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "ob_hash_partitioning_infrastructure.h"
|
||||
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
bool HashPartCols::equal(const HashPartCols& other, const common::ObIArray<common::ObColumnInfo>* hash_col_idxs) const
|
||||
{
|
||||
bool result = true;
|
||||
const ObObj* lcell = NULL;
|
||||
const ObObj* rcell = NULL;
|
||||
if (OB_ISNULL(hash_col_idxs)) {
|
||||
result = false;
|
||||
} else if (nullptr == store_row_ || nullptr == other.store_row_) {
|
||||
result = false;
|
||||
} else {
|
||||
int64_t group_col_count = hash_col_idxs->count();
|
||||
for (int64_t i = 0; i < group_col_count && result; ++i) {
|
||||
int64_t group_idx = hash_col_idxs->at(i).index_;
|
||||
if (group_idx < store_row_->cnt_) {
|
||||
lcell = &store_row_->cells()[group_idx];
|
||||
}
|
||||
if (group_idx < other.store_row_->cnt_) {
|
||||
rcell = &other.store_row_->cells()[group_idx];
|
||||
}
|
||||
if (NULL == lcell || NULL == rcell) {
|
||||
result = false;
|
||||
} else {
|
||||
result = lcell->is_equal(*rcell, hash_col_idxs->at(i).cs_type_);
|
||||
}
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
bool HashPartCols::equal_temp(
|
||||
const TempHashPartCols& other, const common::ObIArray<common::ObColumnInfo>* hash_col_idxs) const
|
||||
{
|
||||
bool result = true;
|
||||
const ObObj* lcell = NULL;
|
||||
const ObObj* rcell = NULL;
|
||||
if (OB_ISNULL(hash_col_idxs)) {
|
||||
result = false;
|
||||
} else if (nullptr == store_row_ || nullptr == other.row_) {
|
||||
result = false;
|
||||
} else {
|
||||
int64_t group_col_count = hash_col_idxs->count();
|
||||
for (int64_t i = 0; i < group_col_count && result; ++i) {
|
||||
int64_t group_idx = hash_col_idxs->at(i).index_;
|
||||
if (group_idx < store_row_->cnt_) {
|
||||
lcell = &store_row_->cells()[group_idx];
|
||||
}
|
||||
if (group_idx < other.row_->get_count()) {
|
||||
rcell = &other.row_->get_cell(group_idx);
|
||||
}
|
||||
if (NULL == lcell || NULL == rcell) {
|
||||
result = false;
|
||||
} else {
|
||||
result = lcell->is_equal(*rcell, hash_col_idxs->at(i).cs_type_);
|
||||
}
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
1886
src/sql/engine/basic/ob_hash_partitioning_infrastructure.h
Normal file
1886
src/sql/engine/basic/ob_hash_partitioning_infrastructure.h
Normal file
File diff suppressed because it is too large
Load Diff
@ -0,0 +1,71 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "ob_hash_partitioning_infrastructure_op.h"
|
||||
|
||||
using namespace oceanbase::common;
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
bool ObHashPartCols::equal(const ObHashPartCols& other, const ObIArray<ObSortFieldCollation>* sort_collations,
|
||||
const ObIArray<ObCmpFunc>* cmp_funcs) const
|
||||
{
|
||||
bool result = true;
|
||||
const ObObj* lcell = NULL;
|
||||
const ObObj* rcell = NULL;
|
||||
if (OB_ISNULL(sort_collations) || OB_ISNULL(cmp_funcs)) {
|
||||
result = false;
|
||||
} else if (nullptr == store_row_ || nullptr == other.store_row_) {
|
||||
result = false;
|
||||
} else {
|
||||
int cmp_result = 0;
|
||||
ObDatum* l_cells = store_row_->cells();
|
||||
ObDatum* r_cells = other.store_row_->cells();
|
||||
for (int64_t i = 0; i < sort_collations->count() && 0 == cmp_result; ++i) {
|
||||
int64_t idx = sort_collations->at(i).field_idx_;
|
||||
cmp_result = cmp_funcs->at(i).cmp_func_(l_cells[idx], r_cells[idx]);
|
||||
}
|
||||
result = (0 == cmp_result);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
int ObHashPartCols::equal_temp(const ObTempHashPartCols& other, const ObIArray<ObSortFieldCollation>* sort_collations,
|
||||
const ObIArray<ObCmpFunc>* cmp_funcs, ObEvalCtx* eval_ctx, bool& result) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
result = true;
|
||||
const ObObj* lcell = NULL;
|
||||
const ObObj* rcell = NULL;
|
||||
if (OB_ISNULL(sort_collations) || OB_ISNULL(cmp_funcs) || OB_ISNULL(eval_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected status: compare info is null", K(sort_collations), K(cmp_funcs), K(eval_ctx), K(ret));
|
||||
} else if (nullptr == store_row_ || nullptr == other.exprs_) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected status: row is null", K(store_row_), K(other.exprs_), K(ret));
|
||||
} else {
|
||||
int cmp_result = 0;
|
||||
ObDatum* l_cells = store_row_->cells();
|
||||
ObDatum* r_cell = nullptr;
|
||||
for (int64_t i = 0; i < sort_collations->count() && 0 == cmp_result; ++i) {
|
||||
int64_t idx = sort_collations->at(i).field_idx_;
|
||||
if (OB_FAIL(other.exprs_->at(idx)->eval(*eval_ctx, r_cell))) {
|
||||
LOG_WARN("failed to eval datum", K(ret), K(i));
|
||||
} else {
|
||||
cmp_result = cmp_funcs->at(i).cmp_func_(l_cells[idx], *r_cell);
|
||||
}
|
||||
}
|
||||
result = (0 == cmp_result);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
1904
src/sql/engine/basic/ob_hash_partitioning_infrastructure_op.h
Normal file
1904
src/sql/engine/basic/ob_hash_partitioning_infrastructure_op.h
Normal file
File diff suppressed because it is too large
Load Diff
586
src/sql/engine/basic/ob_limit.cpp
Normal file
586
src/sql/engine/basic/ob_limit.cpp
Normal file
@ -0,0 +1,586 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/basic/ob_limit.h"
|
||||
#include "lib/utility/utility.h"
|
||||
#include "share/object/ob_obj_cast.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/expr/ob_sql_expression.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
// REGISTER_PHY_OPERATOR(ObLimit, PHY_LIMIT);
|
||||
|
||||
class ObLimit::ObLimitCtx : public ObPhyOperatorCtx {
|
||||
public:
|
||||
explicit ObLimitCtx(ObExecContext& ctx)
|
||||
: ObPhyOperatorCtx(ctx),
|
||||
limit_(-1),
|
||||
offset_(0),
|
||||
input_count_(0),
|
||||
output_count_(0),
|
||||
total_count_(0),
|
||||
is_percent_first_(false),
|
||||
limit_last_row_(NULL)
|
||||
{}
|
||||
virtual void destroy()
|
||||
{
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
|
||||
private:
|
||||
int64_t limit_;
|
||||
int64_t offset_;
|
||||
int64_t input_count_;
|
||||
int64_t output_count_;
|
||||
int64_t total_count_;
|
||||
bool is_percent_first_;
|
||||
ObNewRow* limit_last_row_;
|
||||
|
||||
friend class ObLimit;
|
||||
};
|
||||
|
||||
ObLimit::ObLimit(ObIAllocator& alloc)
|
||||
: ObSingleChildPhyOperator(alloc),
|
||||
org_limit_(NULL),
|
||||
org_offset_(NULL),
|
||||
org_percent_(NULL),
|
||||
is_calc_found_rows_(false),
|
||||
is_top_limit_(false),
|
||||
is_fetch_with_ties_(false),
|
||||
sort_columns_(alloc)
|
||||
{}
|
||||
|
||||
ObLimit::~ObLimit()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
void ObLimit::reset()
|
||||
{
|
||||
org_limit_ = NULL;
|
||||
org_offset_ = NULL;
|
||||
org_percent_ = NULL;
|
||||
is_calc_found_rows_ = false;
|
||||
is_top_limit_ = false;
|
||||
is_fetch_with_ties_ = false;
|
||||
sort_columns_.reset();
|
||||
ObSingleChildPhyOperator::reset();
|
||||
}
|
||||
|
||||
void ObLimit::reuse()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
int ObLimit::set_limit(ObSqlExpression* limit, ObSqlExpression* offset, ObSqlExpression* percent)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (limit) {
|
||||
org_limit_ = limit;
|
||||
}
|
||||
if (offset) {
|
||||
org_offset_ = offset;
|
||||
}
|
||||
if (percent) {
|
||||
org_percent_ = percent;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLimit::get_int_value(
|
||||
ObExecContext& ctx, const ObSqlExpression* in_val, int64_t& out_val, bool& is_null_value) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObNewRow input_row;
|
||||
ObObj result;
|
||||
ObExprCtx expr_ctx;
|
||||
is_null_value = false;
|
||||
if (in_val != NULL && !in_val->is_empty()) {
|
||||
if (OB_FAIL(wrap_expr_ctx(ctx, expr_ctx))) {
|
||||
LOG_WARN("wrap expr context failed", K(ret));
|
||||
} else if (OB_FAIL(in_val->calc(expr_ctx, input_row, result))) {
|
||||
LOG_WARN("Failed to calculate expression", K(ret));
|
||||
} else if (result.is_int()) {
|
||||
if (OB_FAIL(result.get_int(out_val))) {
|
||||
LOG_WARN("get_int error", K(ret), K(result));
|
||||
}
|
||||
} else if (result.is_null()) {
|
||||
out_val = 0;
|
||||
is_null_value = true;
|
||||
} else {
|
||||
EXPR_DEFINE_CAST_CTX(expr_ctx, CM_NONE);
|
||||
EXPR_GET_INT64_V2(result, out_val);
|
||||
if (OB_FAIL(ret)) {
|
||||
LOG_WARN("get_int error", K(ret), K(result));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLimit::get_double_value(ObExecContext& ctx, const ObSqlExpression* double_val, double& out_val) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObNewRow input_row;
|
||||
ObObj result;
|
||||
ObExprCtx expr_ctx;
|
||||
if (double_val != NULL && !double_val->is_empty()) {
|
||||
if (OB_FAIL(wrap_expr_ctx(ctx, expr_ctx))) {
|
||||
LOG_WARN("wrap expr context failed", K(ret));
|
||||
} else if (OB_FAIL(double_val->calc(expr_ctx, input_row, result))) {
|
||||
LOG_WARN("Failed to calculate expression", K(ret));
|
||||
} else if (result.is_double()) {
|
||||
if (OB_FAIL(result.get_double(out_val))) {
|
||||
LOG_WARN("get_double error", K(ret), K(result));
|
||||
}
|
||||
} else if (result.is_null()) {
|
||||
out_val = 0.0;
|
||||
} else {
|
||||
EXPR_DEFINE_CAST_CTX(expr_ctx, CM_NONE);
|
||||
EXPR_GET_DOUBLE_V2(result, out_val);
|
||||
if (OB_FAIL(ret)) {
|
||||
LOG_WARN("get_double error", K(ret), K(result));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLimit::get_limit(ObExecContext& ctx, int64_t& limit, int64_t& offset, bool& is_percent_first) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
double percent = 0.0;
|
||||
bool is_null_value = false;
|
||||
limit = -1;
|
||||
offset = 0;
|
||||
if (OB_FAIL(get_int_value(ctx, org_limit_, limit, is_null_value))) {
|
||||
LOG_WARN("Get limit value failed", K(ret));
|
||||
} else if (!is_null_value && OB_FAIL(get_int_value(ctx, org_offset_, offset, is_null_value))) {
|
||||
LOG_WARN("Get offset value failed", K(ret));
|
||||
} else if (is_null_value) {
|
||||
offset = 0;
|
||||
limit = 0;
|
||||
} else {
|
||||
is_percent_first = (org_percent_ != NULL && !org_percent_->is_empty());
|
||||
// revise limit, offset because rownum < -1 is rewritten as limit -1
|
||||
// offset 2 rows fetch next -3 rows only --> is meaningless
|
||||
offset = offset < 0 ? 0 : offset;
|
||||
if (org_limit_ != NULL && !org_limit_->is_empty()) {
|
||||
limit = limit < 0 ? 0 : limit;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObLimit::is_valid() const
|
||||
{
|
||||
return (get_column_count() > 0 && child_op_ != NULL && child_op_->get_column_count() > 0);
|
||||
}
|
||||
|
||||
int ObLimit::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObLimitCtx* limit_ctx = NULL;
|
||||
|
||||
if (OB_UNLIKELY(!is_valid())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("limit operator is invalid");
|
||||
} else if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("initialize operator context failed", K(ret));
|
||||
} else if (OB_ISNULL(limit_ctx = GET_PHY_OPERATOR_CTX(ObLimitCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K_(id));
|
||||
} else if (OB_FAIL(get_limit(ctx, limit_ctx->limit_, limit_ctx->offset_, limit_ctx->is_percent_first_))) {
|
||||
LOG_WARN("Failed to instantiate limit/offset", K(ret));
|
||||
} else {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLimit::rescan(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObLimitCtx* limit_ctx = NULL;
|
||||
if (OB_FAIL(ObSingleChildPhyOperator::rescan(ctx))) {
|
||||
LOG_WARN("rescan child physical operator failed", K(ret));
|
||||
} else if (OB_ISNULL(limit_ctx = GET_PHY_OPERATOR_CTX(ObLimitCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("limit_ctx is null");
|
||||
} else {
|
||||
limit_ctx->input_count_ = 0;
|
||||
limit_ctx->output_count_ = 0;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLimit::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
UNUSED(ctx);
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
int ObLimit::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhyOperatorCtx* op_ctx = NULL;
|
||||
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObLimitCtx, ctx, get_id(), get_type(), op_ctx))) {
|
||||
LOG_WARN("failed to create LimitCtx", K(ret));
|
||||
} else if (OB_ISNULL(op_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("op_ctx is null");
|
||||
} else if (OB_FAIL(init_cur_row(*op_ctx, need_copy_row_for_compute()))) {
|
||||
LOG_WARN("init current row failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLimit::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObLimitCtx* limit_ctx = NULL;
|
||||
ObSQLSessionInfo* my_session = NULL;
|
||||
const ObNewRow* input_row = NULL;
|
||||
|
||||
if (OB_FAIL(!is_valid())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("limit operator is invalid");
|
||||
} else if (OB_ISNULL(limit_ctx = GET_PHY_OPERATOR_CTX(ObLimitCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator ctx failed");
|
||||
} else if (OB_ISNULL(my_session = GET_MY_SESSION(ctx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get my session", K(ret));
|
||||
} else {
|
||||
}
|
||||
while (OB_SUCC(ret) && limit_ctx->input_count_ < limit_ctx->offset_) {
|
||||
if (OB_FAIL(child_op_->get_next_row(ctx, input_row))) {
|
||||
if (OB_UNLIKELY(OB_ITER_END != ret)) {
|
||||
LOG_WARN("child_op failed to get next row", K(limit_ctx->input_count_), K(limit_ctx->offset_), K(ret));
|
||||
}
|
||||
} else if (limit_ctx->is_percent_first_ && OB_FAIL(convert_limit_percent(ctx, limit_ctx))) {
|
||||
LOG_WARN("failed to convert limit percent", K(ret));
|
||||
} else {
|
||||
++limit_ctx->input_count_;
|
||||
}
|
||||
} // end while
|
||||
|
||||
/*
|
||||
* 1. is_percent_first_: for 'select * from t1 fetch next 50 percent rows only',
|
||||
* need set lower block operators like sort or agg, and then reset is_percent_first_ to false.
|
||||
* 2. is_fetch_with_ties_: when we get enough rows as limit count, shall we keep fetching for
|
||||
* those rows which equal to the last row of specified order (by order by clause) ?
|
||||
*/
|
||||
int64_t left_count = 0;
|
||||
if (OB_SUCC(ret)) {
|
||||
if (limit_ctx->is_percent_first_ || limit_ctx->output_count_ < limit_ctx->limit_ || limit_ctx->limit_ < 0) {
|
||||
if (OB_FAIL(child_op_->get_next_row(ctx, input_row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("child_op failed to get next row",
|
||||
K(ret),
|
||||
K_(limit_ctx->limit),
|
||||
K_(limit_ctx->offset),
|
||||
K_(limit_ctx->input_count),
|
||||
K_(limit_ctx->output_count));
|
||||
}
|
||||
} else if (limit_ctx->is_percent_first_ && OB_FAIL(convert_limit_percent(ctx, limit_ctx))) {
|
||||
LOG_WARN("failed to convert limit percent", K(ret));
|
||||
} else if (limit_ctx->limit_ == 0) {
|
||||
ret = OB_ITER_END;
|
||||
} else {
|
||||
++limit_ctx->output_count_;
|
||||
row = input_row;
|
||||
if (is_fetch_with_ties_ && limit_ctx->output_count_ == limit_ctx->limit_ &&
|
||||
OB_FAIL(deep_copy_limit_last_rows(limit_ctx, *row))) {
|
||||
LOG_WARN("failed to deep copy limit last rows");
|
||||
} else if (OB_FAIL(copy_cur_row(*limit_ctx, row))) {
|
||||
LOG_WARN("copy current row failed", K(ret));
|
||||
} else { /*do nothing*/
|
||||
}
|
||||
}
|
||||
} else if (limit_ctx->limit_ > 0 && is_fetch_with_ties_) {
|
||||
bool is_equal = false;
|
||||
if (OB_FAIL(child_op_->get_next_row(ctx, input_row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("child_op failed to get next row",
|
||||
K(ret),
|
||||
K_(limit_ctx->limit),
|
||||
K_(limit_ctx->offset),
|
||||
K_(limit_ctx->input_count),
|
||||
K_(limit_ctx->output_count));
|
||||
}
|
||||
} else if (OB_FAIL(is_row_order_by_item_value_equal(limit_ctx, input_row, is_equal))) {
|
||||
LOG_WARN("failed to is row order by item value equal", K(ret));
|
||||
} else if (is_equal) {
|
||||
++limit_ctx->output_count_;
|
||||
row = input_row;
|
||||
if (OB_FAIL(copy_cur_row(*limit_ctx, row))) {
|
||||
LOG_WARN("copy current row failed", K(ret));
|
||||
} else {
|
||||
LOG_DEBUG("copy cur row", K(*row));
|
||||
}
|
||||
} else {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
} else {
|
||||
ret = OB_ITER_END;
|
||||
if (is_calc_found_rows_) {
|
||||
const ObNewRow* tmp_row = NULL;
|
||||
while (OB_SUCC(child_op_->get_next_row(ctx, tmp_row))) {
|
||||
++left_count;
|
||||
}
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("fail to get next row from child", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_ITER_END == ret) {
|
||||
if (is_top_limit_) {
|
||||
limit_ctx->total_count_ = left_count + limit_ctx->output_count_ + limit_ctx->input_count_;
|
||||
ObPhysicalPlanCtx* plan_ctx = NULL;
|
||||
if (OB_ISNULL(plan_ctx = ctx.get_physical_plan_ctx())) {
|
||||
ret = OB_ERR_NULL_VALUE;
|
||||
LOG_WARN("get physical plan context failed");
|
||||
} else {
|
||||
NG_TRACE_EXT(
|
||||
found_rows, OB_ID(total_count), limit_ctx->total_count_, OB_ID(input_count), limit_ctx->input_count_);
|
||||
plan_ctx->set_found_rows(limit_ctx->total_count_);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLimit::deep_copy_limit_last_rows(ObLimitCtx* limit_ctx, const ObNewRow row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(limit_ctx) || OB_ISNULL(limit_ctx->calc_buf_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret), K(limit_ctx));
|
||||
} else {
|
||||
const int64_t buf_len = sizeof(ObNewRow) + row.get_deep_copy_size();
|
||||
int64_t pos = sizeof(ObNewRow);
|
||||
char* buf = NULL;
|
||||
if (OB_ISNULL(buf = static_cast<char*>(limit_ctx->calc_buf_->alloc(buf_len)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("alloc new row failed", K(ret), K(buf_len));
|
||||
} else if (OB_ISNULL(limit_ctx->limit_last_row_ = new (buf) ObNewRow())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("new_row is null", K(ret), K(buf_len));
|
||||
} else if (OB_FAIL(limit_ctx->limit_last_row_->deep_copy(row, buf, buf_len, pos))) {
|
||||
LOG_WARN("deep copy row failed", K(ret), K(buf_len), K(pos));
|
||||
} else { /*do nothing*/
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLimit::is_row_order_by_item_value_equal(ObLimitCtx* limit_ctx, const ObNewRow* input_row, bool& is_equal) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObNewRow* limit_last_row = NULL;
|
||||
is_equal = false;
|
||||
if (OB_ISNULL(limit_ctx) || OB_ISNULL(limit_last_row = limit_ctx->limit_last_row_) || OB_ISNULL(input_row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret), K(limit_ctx), K(input_row), K(limit_last_row));
|
||||
} else {
|
||||
is_equal = true;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && is_equal && i < sort_columns_.count(); ++i) {
|
||||
if (OB_UNLIKELY(sort_columns_.at(i).index_ >= input_row->count_ ||
|
||||
sort_columns_.at(i).index_ >= limit_last_row->count_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("order by projector is invalid",
|
||||
K(ret),
|
||||
K(sort_columns_.at(i).index_),
|
||||
K(input_row->count_),
|
||||
K(limit_last_row->count_));
|
||||
} else {
|
||||
is_equal = 0 == input_row->cells_[sort_columns_.at(i).index_].compare(
|
||||
limit_last_row->cells_[sort_columns_.at(i).index_], sort_columns_.at(i).cs_type_);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObLimit::to_string_kv(char* buf, const int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
if (org_limit_ && org_offset_) {
|
||||
J_KV(N_LIMIT, org_limit_, N_OFFSET, org_offset_);
|
||||
} else if (org_limit_) {
|
||||
J_KV(N_LIMIT, org_limit_);
|
||||
} else if (org_offset_) {
|
||||
J_KV(N_OFFSET, org_offset_);
|
||||
}
|
||||
return pos;
|
||||
}
|
||||
|
||||
int ObLimit::add_filter(ObSqlExpression* expr)
|
||||
{
|
||||
UNUSED(expr);
|
||||
LOG_ERROR("limit operator should have no filter expr");
|
||||
return OB_NOT_SUPPORTED;
|
||||
}
|
||||
|
||||
int ObLimit::add_sort_columns(ObSortColumn sort_column)
|
||||
{
|
||||
return sort_columns_.push_back(sort_column);
|
||||
}
|
||||
|
||||
int ObLimit::convert_limit_percent(ObExecContext& ctx, ObLimitCtx* limit_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
double percent = 0.0;
|
||||
if (OB_ISNULL(limit_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret), K(limit_ctx));
|
||||
} else if (OB_FAIL(get_double_value(ctx, org_percent_, percent))) {
|
||||
LOG_WARN("failed to get double value", K(ret));
|
||||
} else if (percent > 0) {
|
||||
int64_t tot_count = 0;
|
||||
if (OB_UNLIKELY(limit_ctx->limit_ != -1) || OB_ISNULL(child_op_) ||
|
||||
OB_UNLIKELY(child_op_->get_type() != PHY_MATERIAL && child_op_->get_type() != PHY_SORT)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(ret), K(limit_ctx->limit_), K(child_op_));
|
||||
} else if (child_op_->get_type() == PHY_MATERIAL &&
|
||||
OB_FAIL(static_cast<ObMaterial*>(child_op_)->get_material_row_count(ctx, tot_count))) {
|
||||
LOG_WARN("failed to get op row count", K(ret));
|
||||
} else if (child_op_->get_type() == PHY_SORT &&
|
||||
OB_FAIL(static_cast<ObSort*>(child_op_)->get_sort_row_count(ctx, tot_count))) {
|
||||
LOG_WARN("failed to get op row count", K(ret));
|
||||
} else if (OB_UNLIKELY(tot_count < 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get invalid child op row count", K(tot_count), K(ret));
|
||||
} else if (percent < 100) {
|
||||
int64_t percent_int64 = static_cast<int64_t>(percent);
|
||||
int64_t offset = (tot_count * percent / 100 - tot_count * percent_int64 / 100) > 0 ? 1 : 0;
|
||||
limit_ctx->limit_ = tot_count * percent_int64 / 100 + offset;
|
||||
limit_ctx->is_percent_first_ = false;
|
||||
} else {
|
||||
limit_ctx->limit_ = tot_count;
|
||||
limit_ctx->is_percent_first_ = false;
|
||||
}
|
||||
} else {
|
||||
limit_ctx->limit_ = 0;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE(ObLimit)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool has_limit_count = (org_limit_ != NULL && !org_limit_->is_empty());
|
||||
bool has_limit_offset = (org_offset_ != NULL && !org_offset_->is_empty());
|
||||
bool has_limit_percent = (org_percent_ != NULL && !org_percent_->is_empty());
|
||||
|
||||
OB_UNIS_ENCODE(is_calc_found_rows_);
|
||||
OB_UNIS_ENCODE(has_limit_count);
|
||||
OB_UNIS_ENCODE(has_limit_offset);
|
||||
OB_UNIS_ENCODE(is_top_limit_);
|
||||
if (has_limit_count) {
|
||||
OB_UNIS_ENCODE(*org_limit_);
|
||||
}
|
||||
if (has_limit_offset) {
|
||||
OB_UNIS_ENCODE(*org_offset_);
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(ObSingleChildPhyOperator::serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("serialize child operator failed", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
OB_UNIS_ENCODE(has_limit_percent);
|
||||
OB_UNIS_ENCODE(is_fetch_with_ties_);
|
||||
OB_UNIS_ENCODE(sort_columns_);
|
||||
if (has_limit_percent) {
|
||||
OB_UNIS_ENCODE(*org_percent_);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE_SIZE(ObLimit)
|
||||
{
|
||||
int64_t len = 0;
|
||||
bool has_limit_count = (org_limit_ != NULL && !org_limit_->is_empty());
|
||||
bool has_limit_offset = (org_offset_ != NULL && !org_offset_->is_empty());
|
||||
bool has_limit_percent = (org_percent_ != NULL && !org_percent_->is_empty());
|
||||
|
||||
OB_UNIS_ADD_LEN(is_calc_found_rows_);
|
||||
OB_UNIS_ADD_LEN(has_limit_count);
|
||||
OB_UNIS_ADD_LEN(has_limit_offset);
|
||||
OB_UNIS_ADD_LEN(is_top_limit_);
|
||||
if (has_limit_count) {
|
||||
OB_UNIS_ADD_LEN(*org_limit_);
|
||||
}
|
||||
if (has_limit_offset) {
|
||||
OB_UNIS_ADD_LEN(*org_offset_);
|
||||
}
|
||||
len += ObSingleChildPhyOperator::get_serialize_size();
|
||||
OB_UNIS_ADD_LEN(has_limit_percent);
|
||||
OB_UNIS_ADD_LEN(is_fetch_with_ties_);
|
||||
OB_UNIS_ADD_LEN(sort_columns_);
|
||||
if (has_limit_percent) {
|
||||
OB_UNIS_ADD_LEN(*org_percent_);
|
||||
}
|
||||
return len;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObLimit)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool has_limit_count = false;
|
||||
bool has_limit_offset = false;
|
||||
bool has_limit_percent = false;
|
||||
|
||||
OB_UNIS_DECODE(is_calc_found_rows_);
|
||||
OB_UNIS_DECODE(has_limit_count);
|
||||
OB_UNIS_DECODE(has_limit_offset);
|
||||
OB_UNIS_DECODE(is_top_limit_);
|
||||
if (OB_SUCC(ret) && has_limit_count) {
|
||||
if (OB_FAIL(ObSqlExpressionUtil::make_sql_expr(my_phy_plan_, org_limit_))) {
|
||||
LOG_WARN("make sql expression failed", K(ret));
|
||||
} else if (OB_LIKELY(org_limit_ != NULL)) {
|
||||
OB_UNIS_DECODE(*org_limit_);
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && has_limit_offset) {
|
||||
if (OB_FAIL(ObSqlExpressionUtil::make_sql_expr(my_phy_plan_, org_offset_))) {
|
||||
LOG_WARN("make sql expression failed", K(ret));
|
||||
} else if (OB_LIKELY(org_offset_ != NULL)) {
|
||||
OB_UNIS_DECODE(*org_offset_);
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
ret = ObSingleChildPhyOperator::deserialize(buf, data_len, pos);
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
OB_UNIS_DECODE(has_limit_percent);
|
||||
OB_UNIS_DECODE(is_fetch_with_ties_);
|
||||
OB_UNIS_DECODE(sort_columns_);
|
||||
if (OB_SUCC(ret) && has_limit_percent) {
|
||||
if (OB_FAIL(ObSqlExpressionUtil::make_sql_expr(my_phy_plan_, org_percent_))) {
|
||||
LOG_WARN("make sql expression failed", K(ret));
|
||||
} else if (OB_LIKELY(org_percent_ != NULL)) {
|
||||
OB_UNIS_DECODE(*org_percent_);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
113
src/sql/engine/basic/ob_limit.h
Normal file
113
src/sql/engine/basic/ob_limit.h
Normal file
@ -0,0 +1,113 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef _OB_LIMIT_H
|
||||
#define _OB_LIMIT_H 1
|
||||
#include "sql/engine/ob_single_child_phy_operator.h"
|
||||
#include "sql/engine/basic/ob_material.h"
|
||||
#include "sql/engine/sort/ob_sort.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObSqlExpression;
|
||||
|
||||
class ObLimit : public ObSingleChildPhyOperator {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
private:
|
||||
class ObLimitCtx;
|
||||
|
||||
public:
|
||||
explicit ObLimit(common::ObIAllocator& alloc);
|
||||
virtual ~ObLimit();
|
||||
|
||||
virtual void reset();
|
||||
virtual void reuse();
|
||||
/// @param limit -1 means no limit
|
||||
int set_limit(ObSqlExpression* limit, ObSqlExpression* offset, ObSqlExpression* percent);
|
||||
int get_limit(ObExecContext& ctx, int64_t& limit, int64_t& offset, bool& is_percent_first) const;
|
||||
virtual int rescan(ObExecContext& ctx) const;
|
||||
|
||||
void set_is_top_limit(const bool is_top)
|
||||
{
|
||||
is_top_limit_ = is_top;
|
||||
}
|
||||
void set_calc_found_rows(const bool found_rows)
|
||||
{
|
||||
is_calc_found_rows_ = found_rows;
|
||||
}
|
||||
void set_is_fetch_with_ties(const bool is_fetch_with_ties)
|
||||
{
|
||||
is_fetch_with_ties_ = is_fetch_with_ties;
|
||||
}
|
||||
int add_sort_columns(ObSortColumn sort_column);
|
||||
int init_sort_columns(int64_t count)
|
||||
{
|
||||
return init_array_size<>(sort_columns_, count);
|
||||
}
|
||||
int convert_limit_percent(ObExecContext& ctx, ObLimitCtx* limit_ctx) const;
|
||||
|
||||
private:
|
||||
bool is_valid() const;
|
||||
int get_int_value(ObExecContext& ctx, const ObSqlExpression* in_val, int64_t& out_val, bool& is_null_value) const;
|
||||
int get_double_value(ObExecContext& ctx, const ObSqlExpression* double_val, double& out_val) const;
|
||||
/**
|
||||
* @brief init operator context, will create a physical operator context (and a current row space)
|
||||
* @param ctx[in], execute context
|
||||
* @return if success, return OB_SUCCESS, otherwise, return errno
|
||||
*/
|
||||
int init_op_ctx(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief: called by get_next_row(), get a row from the child operator or row_store
|
||||
* @param: ctx[in], execute context
|
||||
* @param: row[out], ObSqlRow an obj array and row_size
|
||||
*/
|
||||
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
/**
|
||||
* @brief open operator, not including children operators.
|
||||
* called by open.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_open(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief close operator, not including children operators.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_close(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief for specified phy operator to print it's member variable with json key-value format
|
||||
* @param buf[in] to string buffer
|
||||
* @param buf_len[in] buffer length
|
||||
* @return if success, return the length used by print string, otherwise return 0
|
||||
*/
|
||||
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
|
||||
int add_filter(ObSqlExpression* expr);
|
||||
int deep_copy_limit_last_rows(ObLimitCtx* limit_ctx, const common::ObNewRow row) const;
|
||||
int is_row_order_by_item_value_equal(ObLimitCtx* limit_ctx, const common::ObNewRow* input_row, bool& is_equal) const;
|
||||
// disallow copy
|
||||
DISALLOW_COPY_AND_ASSIGN(ObLimit);
|
||||
|
||||
private:
|
||||
// data members
|
||||
ObSqlExpression* org_limit_;
|
||||
ObSqlExpression* org_offset_;
|
||||
ObSqlExpression* org_percent_;
|
||||
bool is_calc_found_rows_;
|
||||
bool is_top_limit_;
|
||||
bool is_fetch_with_ties_;
|
||||
// data members
|
||||
common::ObFixedArray<ObSortColumn, common::ObIAllocator> sort_columns_;
|
||||
};
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* _OB_LIMIT_H */
|
||||
263
src/sql/engine/basic/ob_limit_op.cpp
Normal file
263
src/sql/engine/basic/ob_limit_op.cpp
Normal file
@ -0,0 +1,263 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "ob_limit_op.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/engine/basic/ob_material_op.h"
|
||||
#include "sql/engine/sort/ob_sort_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
ObLimitSpec::ObLimitSpec(ObIAllocator& alloc, const ObPhyOperatorType type)
|
||||
: ObOpSpec(alloc, type),
|
||||
limit_expr_(NULL),
|
||||
offset_expr_(NULL),
|
||||
percent_expr_(NULL),
|
||||
calc_found_rows_(false),
|
||||
is_top_limit_(false),
|
||||
is_fetch_with_ties_(false),
|
||||
sort_columns_(alloc)
|
||||
{}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObLimitSpec, ObOpSpec), limit_expr_, offset_expr_, percent_expr_, calc_found_rows_, is_top_limit_,
|
||||
is_fetch_with_ties_, sort_columns_);
|
||||
|
||||
ObLimitOp::ObLimitOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObOperator(exec_ctx, spec, input),
|
||||
limit_(-1),
|
||||
offset_(0),
|
||||
input_cnt_(0),
|
||||
output_cnt_(0),
|
||||
total_cnt_(0),
|
||||
is_percent_first_(false),
|
||||
pre_sort_columns_(exec_ctx.get_allocator())
|
||||
{}
|
||||
|
||||
int ObLimitOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool is_null_value = false;
|
||||
if (OB_ISNULL(child_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("limit operator has no child", K(ret));
|
||||
} else if (OB_FAIL(get_int_val(MY_SPEC.limit_expr_, eval_ctx_, limit_, is_null_value))) {
|
||||
LOG_WARN("get limit values failed", K(ret));
|
||||
} else if (!is_null_value && OB_FAIL(get_int_val(MY_SPEC.offset_expr_, eval_ctx_, offset_, is_null_value))) {
|
||||
LOG_WARN("get offset values failed", K(ret));
|
||||
} else if (is_null_value) {
|
||||
offset_ = 0;
|
||||
limit_ = 0;
|
||||
} else {
|
||||
is_percent_first_ = NULL != MY_SPEC.percent_expr_;
|
||||
// revise limit, offset because rownum < -1 is rewritten as limit -1
|
||||
// offset 2 rows fetch next -3 rows only --> is meaningless
|
||||
offset_ = offset_ < 0 ? 0 : offset_;
|
||||
if (MY_SPEC.limit_expr_ != NULL) {
|
||||
limit_ = limit_ < 0 ? 0 : limit_;
|
||||
}
|
||||
pre_sort_columns_.reuse_ = true;
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLimitOp::rescan()
|
||||
{
|
||||
input_cnt_ = 0;
|
||||
output_cnt_ = 0;
|
||||
return ObOperator::rescan();
|
||||
}
|
||||
|
||||
int ObLimitOp::get_int_val(ObExpr* expr, ObEvalCtx& eval_ctx, int64_t& val, bool& is_null_value)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (NULL != expr) {
|
||||
OB_ASSERT(ob_is_int_tc(expr->datum_meta_.type_));
|
||||
ObDatum* datum = NULL;
|
||||
if (OB_FAIL(expr->eval(eval_ctx, datum))) {
|
||||
LOG_WARN("expr evaluate failed", K(ret), K(expr));
|
||||
} else if (datum->null_) {
|
||||
is_null_value = true;
|
||||
val = 0;
|
||||
} else {
|
||||
val = *datum->int_;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLimitOp::get_double_val(ObExpr* expr, ObEvalCtx& eval_ctx, double& val)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (NULL != expr) {
|
||||
OB_ASSERT(ob_is_double_tc(expr->datum_meta_.type_));
|
||||
ObDatum* datum = NULL;
|
||||
if (OB_FAIL(expr->eval(eval_ctx, datum))) {
|
||||
LOG_WARN("expr evaluate failed", K(ret), K(expr));
|
||||
} else if (datum->null_) {
|
||||
val = 0.0;
|
||||
} else {
|
||||
val = *datum->double_;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
// copy from ObLimit::inner_get_next_row
|
||||
int ObLimitOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
clear_evaluated_flag();
|
||||
while (OB_SUCC(ret) && input_cnt_ < offset_) {
|
||||
if (OB_FAIL(child_->get_next_row())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("child_op failed to get next row", K(input_cnt_), K(offset_), K(ret));
|
||||
}
|
||||
} else if (is_percent_first_ && OB_FAIL(convert_limit_percent())) {
|
||||
LOG_WARN("failed to convert limit percent", K(ret));
|
||||
} else {
|
||||
++input_cnt_;
|
||||
}
|
||||
} // end while
|
||||
|
||||
/*
|
||||
* 1. is_percent_first_: for 'select * from t1 fetch next 50 percent rows only',
|
||||
* need set lower block operators like sort or agg, and then reset is_percent_first_ to false.
|
||||
* 2. is_fetch_with_ties_: when we get enough rows as limit count, shall we keep fetching for
|
||||
* those rows which equal to the last row of specified order (by order by clause) ?
|
||||
*/
|
||||
int64_t left_count = 0;
|
||||
if (OB_SUCC(ret)) {
|
||||
if (is_percent_first_ || output_cnt_ < limit_ || limit_ < 0) {
|
||||
if (OB_FAIL(child_->get_next_row())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("child_op failed to get next row", K(ret), K_(limit), K_(offset), K_(input_cnt), K_(output_cnt));
|
||||
}
|
||||
} else if (is_percent_first_ && OB_FAIL(convert_limit_percent())) {
|
||||
LOG_WARN("failed to convert limit percent", K(ret));
|
||||
} else if (limit_ == 0) {
|
||||
ret = OB_ITER_END;
|
||||
} else {
|
||||
++output_cnt_;
|
||||
LOG_DEBUG("output row", "row", ROWEXPR2STR(eval_ctx_, MY_SPEC.output_));
|
||||
if (MY_SPEC.is_fetch_with_ties_ && output_cnt_ == limit_ &&
|
||||
OB_FAIL(pre_sort_columns_.save_store_row(MY_SPEC.sort_columns_, eval_ctx_))) {
|
||||
LOG_WARN("failed to deep copy limit last rows", K(ret));
|
||||
}
|
||||
}
|
||||
} else if (limit_ > 0 && MY_SPEC.is_fetch_with_ties_) {
|
||||
bool is_equal = false;
|
||||
if (OB_FAIL(child_->get_next_row())) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("child_op failed to get next row", K(ret), K_(limit), K_(offset), K_(input_cnt), K_(output_cnt));
|
||||
}
|
||||
} else if (OB_FAIL(is_row_order_by_item_value_equal(is_equal))) {
|
||||
LOG_WARN("failed to is row order by item value equal", K(ret));
|
||||
} else if (is_equal) {
|
||||
++output_cnt_;
|
||||
} else {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
} else {
|
||||
ret = OB_ITER_END;
|
||||
if (MY_SPEC.calc_found_rows_) {
|
||||
while (OB_SUCC(child_->get_next_row())) {
|
||||
++left_count;
|
||||
}
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("fail to get next row from child", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_ITER_END == ret) {
|
||||
if (MY_SPEC.is_top_limit_) {
|
||||
total_cnt_ = left_count + output_cnt_ + input_cnt_;
|
||||
ObPhysicalPlanCtx* plan_ctx = NULL;
|
||||
if (OB_ISNULL(plan_ctx = ctx_.get_physical_plan_ctx())) {
|
||||
ret = OB_ERR_NULL_VALUE;
|
||||
LOG_WARN("get physical plan context failed");
|
||||
} else {
|
||||
NG_TRACE_EXT(found_rows, OB_ID(total_count), total_cnt_, OB_ID(input_count), input_cnt_);
|
||||
plan_ctx->set_found_rows(total_cnt_);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLimitOp::is_row_order_by_item_value_equal(bool& is_equal)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (MY_SPEC.sort_columns_.empty()) {
|
||||
// %sort_columns_ is empty if order by const value, set is_equal to true directly.
|
||||
// pre_sort_columns_.store_row_ is NULL here.
|
||||
is_equal = true;
|
||||
} else {
|
||||
is_equal = true;
|
||||
CK(NULL != pre_sort_columns_.store_row_ && pre_sort_columns_.store_row_->cnt_ == MY_SPEC.sort_columns_.count());
|
||||
for (int64_t i = 0; OB_SUCC(ret) && is_equal && i < MY_SPEC.sort_columns_.count(); ++i) {
|
||||
const ObExpr* expr = MY_SPEC.sort_columns_.at(i);
|
||||
ObDatum* datum = NULL;
|
||||
if (OB_FAIL(expr->eval(eval_ctx_, datum))) {
|
||||
LOG_WARN("expression evaluate failed", K(ret));
|
||||
} else {
|
||||
is_equal = 0 == expr->basic_funcs_->null_first_cmp_(pre_sort_columns_.store_row_->cells()[i], *datum);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLimitOp::convert_limit_percent()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
double percent = 0.0;
|
||||
if (OB_FAIL(get_double_val(MY_SPEC.percent_expr_, eval_ctx_, percent))) {
|
||||
LOG_WARN("failed to get double value", K(ret));
|
||||
} else if (percent > 0) {
|
||||
int64_t tot_count = 0;
|
||||
if (OB_UNLIKELY(limit_ != -1) || OB_ISNULL(child_) ||
|
||||
OB_UNLIKELY(child_->get_spec().get_type() != PHY_MATERIAL && child_->get_spec().get_type() != PHY_SORT)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(ret), K(limit_), K(child_));
|
||||
} else if (child_->get_spec().get_type() == PHY_MATERIAL &&
|
||||
OB_FAIL(static_cast<ObMaterialOp*>(child_)->get_material_row_count(tot_count))) {
|
||||
LOG_WARN("failed to get op row count", K(ret));
|
||||
} else if (child_->get_spec().get_type() == PHY_SORT &&
|
||||
FALSE_IT(tot_count = static_cast<ObSortOp*>(child_)->get_sort_row_count())) {
|
||||
LOG_WARN("failed to get op row count", K(ret));
|
||||
} else if (OB_UNLIKELY(tot_count < 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get invalid child op row count", K(tot_count), K(ret));
|
||||
} else if (percent < 100) {
|
||||
int64_t percent_int64 = static_cast<int64_t>(percent);
|
||||
int64_t offset = (tot_count * percent / 100 - tot_count * percent_int64 / 100) > 0 ? 1 : 0;
|
||||
limit_ = tot_count * percent_int64 / 100 + offset;
|
||||
is_percent_first_ = false;
|
||||
} else {
|
||||
limit_ = tot_count;
|
||||
is_percent_first_ = false;
|
||||
}
|
||||
} else {
|
||||
limit_ = 0;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
77
src/sql/engine/basic/ob_limit_op.h
Normal file
77
src/sql/engine/basic/ob_limit_op.h
Normal file
@ -0,0 +1,77 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_LIMIT_OP_H_
|
||||
#define OCEANBASE_BASIC_OB_LIMIT_OP_H_
|
||||
|
||||
#include "sql/engine/ob_operator.h"
|
||||
#include "sql/engine/basic/ob_chunk_datum_store.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObLimitSpec : public ObOpSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObLimitSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
|
||||
INHERIT_TO_STRING_KV("op_spec", ObOpSpec, K_(limit_expr), K_(offset_expr), K_(percent_expr), K_(calc_found_rows),
|
||||
K_(is_top_limit), K_(is_fetch_with_ties), K_(sort_columns));
|
||||
|
||||
ObExpr* limit_expr_;
|
||||
ObExpr* offset_expr_;
|
||||
ObExpr* percent_expr_;
|
||||
// mysql SQL_CALC_FOUND_ROWS query modifier, count found rows after reach limit.
|
||||
bool calc_found_rows_;
|
||||
bool is_top_limit_;
|
||||
bool is_fetch_with_ties_;
|
||||
|
||||
ExprFixedArray sort_columns_;
|
||||
};
|
||||
|
||||
class ObLimitOp : public ObOperator {
|
||||
public:
|
||||
ObLimitOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int rescan() override;
|
||||
|
||||
virtual int inner_get_next_row() override;
|
||||
|
||||
virtual void destroy() override
|
||||
{
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
static int get_int_val(ObExpr* expr, ObEvalCtx& eval_ctx, int64_t& val, bool& is_null_value);
|
||||
static int get_double_val(ObExpr* expr, ObEvalCtx& eval_ctx, double& val);
|
||||
|
||||
private:
|
||||
int convert_limit_percent();
|
||||
int is_row_order_by_item_value_equal(bool& is_equal);
|
||||
|
||||
private:
|
||||
int64_t limit_;
|
||||
int64_t offset_;
|
||||
int64_t input_cnt_;
|
||||
int64_t output_cnt_;
|
||||
int64_t total_cnt_;
|
||||
bool is_percent_first_;
|
||||
|
||||
ObChunkDatumStore::LastStoredRow<> pre_sort_columns_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_BASIC_OB_LIMIT_OP_H_
|
||||
275
src/sql/engine/basic/ob_material.cpp
Normal file
275
src/sql/engine/basic/ob_material.cpp
Normal file
@ -0,0 +1,275 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "sql/engine/basic/ob_material.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
OB_SERIALIZE_MEMBER(ObMaterialInput, bypass_);
|
||||
|
||||
int ObMaterialInput::init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op)
|
||||
{
|
||||
UNUSED(ctx);
|
||||
UNUSED(op);
|
||||
UNUSED(task_info);
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
ObPhyOperatorType ObMaterialInput::get_phy_op_type() const
|
||||
{
|
||||
return PHY_MATERIAL;
|
||||
}
|
||||
|
||||
////////////////////////////////////////////////////////////////////////
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObMaterial, ObSingleChildPhyOperator));
|
||||
|
||||
void ObMaterial::reset()
|
||||
{
|
||||
ObSingleChildPhyOperator::reset();
|
||||
}
|
||||
|
||||
void ObMaterial::reuse()
|
||||
{
|
||||
ObSingleChildPhyOperator::reuse();
|
||||
}
|
||||
|
||||
int ObMaterial::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhyOperatorCtx* op_ctx = NULL;
|
||||
ObSQLSessionInfo* session = NULL;
|
||||
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObMaterialCtx, ctx, get_id(), get_type(), op_ctx))) {
|
||||
SQL_ENG_LOG(WARN, "create physical operator context failed", K(ret));
|
||||
} else if (OB_ISNULL(session = ctx.get_my_session())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_ENG_LOG(WARN, "fail to get my session", K(ret));
|
||||
} else if (OB_FAIL(init_cur_row(*op_ctx, true))) {
|
||||
SQL_ENG_LOG(WARN, "create current row failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMaterial::inner_open(ObExecContext& exec_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMaterialCtx* mat_ctx = NULL;
|
||||
ObSQLSessionInfo* session = NULL;
|
||||
if (OB_FAIL(init_op_ctx(exec_ctx))) {
|
||||
SQL_ENG_LOG(WARN, "failed to init mat ctx", K(ret));
|
||||
} else if (OB_ISNULL(mat_ctx = GET_PHY_OPERATOR_CTX(ObMaterialCtx, exec_ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_ENG_LOG(WARN, "failed to get mat ctx", K(ret));
|
||||
} else if (OB_ISNULL(session = exec_ctx.get_my_session())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_ENG_LOG(WARN, "failed to get my session", K(ret));
|
||||
} else if (OB_ISNULL(child_op_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_ENG_LOG(WARN, "child operator is NULL", K(child_op_), K(ret));
|
||||
} else {
|
||||
mat_ctx->is_first_ = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMaterial::rescan(ObExecContext& exec_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMaterialCtx* mat_ctx = NULL;
|
||||
ObSQLSessionInfo* session = NULL;
|
||||
if (OB_ISNULL(mat_ctx = GET_PHY_OPERATOR_CTX(ObMaterialCtx, exec_ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_ENG_LOG(WARN, "failed to get mat ctx", K(ret));
|
||||
} else if (OB_ISNULL(session = exec_ctx.get_my_session())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_ENG_LOG(WARN, "fail to get my session", K(ret));
|
||||
} else if (exec_ctx.is_restart_plan()) {
|
||||
mat_ctx->row_store_it_.reset();
|
||||
mat_ctx->row_store_.reset();
|
||||
// restart material op
|
||||
if (OB_FAIL(ObPhyOperator::rescan(exec_ctx))) {
|
||||
LOG_WARN("operator rescan failed", K(ret));
|
||||
}
|
||||
mat_ctx->row_id_ = 0;
|
||||
mat_ctx->is_first_ = true;
|
||||
} else {
|
||||
mat_ctx->row_store_it_.reset();
|
||||
mat_ctx->row_id_ = 0;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMaterial::inner_close(ObExecContext& exec_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMaterialCtx* mat_ctx = NULL;
|
||||
if (OB_ISNULL(mat_ctx = GET_PHY_OPERATOR_CTX(ObMaterialCtx, exec_ctx, get_id()))) {
|
||||
LOG_DEBUG("The operator has not been opened.", K(ret), K_(id), "op_type", ob_phy_operator_type_str(get_type()));
|
||||
} else {
|
||||
mat_ctx->row_store_it_.reset();
|
||||
mat_ctx->row_store_.reset();
|
||||
mat_ctx->row_id_ = 0;
|
||||
mat_ctx->sql_mem_processor_.unregister_profile();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMaterial::inner_get_next_row(ObExecContext& exec_ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMaterialCtx* mat_ctx = NULL;
|
||||
if (OB_ISNULL(mat_ctx = GET_PHY_OPERATOR_CTX(ObMaterialCtx, exec_ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_ENG_LOG(WARN, "fail to get physical operator context", K(ret));
|
||||
} else if (OB_FAIL(THIS_WORKER.check_status())) {
|
||||
SQL_ENG_LOG(WARN, "check physical plan status failed", K(ret));
|
||||
} else if (mat_ctx->is_first_ && OB_FAIL(get_all_row_from_child(*mat_ctx, *(exec_ctx.get_my_session())))) {
|
||||
SQL_ENG_LOG(WARN, "failed toget all row from child", K(child_op_), K(ret));
|
||||
} else if (OB_FAIL(mat_ctx->row_store_it_.get_next_row(mat_ctx->get_cur_row()))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get row from row store failed", K(ret), K(mat_ctx->row_id_));
|
||||
} else if (exec_ctx.is_restart_plan()) {
|
||||
mat_ctx->row_store_it_.reset();
|
||||
mat_ctx->row_store_.reset();
|
||||
}
|
||||
} else {
|
||||
mat_ctx->row_id_++;
|
||||
row = &mat_ctx->get_cur_row();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMaterial::get_material_row_count(ObExecContext& exec_ctx, int64_t& material_row_count) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMaterialCtx* material_ctx = NULL;
|
||||
if (OB_ISNULL(material_ctx = GET_PHY_OPERATOR_CTX(ObMaterialCtx, exec_ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
SQL_ENG_LOG(WARN, "get material ctx failed", K(ret));
|
||||
} else {
|
||||
material_row_count = material_ctx->row_store_.get_row_cnt();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMaterial::process_dump(ObMaterialCtx& mat_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool updated = false;
|
||||
bool dumped = false;
|
||||
UNUSED(updated);
|
||||
// as it needs to preserve ordering, it needs to dump all of them except the last piece of memory.
|
||||
// currently it does this from front to end
|
||||
if (OB_FAIL(mat_ctx.sql_mem_processor_.update_max_available_mem_size_periodically(
|
||||
&mat_ctx.mem_context_->get_malloc_allocator(),
|
||||
[&](int64_t cur_cnt) { return mat_ctx.row_store_.get_row_cnt_in_memory() > cur_cnt; },
|
||||
updated))) {
|
||||
LOG_WARN("failed to update max available memory size periodically", K(ret));
|
||||
} else if (mat_ctx.need_dump() && GCONF.is_sql_operator_dump_enabled() &&
|
||||
OB_FAIL(mat_ctx.sql_mem_processor_.extend_max_memory_size(
|
||||
&mat_ctx.mem_context_->get_malloc_allocator(),
|
||||
[&](int64_t max_memory_size) { return mat_ctx.sql_mem_processor_.get_data_size() > max_memory_size; },
|
||||
dumped,
|
||||
mat_ctx.sql_mem_processor_.get_data_size()))) {
|
||||
LOG_WARN("failed to extend max memory size", K(ret));
|
||||
} else if (dumped) {
|
||||
if (OB_FAIL(mat_ctx.row_store_.dump(false, true))) {
|
||||
LOG_WARN("failed to dump row store", K(ret));
|
||||
} else {
|
||||
mat_ctx.sql_mem_processor_.reset();
|
||||
mat_ctx.sql_mem_processor_.set_number_pass(1);
|
||||
LOG_TRACE("trace material dump",
|
||||
K(mat_ctx.sql_mem_processor_.get_data_size()),
|
||||
K(mat_ctx.row_store_.get_row_cnt_in_memory()),
|
||||
K(mat_ctx.sql_mem_processor_.get_mem_bound()));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMaterial::get_all_row_from_child(ObMaterialCtx& mat_ctx, ObSQLSessionInfo& session) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ObNewRow* input_row = NULL;
|
||||
bool first_row = true;
|
||||
int64_t tenant_id = session.get_effective_tenant_id();
|
||||
if (OB_ISNULL(mat_ctx.mem_context_)) {
|
||||
lib::ContextParam param;
|
||||
param.set_mem_attr(tenant_id, ObModIds::OB_SQL_SORT_ROW, ObCtxIds::WORK_AREA)
|
||||
.set_properties(lib::USE_TL_PAGE_OPTIONAL);
|
||||
if (OB_FAIL(CURRENT_CONTEXT.CREATE_CONTEXT(mat_ctx.mem_context_, param))) {
|
||||
LOG_WARN("create entity failed", K(ret));
|
||||
} else if (OB_ISNULL(mat_ctx.mem_context_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("null memory entity returned", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(mat_ctx.row_store_.init(UINT64_MAX, tenant_id, ObCtxIds::WORK_AREA))) {
|
||||
LOG_WARN("init row store failed", K(ret));
|
||||
} else {
|
||||
mat_ctx.row_store_.set_allocator(mat_ctx.mem_context_->get_malloc_allocator());
|
||||
mat_ctx.row_store_.set_callback(&mat_ctx.sql_mem_processor_);
|
||||
}
|
||||
while (OB_SUCCESS == ret && OB_SUCC(child_op_->get_next_row(mat_ctx.exec_ctx_, input_row))) {
|
||||
if (first_row) {
|
||||
int64_t row_count = get_rows();
|
||||
if (OB_FAIL(ObPxEstimateSizeUtil::get_px_size(&mat_ctx.exec_ctx_, px_est_size_factor_, row_count, row_count))) {
|
||||
LOG_WARN("failed to get px size", K(ret));
|
||||
} else if (OB_FAIL(mat_ctx.sql_mem_processor_.init(&mat_ctx.mem_context_->get_malloc_allocator(),
|
||||
tenant_id,
|
||||
row_count * get_width(),
|
||||
get_type(),
|
||||
get_id(),
|
||||
&mat_ctx.exec_ctx_))) {
|
||||
LOG_WARN("failed to init sql memory manager processor", K(ret));
|
||||
} else {
|
||||
mat_ctx.row_store_.set_dir_id(mat_ctx.sql_mem_processor_.get_dir_id());
|
||||
LOG_TRACE("trace init sql mem mgr for material",
|
||||
K(row_count),
|
||||
K(get_width()),
|
||||
K(mat_ctx.profile_.get_cache_size()),
|
||||
K(mat_ctx.profile_.get_expect_size()));
|
||||
}
|
||||
first_row = false;
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(process_dump(mat_ctx))) {
|
||||
LOG_WARN("failed to process dump", K(ret));
|
||||
} else if (OB_FAIL(mat_ctx.row_store_.add_row(*input_row))) {
|
||||
SQL_ENG_LOG(WARN, "failed to add row to row store", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_UNLIKELY(OB_ITER_END != ret)) {
|
||||
SQL_ENG_LOG(WARN, "fail to get next row", K(ret));
|
||||
} else {
|
||||
ret = OB_SUCCESS;
|
||||
// last batch of data got retained in memory
|
||||
if (OB_FAIL(mat_ctx.row_store_.finish_add_row(false))) {
|
||||
SQL_ENG_LOG(WARN, "failed to finish add row to row store", K(ret));
|
||||
} else if (OB_FAIL(mat_ctx.row_store_.begin(mat_ctx.row_store_it_))) {
|
||||
SQL_ENG_LOG(WARN, "failed to begin iterator for chunk row store", K(ret));
|
||||
} else {
|
||||
mat_ctx.is_first_ = false;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
144
src/sql/engine/basic/ob_material.h
Normal file
144
src/sql/engine/basic/ob_material.h
Normal file
@ -0,0 +1,144 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_MATERIAL_H_
|
||||
#define OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_MATERIAL_H_
|
||||
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/engine/ob_single_child_phy_operator.h"
|
||||
#include "sql/engine/basic/ob_ra_row_store.h"
|
||||
#include "sql/engine/basic/ob_chunk_row_store.h"
|
||||
#include "sql/engine/ob_sql_mem_mgr_processor.h"
|
||||
#include "sql/engine/ob_tenant_sql_memory_manager.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
/* only for adapt deserialize between 22x and 31x
|
||||
* from 22x to 31x: only deserialize MaterialInput
|
||||
* from 31x to 22x: no 2+dfo schedule in 22x, just affect performance
|
||||
*/
|
||||
class ObMaterialInput : public ObIPhyOperatorInput {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObMaterialInput() : bypass_(false)
|
||||
{}
|
||||
virtual ~ObMaterialInput() = default;
|
||||
virtual int init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op);
|
||||
virtual ObPhyOperatorType get_phy_op_type() const;
|
||||
virtual void reset() override
|
||||
{
|
||||
bypass_ = false;
|
||||
}
|
||||
void set_bypass(bool bypass)
|
||||
{
|
||||
bypass_ = bypass;
|
||||
}
|
||||
int64_t is_bypass() const
|
||||
{
|
||||
return bypass_;
|
||||
}
|
||||
|
||||
protected:
|
||||
bool bypass_;
|
||||
};
|
||||
|
||||
class ObExecContext;
|
||||
class ObMaterial : public ObSingleChildPhyOperator {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
explicit ObMaterial(common::ObIAllocator& alloc) : ObSingleChildPhyOperator(alloc)
|
||||
{}
|
||||
virtual ~ObMaterial()
|
||||
{}
|
||||
void reset();
|
||||
void reuse();
|
||||
int rescan(ObExecContext& exec_ctx) const;
|
||||
|
||||
int get_material_row_count(ObExecContext& exec_ctx, int64_t& row_count) const;
|
||||
|
||||
private:
|
||||
class ObMaterialCtx : public ObPhyOperatorCtx {
|
||||
friend class ObMaterial;
|
||||
|
||||
public:
|
||||
explicit ObMaterialCtx(ObExecContext& ctx)
|
||||
: ObPhyOperatorCtx(ctx),
|
||||
mem_context_(nullptr),
|
||||
row_id_(0),
|
||||
profile_(ObSqlWorkAreaType::HASH_WORK_AREA),
|
||||
sql_mem_processor_(profile_),
|
||||
is_first_(false)
|
||||
{}
|
||||
virtual ~ObMaterialCtx()
|
||||
{}
|
||||
virtual void destroy()
|
||||
{
|
||||
row_store_.reset();
|
||||
destroy_mem_entiry();
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
|
||||
void destroy_mem_entiry()
|
||||
{
|
||||
if (nullptr != mem_context_) {
|
||||
DESTROY_CONTEXT(mem_context_);
|
||||
mem_context_ = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
bool need_dump()
|
||||
{
|
||||
return sql_mem_processor_.get_data_size() > sql_mem_processor_.get_mem_bound();
|
||||
}
|
||||
|
||||
private:
|
||||
lib::MemoryContext* mem_context_;
|
||||
ObChunkRowStore row_store_;
|
||||
ObChunkRowStore::Iterator row_store_it_;
|
||||
int64_t row_id_;
|
||||
friend class ObValues;
|
||||
ObSqlWorkAreaProfile profile_;
|
||||
ObSqlMemMgrProcessor sql_mem_processor_;
|
||||
bool is_first_;
|
||||
};
|
||||
|
||||
private:
|
||||
virtual int inner_get_next_row(ObExecContext& exec_ctx, const common::ObNewRow*& row) const;
|
||||
/**
|
||||
* @brief open operator, not including children operators.
|
||||
* called by open.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_open(ObExecContext& exec_ctx) const;
|
||||
/**
|
||||
* @brief close operator, not including children operators.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_close(ObExecContext& exec_ctx) const;
|
||||
virtual int init_op_ctx(ObExecContext& ctx) const;
|
||||
|
||||
int get_all_row_from_child(ObMaterialCtx& mat_ctx, ObSQLSessionInfo& session) const;
|
||||
int process_dump(ObMaterialCtx& mat_ctx) const;
|
||||
|
||||
private:
|
||||
// no data member
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObMaterial);
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_MATERIAL_H_ */
|
||||
200
src/sql/engine/basic/ob_material_op.cpp
Normal file
200
src/sql/engine/basic/ob_material_op.cpp
Normal file
@ -0,0 +1,200 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "sql/engine/basic/ob_material_op.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObMaterialSpec, ObOpSpec));
|
||||
|
||||
int ObMaterialOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(child_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("child is null", K(ret));
|
||||
} else {
|
||||
is_first_ = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObMaterialOp::destroy()
|
||||
{
|
||||
datum_store_it_.reset();
|
||||
datum_store_.reset();
|
||||
destroy_mem_context();
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
int ObMaterialOp::process_dump()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool updated = false;
|
||||
bool dumped = false;
|
||||
UNUSED(updated);
|
||||
if (OB_FAIL(sql_mem_processor_.update_max_available_mem_size_periodically(
|
||||
&mem_context_->get_malloc_allocator(),
|
||||
[&](int64_t cur_cnt) { return datum_store_.get_row_cnt_in_memory() > cur_cnt; },
|
||||
updated))) {
|
||||
LOG_WARN("failed to update max available memory size periodically", K(ret));
|
||||
} else if (need_dump() && GCONF.is_sql_operator_dump_enabled() &&
|
||||
OB_FAIL(sql_mem_processor_.extend_max_memory_size(
|
||||
&mem_context_->get_malloc_allocator(),
|
||||
[&](int64_t max_memory_size) { return sql_mem_processor_.get_data_size() > max_memory_size; },
|
||||
dumped,
|
||||
sql_mem_processor_.get_data_size()))) {
|
||||
LOG_WARN("failed to extend max memory size", K(ret));
|
||||
} else if (dumped) {
|
||||
if (OB_FAIL(datum_store_.dump(false, true))) {
|
||||
LOG_WARN("failed to dump row store", K(ret));
|
||||
} else {
|
||||
sql_mem_processor_.reset();
|
||||
sql_mem_processor_.set_number_pass(1);
|
||||
LOG_TRACE("trace material dump",
|
||||
K(sql_mem_processor_.get_data_size()),
|
||||
K(datum_store_.get_row_cnt_in_memory()),
|
||||
K(sql_mem_processor_.get_mem_bound()));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMaterialOp::get_all_row_from_child(ObSQLSessionInfo& session)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool first_row = true;
|
||||
int64_t tenant_id = session.get_effective_tenant_id();
|
||||
if (OB_ISNULL(mem_context_)) {
|
||||
lib::ContextParam param;
|
||||
param.set_mem_attr(tenant_id, ObModIds::OB_SQL_SORT_ROW, ObCtxIds::WORK_AREA)
|
||||
.set_properties(lib::USE_TL_PAGE_OPTIONAL);
|
||||
if (OB_FAIL(CURRENT_CONTEXT.CREATE_CONTEXT(mem_context_, param))) {
|
||||
LOG_WARN("create entity failed", K(ret));
|
||||
} else if (OB_ISNULL(mem_context_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("null memory entity returned", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(datum_store_.init(UINT64_MAX, tenant_id, ObCtxIds::WORK_AREA))) {
|
||||
LOG_WARN("init row store failed", K(ret));
|
||||
} else {
|
||||
datum_store_.set_allocator(mem_context_->get_malloc_allocator());
|
||||
datum_store_.set_callback(&sql_mem_processor_);
|
||||
}
|
||||
while (OB_SUCCESS == ret) {
|
||||
clear_evaluated_flag();
|
||||
if (OB_FAIL(child_->get_next_row())) {
|
||||
} else if (first_row) {
|
||||
int64_t row_count = MY_SPEC.rows_;
|
||||
if (OB_FAIL(ObPxEstimateSizeUtil::get_px_size(&ctx_, MY_SPEC.px_est_size_factor_, row_count, row_count))) {
|
||||
LOG_WARN("failed to get px size", K(ret));
|
||||
} else if (OB_FAIL(sql_mem_processor_.init(&mem_context_->get_malloc_allocator(),
|
||||
tenant_id,
|
||||
row_count * MY_SPEC.width_,
|
||||
MY_SPEC.type_,
|
||||
MY_SPEC.id_,
|
||||
&ctx_))) {
|
||||
LOG_WARN("failed to init sql memory manager processor", K(ret));
|
||||
} else {
|
||||
datum_store_.set_dir_id(sql_mem_processor_.get_dir_id());
|
||||
LOG_TRACE("trace init sql mem mgr for material",
|
||||
K(row_count),
|
||||
K(MY_SPEC.width_),
|
||||
K(profile_.get_cache_size()),
|
||||
K(profile_.get_expect_size()));
|
||||
}
|
||||
first_row = false;
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(process_dump())) {
|
||||
LOG_WARN("failed to process dump", K(ret));
|
||||
} else if (OB_FAIL(datum_store_.add_row(child_->get_spec().output_, &eval_ctx_))) {
|
||||
LOG_WARN("failed to add row to row store", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_UNLIKELY(OB_ITER_END != ret)) {
|
||||
LOG_WARN("fail to get next row", K(ret));
|
||||
} else {
|
||||
ret = OB_SUCCESS;
|
||||
if (OB_FAIL(datum_store_.finish_add_row(false))) {
|
||||
LOG_WARN("failed to finish add row to row store", K(ret));
|
||||
} else if (OB_FAIL(datum_store_.begin(datum_store_it_))) {
|
||||
LOG_WARN("failed to begin iterator for chunk row store", K(ret));
|
||||
} else {
|
||||
is_first_ = false;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMaterialOp::rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSQLSessionInfo* session = ctx_.get_my_session();
|
||||
if (ctx_.is_restart_plan()) {
|
||||
datum_store_it_.reset();
|
||||
datum_store_.reset();
|
||||
// restart material op
|
||||
if (OB_FAIL(ObOperator::rescan())) {
|
||||
LOG_WARN("operator rescan failed", K(ret));
|
||||
}
|
||||
is_first_ = true;
|
||||
} else {
|
||||
datum_store_it_.reset();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMaterialOp::inner_close()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
datum_store_it_.reset();
|
||||
datum_store_.reset();
|
||||
sql_mem_processor_.unregister_profile();
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMaterialOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(try_check_status())) {
|
||||
LOG_WARN("check physical plan status failed", K(ret));
|
||||
} else {
|
||||
clear_evaluated_flag();
|
||||
if (is_first_ && OB_FAIL(get_all_row_from_child(*ctx_.get_my_session()))) {
|
||||
LOG_WARN("failed toget all row from child", K(child_), K(ret));
|
||||
} else if (OB_FAIL(datum_store_it_.get_next_row(child_->get_spec().output_, eval_ctx_))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get row from row store failed", K(ret));
|
||||
} else {
|
||||
if (ctx_.is_restart_plan()) {
|
||||
datum_store_it_.reset();
|
||||
datum_store_.reset();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
84
src/sql/engine/basic/ob_material_op.h
Normal file
84
src/sql/engine/basic/ob_material_op.h
Normal file
@ -0,0 +1,84 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_MATERIAL_OP_H_
|
||||
#define OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_MATERIAL_OP_H_
|
||||
|
||||
#include "sql/engine/ob_operator.h"
|
||||
#include "sql/engine/basic/ob_chunk_datum_store.h"
|
||||
#include "sql/engine/ob_sql_mem_mgr_processor.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObMaterialSpec : public ObOpSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObMaterialSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type) : ObOpSpec(alloc, type)
|
||||
{}
|
||||
};
|
||||
|
||||
class ObMaterialOp : public ObOperator {
|
||||
public:
|
||||
ObMaterialOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObOperator(exec_ctx, spec, input),
|
||||
mem_context_(nullptr),
|
||||
datum_store_(),
|
||||
datum_store_it_(),
|
||||
profile_(ObSqlWorkAreaType::HASH_WORK_AREA),
|
||||
sql_mem_processor_(profile_),
|
||||
is_first_(false)
|
||||
{}
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int rescan() override;
|
||||
virtual int inner_get_next_row() override;
|
||||
virtual int inner_close() override;
|
||||
virtual void destroy() override;
|
||||
|
||||
int get_material_row_count(int64_t& count) const
|
||||
{
|
||||
count = datum_store_.get_row_cnt();
|
||||
return common::OB_SUCCESS;
|
||||
}
|
||||
|
||||
private:
|
||||
int process_dump();
|
||||
int get_all_row_from_child(ObSQLSessionInfo& session);
|
||||
|
||||
bool need_dump()
|
||||
{
|
||||
return sql_mem_processor_.get_data_size() > sql_mem_processor_.get_mem_bound();
|
||||
}
|
||||
void destroy_mem_context()
|
||||
{
|
||||
if (nullptr != mem_context_) {
|
||||
DESTROY_CONTEXT(mem_context_);
|
||||
mem_context_ = nullptr;
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
lib::MemoryContext* mem_context_;
|
||||
ObChunkDatumStore datum_store_;
|
||||
ObChunkDatumStore::Iterator datum_store_it_;
|
||||
friend class ObValues;
|
||||
ObSqlWorkAreaProfile profile_;
|
||||
ObSqlMemMgrProcessor sql_mem_processor_;
|
||||
bool is_first_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_MATERIAL_OP_H_ */
|
||||
177
src/sql/engine/basic/ob_monitoring_dump.cpp
Normal file
177
src/sql/engine/basic/ob_monitoring_dump.cpp
Normal file
@ -0,0 +1,177 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "ob_monitoring_dump.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
using namespace share;
|
||||
namespace sql {
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObMonitoringDump, ObSingleChildPhyOperator), flags_, dst_op_id_);
|
||||
|
||||
class ObMonitoringDump::ObMonitoringDumpCtx : public ObPhyOperatorCtx {
|
||||
public:
|
||||
explicit ObMonitoringDumpCtx(ObExecContext& ctx)
|
||||
: ObPhyOperatorCtx(ctx),
|
||||
op_name_(),
|
||||
tracefile_identifier_(),
|
||||
open_time_(0),
|
||||
close_time_(0),
|
||||
rows_(0),
|
||||
first_row_time_(0),
|
||||
last_row_time_(0),
|
||||
first_row_fetched_(false)
|
||||
{}
|
||||
virtual void destroy()
|
||||
{
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
ObString op_name_;
|
||||
ObString tracefile_identifier_;
|
||||
uint64_t open_time_;
|
||||
uint64_t close_time_;
|
||||
uint64_t rows_;
|
||||
uint64_t first_row_time_;
|
||||
uint64_t last_row_time_;
|
||||
bool first_row_fetched_;
|
||||
|
||||
private:
|
||||
friend class ObMonitoringDump;
|
||||
};
|
||||
|
||||
int ObMonitoringDump::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhyOperatorCtx* op_ctx = nullptr;
|
||||
ObSQLSessionInfo* my_session = nullptr;
|
||||
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObMonitoringDumpCtx, ctx, get_id(), get_type(), op_ctx))) {
|
||||
LOG_WARN("Create physical operator context failed", K(ret));
|
||||
} else if (OB_ISNULL(op_ctx) || OB_ISNULL(child_op_) || OB_ISNULL(my_session = GET_MY_SESSION(ctx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("Op_ctx or child_op_ is null", K(ret));
|
||||
} else if (OB_FAIL(op_ctx->create_cur_row(get_column_count(), projector_, projector_size_))) {
|
||||
LOG_WARN("create current row failed", K(ret), "#columns", get_column_count());
|
||||
} else {
|
||||
ObObj val;
|
||||
ObMonitoringDumpCtx* md_ctx = static_cast<ObMonitoringDumpCtx*>(op_ctx);
|
||||
const char* name = get_phy_op_name(child_op_->get_type());
|
||||
md_ctx->op_name_.assign_ptr(name, strlen(name));
|
||||
if (OB_FAIL(my_session->get_sys_variable(SYS_VAR_TRACEFILE_IDENTIFIER, val))) {
|
||||
LOG_WARN("Get sys variable error", K(ret));
|
||||
} else if (OB_FAIL(val.get_string(md_ctx->tracefile_identifier_))) {
|
||||
LOG_WARN("Failed to get tracefile identifier", K(ret));
|
||||
} else {
|
||||
md_ctx->open_time_ = ObTimeUtility::current_time();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMonitoringDump::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("Failed to init operator context", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMonitoringDump::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMonitoringDumpCtx* md_ctx = nullptr;
|
||||
if (OB_ISNULL(md_ctx = GET_PHY_OPERATOR_CTX(ObMonitoringDumpCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K(ret), K_(id));
|
||||
} else if (flags_ & OB_MONITOR_STAT) {
|
||||
const ObString& OPERATOR_NAME = md_ctx->op_name_;
|
||||
const ObString& TRACEFILE_IDENTIFIER = md_ctx->tracefile_identifier_;
|
||||
uint64_t ROWS_COUNT = md_ctx->rows_;
|
||||
uint64_t CLOSE_TIME = ObTimeUtility::current_time();
|
||||
uint64_t OPEN_TIME = md_ctx->open_time_;
|
||||
uint64_t FIRST_ROW_TIME = md_ctx->first_row_time_;
|
||||
uint64_t LAST_ROW_TIME = md_ctx->last_row_time_;
|
||||
const uint64_t OP_ID = dst_op_id_;
|
||||
if (TRACEFILE_IDENTIFIER.length() > 0) {
|
||||
LOG_INFO("",
|
||||
K(TRACEFILE_IDENTIFIER),
|
||||
K(OPERATOR_NAME),
|
||||
K(ROWS_COUNT),
|
||||
K(OPEN_TIME),
|
||||
K(CLOSE_TIME),
|
||||
K(FIRST_ROW_TIME),
|
||||
K(LAST_ROW_TIME),
|
||||
K(OP_ID));
|
||||
} else {
|
||||
LOG_INFO("",
|
||||
K(OPERATOR_NAME),
|
||||
K(ROWS_COUNT),
|
||||
K(OPEN_TIME),
|
||||
K(CLOSE_TIME),
|
||||
K(FIRST_ROW_TIME),
|
||||
K(LAST_ROW_TIME),
|
||||
K(OP_ID));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMonitoringDump::inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObMonitoringDumpCtx* md_ctx = nullptr;
|
||||
const uint64_t OP_ID = dst_op_id_;
|
||||
const common::ObNewRow* tmp_row;
|
||||
if (OB_ISNULL(md_ctx = GET_PHY_OPERATOR_CTX(ObMonitoringDumpCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator context failed", K(ret), K_(id));
|
||||
} else if (OB_FAIL(child_op_->get_next_row(ctx, tmp_row))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
LOG_DEBUG("OB_ITER_END");
|
||||
md_ctx->last_row_time_ = ObTimeUtility::current_time();
|
||||
} else {
|
||||
LOG_WARN("Failed to get next row", K(ret));
|
||||
}
|
||||
} else {
|
||||
md_ctx->cur_row_.assign(tmp_row->cells_, tmp_row->count_);
|
||||
row = &md_ctx->cur_row_;
|
||||
const ObString& OPERATOR_NAME = md_ctx->op_name_;
|
||||
const ObString& TRACEFILE_IDENTIFIER = md_ctx->tracefile_identifier_;
|
||||
md_ctx->rows_++;
|
||||
if (flags_ & OB_MONITOR_TRACING) {
|
||||
if (TRACEFILE_IDENTIFIER.length() > 0) {
|
||||
LOG_INFO("", K(TRACEFILE_IDENTIFIER), K(OPERATOR_NAME), KPC(row), K(OP_ID));
|
||||
} else {
|
||||
LOG_INFO("", K(OPERATOR_NAME), KPC(row), K(OP_ID));
|
||||
}
|
||||
}
|
||||
if (!md_ctx->first_row_fetched_) {
|
||||
md_ctx->first_row_fetched_ = true;
|
||||
md_ctx->first_row_time_ = ObTimeUtility::current_time();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObMonitoringDump::to_string_kv(char* buf, const int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
UNUSED(buf);
|
||||
UNUSED(buf_len);
|
||||
return pos;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
78
src/sql/engine/basic/ob_monitoring_dump.h
Normal file
78
src/sql/engine/basic/ob_monitoring_dump.h
Normal file
@ -0,0 +1,78 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OB_MONITORING_DUMP_H_
|
||||
#define OB_MONITORING_DUMP_H_
|
||||
|
||||
#include "sql/engine/ob_phy_operator.h"
|
||||
#include "sql/engine/ob_single_child_phy_operator.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
|
||||
namespace sql {
|
||||
|
||||
class ObMonitoringDump : public ObSingleChildPhyOperator {
|
||||
OB_UNIS_VERSION(1);
|
||||
class ObMonitoringDumpCtx;
|
||||
|
||||
public:
|
||||
explicit ObMonitoringDump(common::ObIAllocator& alloc) : ObSingleChildPhyOperator(alloc), flags_(0), dst_op_id_(-1)
|
||||
{}
|
||||
virtual ~ObMonitoringDump() = default;
|
||||
|
||||
void set_flags(uint64_t flags)
|
||||
{
|
||||
flags_ = flags;
|
||||
}
|
||||
void set_dst_op_id(uint64_t dst_op_id)
|
||||
{
|
||||
dst_op_id_ = dst_op_id;
|
||||
}
|
||||
|
||||
private:
|
||||
/**
|
||||
* @brief init operator context, will create a physical operator context (and a current row space)
|
||||
* @param ctx[in], execute context
|
||||
* @return if success, return OB_SUCCESS, otherwise, return errno
|
||||
*/
|
||||
virtual int init_op_ctx(ObExecContext& ctx) const override;
|
||||
/**
|
||||
* @brief open operator, not including children operators.
|
||||
* called by open.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_open(ObExecContext& ctx) const override;
|
||||
/**
|
||||
* @brief close operator, not including children operators.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_close(ObExecContext& ctx) const override;
|
||||
/**
|
||||
* @brief called by get_next_row(), get a row from the child operator or row_store
|
||||
* @param ctx[in], execute context
|
||||
* @param row[out], ObSqlRow an obj array and row_size
|
||||
*/
|
||||
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const override;
|
||||
|
||||
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
|
||||
|
||||
private:
|
||||
uint64_t flags_;
|
||||
uint64_t dst_op_id_;
|
||||
DISALLOW_COPY_AND_ASSIGN(ObMonitoringDump);
|
||||
};
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
|
||||
#endif
|
||||
133
src/sql/engine/basic/ob_monitoring_dump_op.cpp
Normal file
133
src/sql/engine/basic/ob_monitoring_dump_op.cpp
Normal file
@ -0,0 +1,133 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "ob_monitoring_dump_op.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
using namespace share;
|
||||
namespace sql {
|
||||
|
||||
ObMonitoringDumpSpec::ObMonitoringDumpSpec(ObIAllocator& alloc, const ObPhyOperatorType type)
|
||||
: ObOpSpec(alloc, type), flags_(0), dst_op_id_(-1)
|
||||
{}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObMonitoringDumpSpec, ObOpSpec), flags_, dst_op_id_);
|
||||
|
||||
ObMonitoringDumpOp::ObMonitoringDumpOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObOperator(exec_ctx, spec, input),
|
||||
op_name_(),
|
||||
tracefile_identifier_(),
|
||||
open_time_(0),
|
||||
rows_(0),
|
||||
first_row_time_(0),
|
||||
last_row_time_(0),
|
||||
first_row_fetched_(false)
|
||||
{}
|
||||
|
||||
int ObMonitoringDumpOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(child_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected status: child is null", K(ret));
|
||||
} else {
|
||||
ObObj val;
|
||||
const char* name = get_phy_op_name(spec_.get_left()->type_);
|
||||
op_name_.set_string(name, strlen(name));
|
||||
if (OB_FAIL(ctx_.get_my_session()->get_sys_variable(SYS_VAR_TRACEFILE_IDENTIFIER, val))) {
|
||||
LOG_WARN("Get sys variable error", K(ret));
|
||||
} else if (OB_FAIL(tracefile_identifier_.from_obj(val))) {
|
||||
LOG_WARN("failed to convert datum from obj", K(ret));
|
||||
} else {
|
||||
open_time_ = ObTimeUtility::current_time();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMonitoringDumpOp::inner_close()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (MY_SPEC.flags_ & OB_MONITOR_STAT) {
|
||||
uint64_t CLOSE_TIME = ObTimeUtility::current_time();
|
||||
if (!tracefile_identifier_.null_ && tracefile_identifier_.len_ > 0) {
|
||||
LOG_INFO("",
|
||||
K(tracefile_identifier_),
|
||||
K(op_name_),
|
||||
K(rows_),
|
||||
K(open_time_),
|
||||
K(CLOSE_TIME),
|
||||
K(first_row_time_),
|
||||
K(last_row_time_),
|
||||
K(MY_SPEC.dst_op_id_));
|
||||
} else {
|
||||
LOG_INFO("",
|
||||
K(op_name_),
|
||||
K(rows_),
|
||||
K(open_time_),
|
||||
K(CLOSE_TIME),
|
||||
K(first_row_time_),
|
||||
K(last_row_time_),
|
||||
K(MY_SPEC.dst_op_id_));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMonitoringDumpOp::rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
first_row_fetched_ = false;
|
||||
if (OB_FAIL(ObOperator::rescan())) {
|
||||
LOG_WARN("failed to rescan", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObMonitoringDumpOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
clear_evaluated_flag();
|
||||
if (OB_FAIL(child_->get_next_row())) {
|
||||
if (OB_ITER_END == ret) {
|
||||
LOG_DEBUG("OB_ITER_END");
|
||||
last_row_time_ = ObTimeUtility::current_time();
|
||||
} else {
|
||||
LOG_WARN("Failed to get next row", K(ret));
|
||||
}
|
||||
} else {
|
||||
rows_++;
|
||||
if (MY_SPEC.flags_ & OB_MONITOR_TRACING) {
|
||||
if (!tracefile_identifier_.null_ && tracefile_identifier_.len_ > 0) {
|
||||
LOG_INFO("",
|
||||
K(tracefile_identifier_.get_string()),
|
||||
K(op_name_.get_string()),
|
||||
K(ObToStringExprRow(eval_ctx_, MY_SPEC.output_)),
|
||||
K(MY_SPEC.dst_op_id_));
|
||||
} else {
|
||||
LOG_INFO("", K(op_name_.get_string()), K(ObToStringExprRow(eval_ctx_, MY_SPEC.output_)), K(MY_SPEC.dst_op_id_));
|
||||
}
|
||||
}
|
||||
if (!first_row_fetched_) {
|
||||
first_row_fetched_ = true;
|
||||
first_row_time_ = ObTimeUtility::current_time();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
60
src/sql/engine/basic/ob_monitoring_dump_op.h
Normal file
60
src/sql/engine/basic/ob_monitoring_dump_op.h
Normal file
@ -0,0 +1,60 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef SRC_SQL_ENGINE_BASIC_OB_MONITORING_DUMP_OP_H_
|
||||
#define SRC_SQL_ENGINE_BASIC_OB_MONITORING_DUMP_OP_H_
|
||||
|
||||
#include "sql/engine/ob_operator.h"
|
||||
#include "share/datum/ob_datum.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObMonitoringDumpSpec : public ObOpSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObMonitoringDumpSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
|
||||
INHERIT_TO_STRING_KV("op_spec", ObOpSpec, K_(flags), K_(dst_op_id));
|
||||
|
||||
uint64_t flags_;
|
||||
uint64_t dst_op_id_;
|
||||
};
|
||||
|
||||
class ObMonitoringDumpOp : public ObOperator {
|
||||
public:
|
||||
ObMonitoringDumpOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int inner_close() override;
|
||||
virtual int rescan() override;
|
||||
virtual int inner_get_next_row() override;
|
||||
virtual void destroy() override
|
||||
{
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
private:
|
||||
common::ObDatum op_name_;
|
||||
common::ObDatum tracefile_identifier_;
|
||||
uint64_t open_time_;
|
||||
uint64_t rows_;
|
||||
uint64_t first_row_time_;
|
||||
uint64_t last_row_time_;
|
||||
bool first_row_fetched_;
|
||||
};
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
|
||||
#endif /* SRC_SQL_ENGINE_BASIC_OB_MONITORING_DUMP_OP_H_ */
|
||||
812
src/sql/engine/basic/ob_pushdown_filter.cpp
Normal file
812
src/sql/engine/basic/ob_pushdown_filter.cpp
Normal file
@ -0,0 +1,812 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "ob_pushdown_filter.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/resolver/expr/ob_raw_expr_util.h"
|
||||
#include "sql/code_generator/ob_static_engine_cg.h"
|
||||
#include "storage/ob_i_store.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
using namespace share;
|
||||
namespace sql {
|
||||
|
||||
ObPushdownFilterFactory::PDFilterAllocFunc
|
||||
ObPushdownFilterFactory::PD_FILTER_ALLOC[PushdownFilterType::MAX_FILTER_TYPE] = {
|
||||
ObPushdownFilterFactory::alloc<ObPushdownBlackFilterNode, BLACK_FILTER>,
|
||||
ObPushdownFilterFactory::alloc<ObPushdownWhiteFilterNode, WHITE_FILTER>,
|
||||
ObPushdownFilterFactory::alloc<ObPushdownAndFilterNode, AND_FILTER>,
|
||||
ObPushdownFilterFactory::alloc<ObPushdownOrFilterNode, OR_FILTER>};
|
||||
|
||||
ObPushdownFilterFactory::FilterExecutorAllocFunc
|
||||
ObPushdownFilterFactory::FILTER_EXECUTOR_ALLOC[FilterExecutorType::MAX_EXECUTOR_TYPE] = {
|
||||
ObPushdownFilterFactory::alloc<ObBlackFilterExecutor, ObPushdownBlackFilterNode, BLACK_FILTER_EXECUTOR>,
|
||||
ObPushdownFilterFactory::alloc<ObWhiteFilterExecutor, ObPushdownWhiteFilterNode, WHITE_FILTER_EXECUTOR>,
|
||||
ObPushdownFilterFactory::alloc<ObAndFilterExecutor, ObPushdownAndFilterNode, AND_FILTER_EXECUTOR>,
|
||||
ObPushdownFilterFactory::alloc<ObOrFilterExecutor, ObPushdownOrFilterNode, OR_FILTER_EXECUTOR>};
|
||||
|
||||
OB_SERIALIZE_MEMBER(ObPushdownFilterNode, type_, n_child_, col_ids_);
|
||||
OB_SERIALIZE_MEMBER((ObPushdownAndFilterNode, ObPushdownFilterNode));
|
||||
OB_SERIALIZE_MEMBER((ObPushdownOrFilterNode, ObPushdownFilterNode));
|
||||
OB_SERIALIZE_MEMBER((ObPushdownBlackFilterNode, ObPushdownFilterNode), column_exprs_, filter_exprs_);
|
||||
OB_SERIALIZE_MEMBER((ObPushdownWhiteFilterNode, ObPushdownFilterNode));
|
||||
|
||||
int ObPushdownBlackFilterNode::merge(ObIArray<ObPushdownFilterNode*>& merged_node)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(filter_exprs_.init(1 + merged_node.count()))) {
|
||||
LOG_WARN("failed to init exprs", K(ret));
|
||||
} else if (OB_FAIL(filter_exprs_.push_back(tmp_expr_))) {
|
||||
LOG_WARN("failed to push back expr", K(ret));
|
||||
} else {
|
||||
for (int64_t i = 0; i < merged_node.count() && OB_SUCC(ret); ++i) {
|
||||
ObPushdownBlackFilterNode* black_node = static_cast<ObPushdownBlackFilterNode*>(merged_node.at(i));
|
||||
if (OB_ISNULL(black_node->tmp_expr_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected status: exprs must be only one", K(ret));
|
||||
} else {
|
||||
if (OB_FAIL(filter_exprs_.push_back(black_node->tmp_expr_))) {
|
||||
LOG_WARN("failed to push back expr", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPushdownBlackFilterNode::postprocess()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (0 == filter_exprs_.count()) {
|
||||
OZ(filter_exprs_.init(1));
|
||||
OZ(filter_exprs_.push_back(tmp_expr_));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPushdownFilterConstructor::create_black_filter_node(ObRawExpr* raw_expr, ObPushdownFilterNode*& filter_node)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObExpr* expr = nullptr;
|
||||
ObArray<ObRawExpr*> column_exprs;
|
||||
ObPushdownBlackFilterNode* black_filter_node = nullptr;
|
||||
if (OB_ISNULL(raw_expr)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("Invalid null raw expr", K(ret));
|
||||
} else if (OB_FAIL(static_cg_.generate_rt_expr(*raw_expr, expr))) {
|
||||
LOG_WARN("failed to generate rt expr", K(ret));
|
||||
} else if (OB_FAIL(ObRawExprUtils::extract_column_exprs(raw_expr, column_exprs))) {
|
||||
LOG_WARN("failed to extract column exprs", K(ret));
|
||||
} else if (OB_FAIL(factory_.alloc(PushdownFilterType::BLACK_FILTER, 0, filter_node))) {
|
||||
LOG_WARN("failed t o alloc pushdown filter", K(ret));
|
||||
} else if (OB_ISNULL(filter_node)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("black filter node is null", K(ret));
|
||||
} else if (FALSE_IT(black_filter_node = static_cast<ObPushdownBlackFilterNode*>(filter_node))) {
|
||||
} else if (0 < column_exprs.count()) {
|
||||
OZ(black_filter_node->col_ids_.init(column_exprs.count()));
|
||||
OZ(black_filter_node->column_exprs_.init(column_exprs.count()));
|
||||
for (int64_t i = 0; i < column_exprs.count() && OB_SUCC(ret); ++i) {
|
||||
ObRawExpr* sub_raw_expr = column_exprs.at(i);
|
||||
ObExpr* sub_expr = nullptr;
|
||||
ObColumnRefRawExpr* ref_expr = static_cast<ObColumnRefRawExpr*>(sub_raw_expr);
|
||||
if (OB_FAIL(static_cg_.generate_rt_expr(*sub_raw_expr, sub_expr))) {
|
||||
LOG_WARN("failed to generate rt expr", K(ret));
|
||||
} else if (OB_FAIL(black_filter_node->col_ids_.push_back(ref_expr->get_column_id()))) {
|
||||
LOG_WARN("failed to push back column id", K(ret));
|
||||
} else if (OB_FAIL(black_filter_node->column_exprs_.push_back(sub_expr))) {
|
||||
LOG_WARN("failed to push back column expr", K(ret));
|
||||
}
|
||||
}
|
||||
} else {
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
black_filter_node->tmp_expr_ = expr;
|
||||
LOG_DEBUG("debug black_filter_node", K(*raw_expr), K(*expr), K(black_filter_node->col_ids_));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPushdownFilterConstructor::merge_filter_node(
|
||||
ObPushdownFilterNode* dst, ObPushdownFilterNode* other, ObIArray<ObPushdownFilterNode*>& merged_node, bool& merged)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
merged = false;
|
||||
if (OB_ISNULL(other) || OB_ISNULL(dst) || dst == other) {
|
||||
} else if (dst->get_type() == other->get_type()) {
|
||||
if (dst->get_type() == PushdownFilterType::BLACK_FILTER) {
|
||||
if (is_array_equal(dst->get_col_ids(), other->get_col_ids())) {
|
||||
merged = true;
|
||||
LOG_DEBUG("merged filter", K(dst->get_col_ids()));
|
||||
if (OB_FAIL(merged_node.push_back(other))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
} else {
|
||||
LOG_DEBUG("unmerged filter", K(other->get_col_ids()));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPushdownFilterConstructor::deduplicate_filter_node(
|
||||
ObIArray<ObPushdownFilterNode*>& filter_nodes, uint32_t& n_node)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
for (int64_t i = 0; i < filter_nodes.count() - 1 && OB_SUCC(ret); ++i) {
|
||||
ObArray<ObPushdownFilterNode*> merged_node;
|
||||
for (int64_t j = i + 1; j < filter_nodes.count() && OB_SUCC(ret); ++j) {
|
||||
bool merged = false;
|
||||
if (OB_FAIL(merge_filter_node(filter_nodes.at(i), filter_nodes.at(j), merged_node, merged))) {
|
||||
LOG_WARN("failed to merge filter node", K(ret));
|
||||
} else if (merged) {
|
||||
filter_nodes.at(j) = nullptr;
|
||||
--n_node;
|
||||
}
|
||||
}
|
||||
if (0 < merged_node.count()) {
|
||||
if (OB_ISNULL(filter_nodes.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("filter node is null", K(ret), K(i));
|
||||
} else if (OB_FAIL(filter_nodes.at(i)->merge(merged_node))) {
|
||||
LOG_WARN("failed to merge filter node", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPushdownFilterConstructor::apply(ObRawExpr* raw_expr, ObPushdownFilterNode*& filter_tree)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPushdownFilterNode* filter_node = nullptr;
|
||||
if (OB_ISNULL(raw_expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("raw expr is null", K(ret));
|
||||
} else if (can_split_or(raw_expr)) {
|
||||
// flatten or expr and create OR filter
|
||||
// if (OB_FAIL(factory_.alloc(PushdownFilterType::OR, filter_node))) {
|
||||
// LOG_WARN("failed t o alloc pushdown filter", K(ret));
|
||||
// }
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
LOG_WARN("not supported", K(ret));
|
||||
} else if (is_white_mode(raw_expr)) {
|
||||
// create white filter
|
||||
// if (OB_FAIL(factory_.alloc(PushdownFilterType::WHITE, filter_node))) {
|
||||
// LOG_WARN("failed t o alloc pushdown filter", K(ret));
|
||||
// }
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
LOG_WARN("not supported", K(ret));
|
||||
} else {
|
||||
if (OB_FAIL(create_black_filter_node(raw_expr, filter_node))) {
|
||||
LOG_WARN("failed t o alloc pushdown filter", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
filter_tree = filter_node;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPushdownFilterConstructor::apply(ObIArray<ObRawExpr*>& exprs, ObPushdownFilterNode*& filter_tree)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(alloc_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected status: alloc is null or expr is null", K(ret), KP(alloc_));
|
||||
} else if (0 < exprs.count()) {
|
||||
if (1 == exprs.count()) {
|
||||
ObPushdownFilterNode* filter_node = nullptr;
|
||||
if (OB_FAIL(apply(exprs.at(0), filter_node))) {
|
||||
LOG_WARN("failed to apply expr", K(ret));
|
||||
} else {
|
||||
filter_tree = filter_node;
|
||||
OZ(filter_node->postprocess());
|
||||
}
|
||||
} else {
|
||||
ObArray<ObPushdownFilterNode*> tmp_filter_nodes;
|
||||
for (int64_t i = 0; i < exprs.count() && OB_SUCC(ret); ++i) {
|
||||
ObRawExpr* raw_expr = exprs.at(i);
|
||||
ObPushdownFilterNode* sub_filter_node = nullptr;
|
||||
if (OB_ISNULL(raw_expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("raw expr is null", K(ret));
|
||||
} else if (OB_FAIL(apply(raw_expr, sub_filter_node))) {
|
||||
LOG_WARN("failed to apply raw expr", K(ret));
|
||||
} else if (OB_FAIL(tmp_filter_nodes.push_back(sub_filter_node))) {
|
||||
LOG_WARN("failed to push back filter node", K(ret));
|
||||
}
|
||||
}
|
||||
ObPushdownFilterNode* filter_node = nullptr;
|
||||
uint32_t n_valid_node = (uint32_t)tmp_filter_nodes.count();
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(deduplicate_filter_node(tmp_filter_nodes, n_valid_node))) {
|
||||
LOG_WARN("failed to deduplicate filter node", K(ret));
|
||||
} else if (1 != n_valid_node) {
|
||||
if (OB_FAIL(factory_.alloc(PushdownFilterType::AND_FILTER, n_valid_node, filter_node))) {
|
||||
LOG_WARN("failed t o alloc pushdown filter", K(ret));
|
||||
} else {
|
||||
uint32_t n_node = 0;
|
||||
for (int64_t i = 0; i < tmp_filter_nodes.count() && OB_SUCC(ret); ++i) {
|
||||
if (nullptr != tmp_filter_nodes.at(i)) {
|
||||
if (n_node >= n_valid_node) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpeced status: valid node count", K(ret), K(n_node), K(n_valid_node));
|
||||
} else {
|
||||
OZ(tmp_filter_nodes.at(i)->postprocess());
|
||||
filter_node->childs_[n_node] = tmp_filter_nodes.at(i);
|
||||
++n_node;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
filter_node = tmp_filter_nodes.at(0);
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_ISNULL(filter_node)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("filter node is null", K(ret));
|
||||
} else {
|
||||
filter_tree = filter_node;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPushdownFilterFactory::alloc(PushdownFilterType type, uint32_t n_child, ObPushdownFilterNode*& pd_filter)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (!(BLACK_FILTER <= type && type < MAX_FILTER_TYPE)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
OB_LOG(WARN, "invalid filter type", K(ret), K(type));
|
||||
} else if (OB_ISNULL(alloc_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
OB_LOG(WARN, "Unexpected null allocator", K(ret));
|
||||
} else if (OB_ISNULL(ObPushdownFilterFactory::PD_FILTER_ALLOC[type])) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
OB_LOG(WARN, "invalid filter type to alloc", K(ret), K(type));
|
||||
} else if (OB_FAIL(ObPushdownFilterFactory::PD_FILTER_ALLOC[type](*alloc_, n_child, pd_filter))) {
|
||||
OB_LOG(WARN, "fail to alloc pushdown filter", K(ret), K(type));
|
||||
} else {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ClassT, PushdownFilterType type>
|
||||
int ObPushdownFilterFactory::alloc(common::ObIAllocator& alloc, uint32_t n_child, ObPushdownFilterNode*& pd_filter)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
void* buf = NULL;
|
||||
if (OB_ISNULL(buf = alloc.alloc(sizeof(ClassT)))) {
|
||||
ret = common::OB_ALLOCATE_MEMORY_FAILED;
|
||||
OB_LOG(ERROR, "fail to alloc pushdown filter", K(ret));
|
||||
} else {
|
||||
pd_filter = new (buf) ClassT(alloc);
|
||||
if (0 < n_child) {
|
||||
void* tmp_buf = alloc.alloc(n_child * sizeof(ObPushdownFilterNode*));
|
||||
if (OB_ISNULL(tmp_buf)) {
|
||||
ret = common::OB_ALLOCATE_MEMORY_FAILED;
|
||||
OB_LOG(ERROR, "fail to alloc pushdown filter", K(ret));
|
||||
} else {
|
||||
pd_filter->childs_ = reinterpret_cast<ObPushdownFilterNode**>(tmp_buf);
|
||||
}
|
||||
pd_filter->n_child_ = n_child;
|
||||
} else {
|
||||
pd_filter->childs_ = nullptr;
|
||||
}
|
||||
pd_filter->set_type(type);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPushdownFilterFactory::alloc(FilterExecutorType type, uint32_t n_child, ObPushdownFilterNode& filter_node,
|
||||
ObPushdownFilterExecutor*& filter_executor)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (!(BLACK_FILTER_EXECUTOR <= type && type < MAX_EXECUTOR_TYPE)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
OB_LOG(WARN, "invalid filter type", K(ret), K(type));
|
||||
} else if (OB_ISNULL(alloc_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
OB_LOG(WARN, "Unexpected null allocator", K(ret));
|
||||
} else if (OB_ISNULL(ObPushdownFilterFactory::FILTER_EXECUTOR_ALLOC[type])) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
OB_LOG(WARN, "invalid filter type to alloc", K(ret), K(type));
|
||||
} else if (OB_FAIL(ObPushdownFilterFactory::FILTER_EXECUTOR_ALLOC[type](
|
||||
*alloc_, n_child, filter_node, filter_executor))) {
|
||||
OB_LOG(WARN, "fail to alloc pushdown filter", K(ret), K(type));
|
||||
} else {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
template <typename ClassT, typename FilterNodeT, FilterExecutorType type>
|
||||
int ObPushdownFilterFactory::alloc(common::ObIAllocator& alloc, uint32_t n_child, ObPushdownFilterNode& filter_node,
|
||||
ObPushdownFilterExecutor*& filter_executor)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
void* buf = NULL;
|
||||
if (OB_ISNULL(buf = alloc.alloc(sizeof(ClassT)))) {
|
||||
ret = common::OB_ALLOCATE_MEMORY_FAILED;
|
||||
OB_LOG(ERROR, "fail to alloc pushdown filter", K(ret), K(sizeof(ClassT)));
|
||||
} else {
|
||||
filter_executor = new (buf) ClassT(alloc, *static_cast<FilterNodeT*>(&filter_node));
|
||||
if (0 < n_child) {
|
||||
void* tmp_buf = alloc.alloc(n_child * sizeof(ObPushdownFilterExecutor*));
|
||||
if (OB_ISNULL(tmp_buf)) {
|
||||
ret = common::OB_ALLOCATE_MEMORY_FAILED;
|
||||
OB_LOG(ERROR, "fail to alloc pushdown filter", K(ret));
|
||||
}
|
||||
filter_executor->set_childs(n_child, reinterpret_cast<ObPushdownFilterExecutor**>(tmp_buf));
|
||||
}
|
||||
filter_executor->set_type(type);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPushdownFilter::serialize_pushdown_filter(
|
||||
char* buf, int64_t buf_len, int64_t& pos, ObPushdownFilterNode* pd_storage_filter)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(pd_storage_filter)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("pd filter is null", K(ret));
|
||||
} else if (OB_FAIL(serialization::encode_vi32(buf, buf_len, pos, pd_storage_filter->type_))) {
|
||||
LOG_WARN("fail to encode op type", K(ret));
|
||||
} else if (OB_FAIL(serialization::encode(buf, buf_len, pos, pd_storage_filter->n_child_))) {
|
||||
LOG_WARN("fail to encode op type", K(ret));
|
||||
} else if (OB_FAIL(pd_storage_filter->serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("fail to encode", K(ret));
|
||||
} else {
|
||||
for (int64_t i = 0; i < pd_storage_filter->n_child_ && OB_SUCC(ret); ++i) {
|
||||
if (OB_FAIL(serialize_pushdown_filter(buf, buf_len, pos, pd_storage_filter->childs_[i]))) {
|
||||
LOG_WARN("failed to serialize pushdown storage filter", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPushdownFilter::deserialize_pushdown_filter(ObPushdownFilterFactory filter_factory, const char* buf,
|
||||
int64_t data_len, int64_t& pos, ObPushdownFilterNode*& pd_storage_filter)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int32_t filter_type;
|
||||
uint32_t child_cnt = 0;
|
||||
if (OB_FAIL(serialization::decode_vi32(buf, data_len, pos, &filter_type))) {
|
||||
LOG_WARN("fail to decode phy operator type", K(ret));
|
||||
} else if (OB_FAIL(serialization::decode(buf, data_len, pos, child_cnt))) {
|
||||
LOG_WARN("fail to encode op type", K(ret));
|
||||
} else {
|
||||
if (OB_FAIL(filter_factory.alloc(static_cast<PushdownFilterType>(filter_type), child_cnt, pd_storage_filter))) {
|
||||
LOG_WARN("failed to allocate filter", K(ret));
|
||||
} else if (OB_FAIL(pd_storage_filter->deserialize(buf, data_len, pos))) {
|
||||
LOG_WARN("failed to deserialize", K(ret));
|
||||
} else if (0 < child_cnt) {
|
||||
for (uint32_t i = 0; i < child_cnt && OB_SUCC(ret); ++i) {
|
||||
ObPushdownFilterNode* sub_pd_storage_filter = nullptr;
|
||||
if (OB_FAIL(deserialize_pushdown_filter(filter_factory, buf, data_len, pos, sub_pd_storage_filter))) {
|
||||
LOG_WARN("failed to deserialize child", K(ret));
|
||||
} else {
|
||||
pd_storage_filter->childs_[i] = sub_pd_storage_filter;
|
||||
}
|
||||
}
|
||||
if (pd_storage_filter->n_child_ != child_cnt) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("child count is not match", K(ret), K(pd_storage_filter->n_child_), K(child_cnt));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObPushdownFilter::get_serialize_pushdown_filter_size(ObPushdownFilterNode* pd_filter_node)
|
||||
{
|
||||
int64_t len = 0;
|
||||
if (OB_NOT_NULL(pd_filter_node)) {
|
||||
len += serialization::encoded_length_vi32(pd_filter_node->type_);
|
||||
len += serialization::encoded_length(pd_filter_node->n_child_);
|
||||
len += pd_filter_node->get_serialize_size();
|
||||
for (int64_t i = 0; i < pd_filter_node->n_child_; ++i) {
|
||||
len += get_serialize_pushdown_filter_size(pd_filter_node->childs_[i]);
|
||||
}
|
||||
} else {
|
||||
int ret = OB_SUCCESS;
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_ERROR("pushdown filter is null", K(ret));
|
||||
}
|
||||
return len;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE(ObPushdownFilter)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(filter_tree_)) {
|
||||
if (OB_FAIL(serialization::encode(buf, buf_len, pos, false))) {
|
||||
LOG_WARN("fail to encode op type", K(ret));
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(serialization::encode(buf, buf_len, pos, true))) {
|
||||
LOG_WARN("fail to encode op type", K(ret));
|
||||
} else if (OB_FAIL(serialize_pushdown_filter(buf, buf_len, pos, filter_tree_))) {
|
||||
LOG_WARN("failed to serialize pushdown filter", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObPushdownFilter)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool has_filter = false;
|
||||
filter_tree_ = nullptr;
|
||||
if (OB_FAIL(serialization::decode(buf, data_len, pos, has_filter))) {
|
||||
LOG_WARN("fail to encode op type", K(ret));
|
||||
} else if (has_filter) {
|
||||
ObPushdownFilterFactory filter_factory(&alloc_);
|
||||
if (OB_FAIL(deserialize_pushdown_filter(filter_factory, buf, data_len, pos, filter_tree_))) {
|
||||
LOG_WARN("failed to deserialize pushdown filter", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE_SIZE(ObPushdownFilter)
|
||||
{
|
||||
int64_t len = 0;
|
||||
if (OB_ISNULL(filter_tree_)) {
|
||||
len += serialization::encoded_length(false);
|
||||
} else {
|
||||
len += serialization::encoded_length(true);
|
||||
len += get_serialize_pushdown_filter_size(filter_tree_);
|
||||
}
|
||||
return len;
|
||||
}
|
||||
|
||||
//--------------------- start filter executor ----------------------------
|
||||
int ObAndFilterExecutor::filter(bool& filtered)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
filtered = false;
|
||||
for (uint32_t i = 0; i < n_child_ && OB_SUCC(ret); ++i) {
|
||||
bool tmp_filtered = false;
|
||||
if (OB_ISNULL(childs_[i])) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("child is null", K(ret), K(i));
|
||||
} else if (OB_FAIL(childs_[i]->filter(tmp_filtered))) {
|
||||
LOG_WARN("failed to filter child", K(ret));
|
||||
} else if (tmp_filtered) {
|
||||
filtered = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOrFilterExecutor::filter(bool& filtered)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
filtered = true;
|
||||
for (uint32_t i = 0; i < n_child_ && OB_SUCC(ret); ++i) {
|
||||
bool tmp_filtered = false;
|
||||
if (OB_ISNULL(childs_[i])) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("child is null", K(ret), K(i));
|
||||
} else if (OB_FAIL(childs_[i]->filter(tmp_filtered))) {
|
||||
LOG_WARN("failed to filter child", K(ret));
|
||||
} else if (!tmp_filtered) {
|
||||
filtered = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObBlackFilterExecutor::filter(bool& filtered)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
filtered = false;
|
||||
ObDatum* cmp_res = NULL;
|
||||
if (OB_ISNULL(eval_ctx_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("eval ctx is null", K(ret));
|
||||
} else {
|
||||
ObPushdownBlackFilterNode& filter = reinterpret_cast<ObPushdownBlackFilterNode&>(filter_);
|
||||
for (uint32_t i = 0; i < filter.filter_exprs_.count() && OB_SUCC(ret); ++i) {
|
||||
if (OB_FAIL(filter.filter_exprs_.at(i)->eval(*eval_ctx_, cmp_res))) {
|
||||
LOG_WARN("failed to filter child", K(ret));
|
||||
} else if (cmp_res->is_null() || 0 == cmp_res->get_int()) {
|
||||
filtered = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObWhiteFilterExecutor::filter(bool& filtered)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(filtered);
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
return ret;
|
||||
}
|
||||
// end for test filter
|
||||
|
||||
int ObPushdownFilterExecutor::find_evaluated_datums(
|
||||
ObExpr* expr, const ObIArray<ObExpr*>& calc_exprs, ObIArray<ObExpr*>& eval_exprs)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (0 < expr->arg_cnt_ && is_contain(calc_exprs, expr)) {
|
||||
if (OB_FAIL(eval_exprs.push_back(expr))) {
|
||||
LOG_WARN("failed to push back expr", K(ret));
|
||||
}
|
||||
}
|
||||
for (uint32_t i = 0; i < expr->arg_cnt_ && OB_SUCC(ret); ++i) {
|
||||
if (OB_FAIL(find_evaluated_datums(expr->args_[i], calc_exprs, eval_exprs))) {
|
||||
LOG_WARN("failed to find evaluated datums", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPushdownFilterExecutor::find_evaluated_datums(
|
||||
ObIArray<ObExpr*>& src_exprs, const ObIArray<ObExpr*>& calc_exprs, ObIArray<ObExpr*>& eval_exprs)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
for (uint32_t i = 0; i < src_exprs.count() && OB_SUCC(ret); ++i) {
|
||||
if (OB_FAIL(find_evaluated_datums(src_exprs.at(i), calc_exprs, eval_exprs))) {
|
||||
LOG_WARN("failed to find evaluated datums", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPushdownFilterExecutor::init_bitmap(const int64_t row_count, common::ObBitmap*& bitmap)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
void* buf = nullptr;
|
||||
if (OB_NOT_NULL(filter_bitmap_)) {
|
||||
if (OB_FAIL(filter_bitmap_->expand_size(row_count))) {
|
||||
LOG_WARN("Failed to expand size of filter bitmap", K(ret));
|
||||
} else if (FALSE_IT(filter_bitmap_->reuse(is_logic_and_node()))) {
|
||||
}
|
||||
} else {
|
||||
if (OB_ISNULL(buf = allocator_.alloc(sizeof(ObBitmap)))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("Failed to alloc memory for filter bitmap", K(ret));
|
||||
} else if (FALSE_IT(filter_bitmap_ = new (buf) ObBitmap(allocator_))) {
|
||||
} else if (OB_FAIL(filter_bitmap_->init(row_count, is_logic_and_node()))) {
|
||||
LOG_WARN("Failed to init result bitmap", K(ret));
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
bitmap = filter_bitmap_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObPushdownFilterExecutor::init_filter_param(const share::schema::ColumnMap* col_id_map,
|
||||
const common::ObIArray<share::schema::ObColumnParam*>* col_params, const bool need_padding)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
void* buf = nullptr;
|
||||
const ObIArray<uint64_t>& col_ids = filter_.get_col_ids();
|
||||
if (OB_ISNULL(col_id_map) || OB_ISNULL(col_params)) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("Invalid filter executor parameter", K(ret), KP(col_id_map), KP(col_params));
|
||||
} else if (is_filter_node()) {
|
||||
reset_filter_param();
|
||||
if (0 == col_ids.count()) {
|
||||
} else if (OB_ISNULL((buf = allocator_.alloc(sizeof(int32_t) * col_ids.count())))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("Fail to allocate memory for filter executor col offsets", K(ret), K(col_ids.count()));
|
||||
} else if (FALSE_IT(col_offsets_ = new (buf) int32_t[col_ids.count()])) {
|
||||
} else if (OB_ISNULL((buf = allocator_.alloc(sizeof(share::schema::ObColumnParam*) * col_ids.count())))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("Fail to allocate memory for filter executor col param", K(ret), K(col_ids.count()));
|
||||
} else if (FALSE_IT(col_params_ = new (buf) const share::schema::ObColumnParam*[col_ids.count()]())) {
|
||||
} else {
|
||||
MEMSET(col_offsets_, -1, sizeof(int32_t) * col_ids.count());
|
||||
MEMSET(col_params_, 0, sizeof(const share::schema::ObColumnParam*) * col_ids.count());
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < col_ids.count(); i++) {
|
||||
if (OB_FAIL(col_id_map->get(col_ids.at(i), col_offsets_[i]))) {
|
||||
LOG_WARN("failed to get column offset from id", K(ret), K(i), K(col_ids), K(col_id_map));
|
||||
} else if (!need_padding) {
|
||||
} else if (OB_UNLIKELY(col_offsets_[i] >= col_params->count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("Unexpected col offset out of range out_cols", K(ret), K(col_offsets_[i]), K(col_params->count()));
|
||||
} else if (col_params->at(col_offsets_[i])->get_meta_type().is_fixed_len_char_type()) {
|
||||
col_params_[i] = col_params->at(col_offsets_[i]);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
reset_filter_param();
|
||||
} else {
|
||||
n_cols_ = col_ids.count();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObAndFilterExecutor::init_evaluated_datums(
|
||||
ObIAllocator& alloc, const ObIArray<ObExpr*>& calc_exprs, ObEvalCtx* eval_ctx)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
for (uint32_t i = 0; i < n_child_ && OB_SUCC(ret); ++i) {
|
||||
if (OB_FAIL(childs_[i]->init_evaluated_datums(alloc, calc_exprs, eval_ctx))) {
|
||||
LOG_WARN("failed to filter child", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOrFilterExecutor::init_evaluated_datums(
|
||||
ObIAllocator& alloc, const ObIArray<ObExpr*>& calc_exprs, ObEvalCtx* eval_ctx)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
for (uint32_t i = 0; i < n_child_ && OB_SUCC(ret); ++i) {
|
||||
if (OB_FAIL(childs_[i]->init_evaluated_datums(alloc, calc_exprs, eval_ctx))) {
|
||||
LOG_WARN("failed to filter child", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObWhiteFilterExecutor::init_evaluated_datums(
|
||||
ObIAllocator& alloc, const ObIArray<ObExpr*>& calc_exprs, ObEvalCtx* eval_ctx)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(alloc);
|
||||
UNUSED(calc_exprs);
|
||||
UNUSED(eval_ctx);
|
||||
ret = OB_NOT_SUPPORTED;
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObBlackFilterExecutor::filter(ObObj* objs, int64_t col_cnt, bool& filtered)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPushdownBlackFilterNode& node = reinterpret_cast<ObPushdownBlackFilterNode&>(filter_);
|
||||
if (col_cnt != node.column_exprs_.count()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected status: column count not match", K(ret), K(col_cnt), K(filter_.col_ids_));
|
||||
} else {
|
||||
for (int64_t i = 0; i < node.column_exprs_.count() && OB_SUCC(ret); ++i) {
|
||||
ObDatum& expr_datum = node.column_exprs_.at(i)->locate_datum_for_write(*eval_ctx_);
|
||||
if (OB_FAIL(expr_datum.from_obj(objs[i]))) {
|
||||
LOG_WARN("Failed to convert object from datum", K(ret), K(objs[i]));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(filter(filtered))) {
|
||||
LOG_WARN("failed to calc filter", K(ret));
|
||||
}
|
||||
clear_evaluated_datums();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObBlackFilterExecutor::init_evaluated_datums(
|
||||
ObIAllocator& alloc, const ObIArray<ObExpr*>& calc_exprs, ObEvalCtx* eval_ctx)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
eval_ctx_ = eval_ctx;
|
||||
ObArray<ObExpr*> eval_exprs;
|
||||
if (OB_FAIL(find_evaluated_datums(
|
||||
reinterpret_cast<ObPushdownBlackFilterNode&>(filter_).filter_exprs_, calc_exprs, eval_exprs))) {
|
||||
LOG_WARN("failed to find evaluated datums", K(ret));
|
||||
} else {
|
||||
eval_infos_ = reinterpret_cast<ObEvalInfo**>(alloc.alloc(eval_exprs.count() * sizeof(ObEvalInfo*)));
|
||||
if (OB_ISNULL(eval_infos_)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("failed to allocator memory", K(ret));
|
||||
} else {
|
||||
for (int64_t i = 0; i < eval_exprs.count() && OB_SUCC(ret); ++i) {
|
||||
eval_infos_[i] = &eval_exprs.at(i)->get_eval_info(*eval_ctx_);
|
||||
}
|
||||
n_eval_infos_ = eval_exprs.count();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
//--------------------- end filter executor ----------------------------
|
||||
|
||||
//--------------------- start filter executor constructor ----------------------------
|
||||
template <typename CLASST, FilterExecutorType type>
|
||||
int ObFilterExecutorConstructor::create_filter_executor(
|
||||
ObPushdownFilterNode* filter_tree, ObPushdownFilterExecutor*& filter_executor)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPushdownFilterExecutor* tmp_filter_executor = nullptr;
|
||||
if (OB_ISNULL(filter_tree)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("filter tree is null", K(ret));
|
||||
} else if (OB_FAIL(factory_.alloc(type, filter_tree->n_child_, *filter_tree, tmp_filter_executor))) {
|
||||
LOG_WARN("failed to alloc pushdown filter", K(ret), K(filter_tree->n_child_));
|
||||
} else {
|
||||
filter_executor = static_cast<CLASST*>(tmp_filter_executor);
|
||||
for (int64_t i = 0; i < filter_tree->n_child_ && OB_SUCC(ret); ++i) {
|
||||
ObPushdownFilterExecutor* sub_filter_executor = nullptr;
|
||||
if (OB_FAIL(apply(filter_tree->childs_[i], sub_filter_executor))) {
|
||||
LOG_WARN("failed to apply filter node", K(ret));
|
||||
} else if (OB_ISNULL(sub_filter_executor)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("sub filter executor is null", K(ret));
|
||||
} else if (OB_ISNULL(filter_executor->get_childs())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("childs is null", K(ret));
|
||||
} else {
|
||||
filter_executor->set_child(i, sub_filter_executor);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObFilterExecutorConstructor::apply(ObPushdownFilterNode* filter_tree, ObPushdownFilterExecutor*& filter_executor)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(filter_tree)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("filter node is null", K(ret));
|
||||
} else {
|
||||
switch (filter_tree->get_type()) {
|
||||
case BLACK_FILTER: {
|
||||
ret = create_filter_executor<ObBlackFilterExecutor, BLACK_FILTER_EXECUTOR>(filter_tree, filter_executor);
|
||||
if (OB_FAIL(ret)) {
|
||||
LOG_WARN("failed to create filter executor", K(ret));
|
||||
}
|
||||
break;
|
||||
}
|
||||
case WHITE_FILTER: {
|
||||
ret = create_filter_executor<ObWhiteFilterExecutor, WHITE_FILTER_EXECUTOR>(filter_tree, filter_executor);
|
||||
if (OB_FAIL(ret)) {
|
||||
LOG_WARN("failed to create filter executor", K(ret));
|
||||
}
|
||||
break;
|
||||
}
|
||||
case AND_FILTER: {
|
||||
ret = create_filter_executor<ObAndFilterExecutor, AND_FILTER_EXECUTOR>(filter_tree, filter_executor);
|
||||
if (OB_FAIL(ret)) {
|
||||
LOG_WARN("failed to create filter executor", K(ret));
|
||||
}
|
||||
break;
|
||||
}
|
||||
case OR_FILTER: {
|
||||
ret = create_filter_executor<ObOrFilterExecutor, OR_FILTER_EXECUTOR>(filter_tree, filter_executor);
|
||||
if (OB_FAIL(ret)) {
|
||||
LOG_WARN("failed to create filter executor", K(ret));
|
||||
}
|
||||
break;
|
||||
}
|
||||
default:
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected filter type", K(ret));
|
||||
break;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
//--------------------- start filter executor constructor ----------------------------
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
490
src/sql/engine/basic/ob_pushdown_filter.h
Normal file
490
src/sql/engine/basic/ob_pushdown_filter.h
Normal file
@ -0,0 +1,490 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OB_SQL_ENGINE_BASIC_OB_PUSHDOWN_FILTER_H_
|
||||
#define OB_SQL_ENGINE_BASIC_OB_PUSHDOWN_FILTER_H_
|
||||
|
||||
#include "lib/container/ob_bitmap.h"
|
||||
#include "sql/engine/expr/ob_expr.h"
|
||||
#include "sql/engine/ob_operator.h"
|
||||
#include "share/datum/ob_datum.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace share {
|
||||
namespace schema {
|
||||
class ObColumnParam;
|
||||
class ColumnMap;
|
||||
} // namespace schema
|
||||
} // namespace share
|
||||
namespace storage {
|
||||
struct ObTableIterParam;
|
||||
struct ObTableAccessContext;
|
||||
} // namespace storage
|
||||
|
||||
namespace sql {
|
||||
|
||||
class ObStaticEngineCG;
|
||||
|
||||
enum PushdownFilterType { BLACK_FILTER, WHITE_FILTER, AND_FILTER, OR_FILTER, MAX_FILTER_TYPE };
|
||||
|
||||
enum FilterExecutorType {
|
||||
BLACK_FILTER_EXECUTOR,
|
||||
WHITE_FILTER_EXECUTOR,
|
||||
AND_FILTER_EXECUTOR,
|
||||
OR_FILTER_EXECUTOR,
|
||||
MAX_EXECUTOR_TYPE
|
||||
};
|
||||
|
||||
class ObPushdownFilterUtils {
|
||||
public:
|
||||
static bool is_pushdown_storage(int32_t pd_storage_flag)
|
||||
{
|
||||
return pd_storage_flag & 0x01;
|
||||
}
|
||||
static bool is_pushdown_storage_index_back(int32_t pd_storage_flag)
|
||||
{
|
||||
return pd_storage_flag & 0x02;
|
||||
}
|
||||
};
|
||||
|
||||
class ObPushdownFilterNode {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObPushdownFilterNode(common::ObIAllocator& alloc)
|
||||
: alloc_(alloc), type_(PushdownFilterType::MAX_FILTER_TYPE), n_child_(0), childs_(nullptr), col_ids_(alloc)
|
||||
{}
|
||||
|
||||
PushdownFilterType get_type() const
|
||||
{
|
||||
return type_;
|
||||
}
|
||||
common::ObIArray<uint64_t>& get_col_ids()
|
||||
{
|
||||
return col_ids_;
|
||||
}
|
||||
void set_type(PushdownFilterType type)
|
||||
{
|
||||
type_ = type;
|
||||
}
|
||||
|
||||
virtual int merge(common::ObIArray<ObPushdownFilterNode*>& merged_node)
|
||||
{
|
||||
UNUSED(merged_node);
|
||||
return common::OB_SUCCESS;
|
||||
}
|
||||
virtual int postprocess()
|
||||
{
|
||||
return common::OB_SUCCESS;
|
||||
}
|
||||
TO_STRING_KV(K_(type), K_(n_child), K_(col_ids));
|
||||
|
||||
public:
|
||||
common::ObIAllocator& alloc_;
|
||||
PushdownFilterType type_;
|
||||
uint32_t n_child_;
|
||||
ObPushdownFilterNode** childs_;
|
||||
common::ObFixedArray<uint64_t, common::ObIAllocator> col_ids_; // cols this node is associated with
|
||||
};
|
||||
|
||||
class ObPushdownAndFilterNode : public ObPushdownFilterNode {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObPushdownAndFilterNode(common::ObIAllocator& alloc) : ObPushdownFilterNode(alloc)
|
||||
{}
|
||||
};
|
||||
|
||||
class ObPushdownOrFilterNode : public ObPushdownFilterNode {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObPushdownOrFilterNode(common::ObIAllocator& alloc) : ObPushdownFilterNode(alloc)
|
||||
{}
|
||||
};
|
||||
|
||||
class ObPushdownBlackFilterNode : public ObPushdownFilterNode {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObPushdownBlackFilterNode(common::ObIAllocator& alloc)
|
||||
: ObPushdownFilterNode(alloc), column_exprs_(alloc), filter_exprs_(alloc), tmp_expr_(nullptr)
|
||||
{}
|
||||
~ObPushdownBlackFilterNode()
|
||||
{}
|
||||
|
||||
int merge(common::ObIArray<ObPushdownFilterNode*>& merged_node) override;
|
||||
virtual int postprocess() override;
|
||||
OB_INLINE void clear_evaluated_datums();
|
||||
|
||||
public:
|
||||
ExprFixedArray column_exprs_;
|
||||
ExprFixedArray filter_exprs_;
|
||||
ObExpr* tmp_expr_;
|
||||
};
|
||||
|
||||
class ObPushdownWhiteFilterNode : public ObPushdownFilterNode {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObPushdownWhiteFilterNode(common::ObIAllocator& alloc) : ObPushdownFilterNode(alloc)
|
||||
{}
|
||||
~ObPushdownWhiteFilterNode()
|
||||
{}
|
||||
};
|
||||
|
||||
class ObPushdownFilterExecutor;
|
||||
class ObPushdownFilterNode;
|
||||
class ObPushdownFilterFactory {
|
||||
public:
|
||||
ObPushdownFilterFactory(common::ObIAllocator* alloc) : alloc_(alloc)
|
||||
{}
|
||||
int alloc(PushdownFilterType type, uint32_t n_child, ObPushdownFilterNode*& pd_filter);
|
||||
int alloc(FilterExecutorType type, uint32_t n_child, ObPushdownFilterNode& filter_node,
|
||||
ObPushdownFilterExecutor*& filter_executor);
|
||||
|
||||
private:
|
||||
// pushdown filter
|
||||
typedef int (*PDFilterAllocFunc)(common::ObIAllocator& alloc, uint32_t n_child, ObPushdownFilterNode*& filter_node);
|
||||
template <typename ClassT, PushdownFilterType type>
|
||||
static int alloc(common::ObIAllocator& alloc, uint32_t n_child, ObPushdownFilterNode*& filter_executor);
|
||||
static PDFilterAllocFunc PD_FILTER_ALLOC[PushdownFilterType::MAX_FILTER_TYPE];
|
||||
|
||||
// filter executor
|
||||
typedef int (*FilterExecutorAllocFunc)(common::ObIAllocator& alloc, uint32_t n_child,
|
||||
ObPushdownFilterNode& filter_node, ObPushdownFilterExecutor*& filter_executor);
|
||||
template <typename ClassT, typename FilterNodeT, FilterExecutorType type>
|
||||
static int alloc(common::ObIAllocator& alloc, uint32_t n_child, ObPushdownFilterNode& filter_node,
|
||||
ObPushdownFilterExecutor*& filter_executor);
|
||||
static FilterExecutorAllocFunc FILTER_EXECUTOR_ALLOC[FilterExecutorType::MAX_EXECUTOR_TYPE];
|
||||
|
||||
private:
|
||||
common::ObIAllocator* alloc_;
|
||||
};
|
||||
|
||||
class ObPushdownFilterConstructor {
|
||||
public:
|
||||
ObPushdownFilterConstructor(common::ObIAllocator* alloc, ObStaticEngineCG& static_cg)
|
||||
: alloc_(alloc), factory_(alloc), static_cg_(static_cg)
|
||||
{}
|
||||
int apply(ObRawExpr* raw_expr, ObPushdownFilterNode*& filter_tree);
|
||||
int apply(common::ObIArray<ObRawExpr*>& exprs, ObPushdownFilterNode*& filter_tree);
|
||||
|
||||
private:
|
||||
int merge_filter_node(ObPushdownFilterNode* dst, ObPushdownFilterNode* other,
|
||||
common::ObIArray<ObPushdownFilterNode*>& merged_node, bool& merged);
|
||||
int deduplicate_filter_node(common::ObIArray<ObPushdownFilterNode*>& filter_nodes, uint32_t& n_node);
|
||||
int create_black_filter_node(ObRawExpr* raw_expr, ObPushdownFilterNode*& filter_tree);
|
||||
bool can_split_or(ObRawExpr* raw_expr)
|
||||
{
|
||||
UNUSED(raw_expr);
|
||||
return false;
|
||||
}
|
||||
bool is_white_mode(ObRawExpr* raw_expr)
|
||||
{
|
||||
UNUSED(raw_expr);
|
||||
return false;
|
||||
}
|
||||
|
||||
private:
|
||||
common::ObIAllocator* alloc_;
|
||||
ObPushdownFilterFactory factory_;
|
||||
ObStaticEngineCG& static_cg_;
|
||||
};
|
||||
|
||||
// wrapper for PushdownFilterNode to support serilize
|
||||
class ObPushdownFilter {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObPushdownFilter(common::ObIAllocator& alloc) : alloc_(alloc), filter_tree_(nullptr)
|
||||
{}
|
||||
|
||||
void set_filter_tree(ObPushdownFilterNode* filter_tree)
|
||||
{
|
||||
filter_tree_ = filter_tree;
|
||||
}
|
||||
ObPushdownFilterNode*& get_pushdown_filter()
|
||||
{
|
||||
return filter_tree_;
|
||||
}
|
||||
ObPushdownFilterNode* get_pushdown_filter() const
|
||||
{
|
||||
return filter_tree_;
|
||||
}
|
||||
static int serialize_pushdown_filter(
|
||||
char* buf, int64_t buf_len, int64_t& pos, ObPushdownFilterNode* pd_storage_filter);
|
||||
static int deserialize_pushdown_filter(ObPushdownFilterFactory filter_factory, const char* buf, int64_t data_len,
|
||||
int64_t& pos, ObPushdownFilterNode*& pd_storage_filter);
|
||||
static int64_t get_serialize_pushdown_filter_size(ObPushdownFilterNode* pd_filter_node);
|
||||
// NEED_SERIALIZE_AND_DESERIALIZE;
|
||||
private:
|
||||
common::ObIAllocator& alloc_;
|
||||
ObPushdownFilterNode* filter_tree_;
|
||||
};
|
||||
|
||||
// executor interface
|
||||
class ObPushdownFilterExecutor {
|
||||
public:
|
||||
ObPushdownFilterExecutor(common::ObIAllocator& alloc, ObPushdownFilterNode& filter)
|
||||
: type_(FilterExecutorType::MAX_EXECUTOR_TYPE),
|
||||
filter_(filter),
|
||||
n_cols_(0),
|
||||
col_offsets_(nullptr),
|
||||
n_child_(0),
|
||||
childs_(nullptr),
|
||||
filter_bitmap_(nullptr),
|
||||
col_params_(nullptr),
|
||||
allocator_(alloc)
|
||||
{}
|
||||
~ObPushdownFilterExecutor()
|
||||
{
|
||||
if (nullptr != filter_bitmap_) {
|
||||
filter_bitmap_->~ObBitmap();
|
||||
allocator_.free(filter_bitmap_);
|
||||
}
|
||||
filter_bitmap_ = nullptr;
|
||||
reset_filter_param();
|
||||
for (uint32_t i = 0; i < n_child_; i++) {
|
||||
if (OB_NOT_NULL(childs_[i])) {
|
||||
childs_[i]->~ObPushdownFilterExecutor();
|
||||
childs_[i] = nullptr;
|
||||
}
|
||||
}
|
||||
n_child_ = 0;
|
||||
}
|
||||
void reset_filter_param()
|
||||
{
|
||||
if (nullptr != col_offsets_) {
|
||||
allocator_.free(col_offsets_);
|
||||
}
|
||||
if (nullptr != col_params_) {
|
||||
allocator_.free(col_params_);
|
||||
}
|
||||
col_offsets_ = nullptr;
|
||||
col_params_ = nullptr;
|
||||
}
|
||||
|
||||
// interface for storage
|
||||
virtual OB_INLINE bool is_filter_black_node() const
|
||||
{
|
||||
return type_ == BLACK_FILTER_EXECUTOR;
|
||||
}
|
||||
virtual OB_INLINE bool is_filter_white_node() const
|
||||
{
|
||||
return type_ == WHITE_FILTER_EXECUTOR;
|
||||
}
|
||||
virtual OB_INLINE bool is_filter_node() const
|
||||
{
|
||||
return is_filter_black_node() || is_filter_white_node();
|
||||
}
|
||||
virtual OB_INLINE bool is_logic_and_node() const
|
||||
{
|
||||
return type_ == AND_FILTER_EXECUTOR;
|
||||
}
|
||||
virtual OB_INLINE bool is_logic_or_node() const
|
||||
{
|
||||
return type_ == OR_FILTER_EXECUTOR;
|
||||
}
|
||||
virtual OB_INLINE bool is_logic_op_node() const
|
||||
{
|
||||
return is_logic_and_node() || is_logic_or_node();
|
||||
}
|
||||
void set_type(FilterExecutorType type)
|
||||
{
|
||||
type_ = type;
|
||||
}
|
||||
OB_INLINE FilterExecutorType get_type()
|
||||
{
|
||||
return type_;
|
||||
}
|
||||
OB_INLINE ObPushdownFilterNode& get_filter_node()
|
||||
{
|
||||
return filter_;
|
||||
}
|
||||
OB_INLINE common::ObIArray<uint64_t>& get_col_ids()
|
||||
{
|
||||
return filter_.get_col_ids();
|
||||
}
|
||||
OB_INLINE int64_t get_col_count() const
|
||||
{
|
||||
return n_cols_;
|
||||
}
|
||||
OB_INLINE const int32_t* get_col_offsets() const
|
||||
{
|
||||
return col_offsets_;
|
||||
}
|
||||
OB_INLINE const share::schema::ObColumnParam** get_col_params() const
|
||||
{
|
||||
return col_params_;
|
||||
}
|
||||
OB_INLINE uint32_t get_child_count() const
|
||||
{
|
||||
return n_child_;
|
||||
}
|
||||
inline int get_child(uint32_t nth_child, ObPushdownFilterExecutor*& filter_executor)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
if (nth_child >= n_child_) {
|
||||
ret = common::OB_ERR_UNEXPECTED;
|
||||
SQL_ENG_LOG(WARN, "invalid child", K(nth_child), K(n_child_));
|
||||
} else {
|
||||
filter_executor = childs_[nth_child];
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
void set_child(uint32_t i, ObPushdownFilterExecutor* child)
|
||||
{
|
||||
childs_[i] = child;
|
||||
}
|
||||
void set_childs(uint32_t n_child, ObPushdownFilterExecutor** childs)
|
||||
{
|
||||
n_child_ = n_child;
|
||||
childs_ = childs;
|
||||
}
|
||||
ObPushdownFilterExecutor** get_childs() const
|
||||
{
|
||||
return childs_;
|
||||
}
|
||||
const common::ObBitmap* get_result() const
|
||||
{
|
||||
return filter_bitmap_;
|
||||
}
|
||||
// for testing purpose
|
||||
virtual int filter(bool& filtered)
|
||||
{
|
||||
UNUSED(filtered);
|
||||
return common::OB_NOT_SUPPORTED;
|
||||
}
|
||||
int init_bitmap(const int64_t row_count, common::ObBitmap*& bitmap);
|
||||
int init_filter_param(const share::schema::ColumnMap* col_id_map,
|
||||
const common::ObIArray<share::schema::ObColumnParam*>* col_params, const bool need_padding);
|
||||
virtual int init_evaluated_datums(
|
||||
common::ObIAllocator& alloc, const common::ObIArray<ObExpr*>& calc_exprs, ObEvalCtx* eval_ctx)
|
||||
{
|
||||
UNUSED(alloc);
|
||||
UNUSED(calc_exprs);
|
||||
UNUSED(eval_ctx);
|
||||
return common::OB_NOT_SUPPORTED;
|
||||
}
|
||||
VIRTUAL_TO_STRING_KV(K_(type), K_(n_cols), "col_offsets", common::ObArrayWrap<int32_t>(col_offsets_, n_cols_),
|
||||
K_(n_child), KP_(childs), KP_(filter_bitmap), KP_(col_params));
|
||||
|
||||
protected:
|
||||
int find_evaluated_datums(
|
||||
ObExpr* expr, const common::ObIArray<ObExpr*>& calc_exprs, common::ObIArray<ObExpr*>& eval_exprs);
|
||||
int find_evaluated_datums(common::ObIArray<ObExpr*>& src_exprs, const common::ObIArray<ObExpr*>& calc_exprs,
|
||||
common::ObIArray<ObExpr*>& eval_exprs);
|
||||
|
||||
protected:
|
||||
FilterExecutorType type_;
|
||||
ObPushdownFilterNode& filter_;
|
||||
int64_t n_cols_;
|
||||
int32_t* col_offsets_;
|
||||
uint32_t n_child_;
|
||||
ObPushdownFilterExecutor** childs_;
|
||||
common::ObBitmap* filter_bitmap_;
|
||||
const share::schema::ObColumnParam** col_params_;
|
||||
common::ObIAllocator& allocator_;
|
||||
};
|
||||
|
||||
class ObBlackFilterExecutor : public ObPushdownFilterExecutor {
|
||||
public:
|
||||
ObBlackFilterExecutor(common::ObIAllocator& alloc, ObPushdownBlackFilterNode& filter)
|
||||
: ObPushdownFilterExecutor(alloc, filter), n_eval_infos_(0), eval_infos_(nullptr), eval_ctx_(nullptr)
|
||||
{}
|
||||
~ObBlackFilterExecutor()
|
||||
{}
|
||||
|
||||
int filter(common::ObObj* objs, int64_t col_cnt, bool& ret_val);
|
||||
virtual int filter(bool& filtered) override;
|
||||
virtual int init_evaluated_datums(
|
||||
common::ObIAllocator& alloc, const common::ObIArray<ObExpr*>& calc_exprs, ObEvalCtx* eval_ctx) override;
|
||||
void clear_evaluated_datums();
|
||||
INHERIT_TO_STRING_KV("ObPushdownBlackFilterExecutor", ObPushdownFilterExecutor, K_(filter), K_(n_eval_infos),
|
||||
K_(n_cols), "col_offsets", common::ObArrayWrap<int32_t>(col_offsets_, n_cols_), KP_(eval_infos), KP_(eval_ctx));
|
||||
|
||||
private:
|
||||
int32_t n_eval_infos_;
|
||||
ObEvalInfo** eval_infos_;
|
||||
ObEvalCtx* eval_ctx_;
|
||||
};
|
||||
|
||||
OB_INLINE void ObBlackFilterExecutor::clear_evaluated_datums()
|
||||
{
|
||||
for (int i = 0; i < n_eval_infos_; i++) {
|
||||
eval_infos_[i]->clear_evaluated_flag();
|
||||
}
|
||||
}
|
||||
|
||||
class ObWhiteFilterExecutor : public ObPushdownFilterExecutor {
|
||||
public:
|
||||
ObWhiteFilterExecutor(common::ObIAllocator& alloc, ObPushdownWhiteFilterNode& filter)
|
||||
: ObPushdownFilterExecutor(alloc, filter)
|
||||
{}
|
||||
~ObWhiteFilterExecutor()
|
||||
{}
|
||||
|
||||
virtual int filter(bool& filtered) override;
|
||||
virtual int init_evaluated_datums(
|
||||
common::ObIAllocator& alloc, const common::ObIArray<ObExpr*>& calc_exprs, ObEvalCtx* eval_ctx) override;
|
||||
INHERIT_TO_STRING_KV("ObPushdownFilterExecutor", ObPushdownFilterExecutor, K_(filter));
|
||||
|
||||
private:
|
||||
};
|
||||
|
||||
class ObAndFilterExecutor : public ObPushdownFilterExecutor {
|
||||
public:
|
||||
ObAndFilterExecutor(common::ObIAllocator& alloc, ObPushdownAndFilterNode& filter)
|
||||
: ObPushdownFilterExecutor(alloc, filter)
|
||||
{}
|
||||
virtual int filter(bool& filtered) override;
|
||||
virtual int init_evaluated_datums(
|
||||
common::ObIAllocator& alloc, const common::ObIArray<ObExpr*>& calc_exprs, ObEvalCtx* eval_ctx) override;
|
||||
INHERIT_TO_STRING_KV("ObPushdownFilterExecutor", ObPushdownFilterExecutor, K_(filter));
|
||||
|
||||
private:
|
||||
};
|
||||
|
||||
class ObOrFilterExecutor : public ObPushdownFilterExecutor {
|
||||
public:
|
||||
ObOrFilterExecutor(common::ObIAllocator& alloc, ObPushdownOrFilterNode& filter)
|
||||
: ObPushdownFilterExecutor(alloc, filter)
|
||||
{}
|
||||
|
||||
virtual int filter(bool& filtered) override;
|
||||
virtual int init_evaluated_datums(
|
||||
common::ObIAllocator& alloc, const common::ObIArray<ObExpr*>& calc_exprs, ObEvalCtx* eval_ctx) override;
|
||||
INHERIT_TO_STRING_KV("ObPushdownFilterExecutor", ObPushdownFilterExecutor, K_(filter));
|
||||
|
||||
private:
|
||||
};
|
||||
|
||||
class ObFilterExecutorConstructor {
|
||||
public:
|
||||
ObFilterExecutorConstructor(common::ObIAllocator* alloc) : alloc_(alloc), factory_(alloc)
|
||||
{}
|
||||
int apply(ObPushdownFilterNode* filter_tree, ObPushdownFilterExecutor*& filter_executor);
|
||||
|
||||
private:
|
||||
template <typename CLASST, FilterExecutorType type>
|
||||
int create_filter_executor(ObPushdownFilterNode* filter_tree, ObPushdownFilterExecutor*& filter_executor);
|
||||
|
||||
private:
|
||||
common::ObIAllocator* alloc_;
|
||||
ObPushdownFilterFactory factory_;
|
||||
};
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
|
||||
#endif // OB_SQL_ENGINE_BASIC_OB_PUSHDOWN_FILTER_H_
|
||||
1228
src/sql/engine/basic/ob_ra_datum_store.cpp
Normal file
1228
src/sql/engine/basic/ob_ra_datum_store.cpp
Normal file
File diff suppressed because it is too large
Load Diff
508
src/sql/engine/basic/ob_ra_datum_store.h
Normal file
508
src/sql/engine/basic/ob_ra_datum_store.h
Normal file
@ -0,0 +1,508 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_RA_DATUM_STORE_H_
|
||||
#define OCEANBASE_BASIC_OB_RA_DATUM_STORE_H_
|
||||
|
||||
#include "share/ob_define.h"
|
||||
#include "lib/container/ob_se_array.h"
|
||||
#include "lib/allocator/page_arena.h"
|
||||
#include "lib/utility/ob_print_utils.h"
|
||||
#include "lib/list/ob_dlist.h"
|
||||
#include "common/row/ob_row.h"
|
||||
#include "share/datum/ob_datum.h"
|
||||
#include "sql/engine/expr/ob_expr.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
// Random access row store, support disk store.
|
||||
// All row must have same cell count and projector.
|
||||
class ObRADatumStore {
|
||||
public:
|
||||
class ShrinkBuffer {
|
||||
public:
|
||||
ShrinkBuffer() : data_(NULL), head_(0), tail_(0), cap_(0)
|
||||
{}
|
||||
|
||||
int init(char* buf, const int64_t buf_size);
|
||||
inline int64_t remain() const
|
||||
{
|
||||
return tail_ - head_;
|
||||
}
|
||||
inline char* data()
|
||||
{
|
||||
return data_;
|
||||
}
|
||||
inline char* head() const
|
||||
{
|
||||
return data_ + head_;
|
||||
}
|
||||
inline int64_t head_size() const
|
||||
{
|
||||
return head_;
|
||||
}
|
||||
inline char* tail() const
|
||||
{
|
||||
return data_ + tail_;
|
||||
}
|
||||
inline int64_t tail_size() const
|
||||
{
|
||||
return cap_ - tail_;
|
||||
}
|
||||
inline int64_t capacity() const
|
||||
{
|
||||
return cap_;
|
||||
}
|
||||
inline bool is_inited() const
|
||||
{
|
||||
return NULL != data_;
|
||||
}
|
||||
|
||||
inline void reset()
|
||||
{
|
||||
*this = ShrinkBuffer();
|
||||
}
|
||||
inline void reuse()
|
||||
{
|
||||
head_ = 0;
|
||||
tail_ = cap_;
|
||||
}
|
||||
inline int fill_head(int64_t size);
|
||||
inline int fill_tail(int64_t size);
|
||||
inline int compact();
|
||||
|
||||
TO_STRING_KV(KP_(data), K_(head), K_(tail), K_(cap));
|
||||
|
||||
private:
|
||||
char* data_;
|
||||
int64_t head_;
|
||||
int64_t tail_;
|
||||
int64_t cap_;
|
||||
};
|
||||
|
||||
class LinkNode : public common::ObDLinkBase<LinkNode> {};
|
||||
|
||||
static inline int row_copy_size(const common::ObIArray<ObExpr*>& exprs, ObEvalCtx& ctx, int64_t& size)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
common::ObDatum* datum = nullptr;
|
||||
size = DATUM_SIZE * exprs.count();
|
||||
for (int64_t i = 0; i < exprs.count() && OB_SUCC(ret); ++i) {
|
||||
if (OB_FAIL(exprs.at(i)->eval(ctx, datum))) {
|
||||
SQL_ENG_LOG(WARN, "failed to eval expr datum", K(ret));
|
||||
} else {
|
||||
size += datum->len_;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
static inline int row_copy_size(const common::ObIArray<common::ObDatum>& datums, int64_t& size)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
size = DATUM_SIZE * datums.count();
|
||||
for (int64_t i = 0; i < datums.count(); ++i) {
|
||||
size += datums.at(i).len_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
struct StoredRow {
|
||||
StoredRow() : readable_(true), cnt_(0), row_size_(0)
|
||||
{}
|
||||
|
||||
int copy_datums(const common::ObIArray<ObExpr*>& exprs, ObEvalCtx& ctx, char* buf, const int64_t size,
|
||||
const int64_t row_size, const uint32_t row_extend_size);
|
||||
int copy_datums(const common::ObIArray<common::ObDatum>& datums, char* buf, const int64_t size,
|
||||
const int64_t row_size, const uint32_t row_extend_size);
|
||||
int to_expr(const common::ObIArray<ObExpr*>& exprs, ObEvalCtx& ctx) const;
|
||||
inline common::ObDatum* cells()
|
||||
{
|
||||
return reinterpret_cast<common::ObDatum*>(payload_);
|
||||
}
|
||||
inline const common::ObDatum* cells() const
|
||||
{
|
||||
return reinterpret_cast<const common::ObDatum*>(payload_);
|
||||
}
|
||||
inline void* get_extra_payload() const
|
||||
{
|
||||
return static_cast<void*>(const_cast<char*>(payload_ + sizeof(common::ObDatum) * cnt_));
|
||||
}
|
||||
int assign(const StoredRow* sr);
|
||||
|
||||
int to_copyable();
|
||||
int to_readable();
|
||||
TO_STRING_KV(K_(readable), K_(cnt), K_(row_size), "cells", common::ObArrayWrap<common::ObDatum>(cells(), cnt_));
|
||||
|
||||
uint32_t readable_ : 1;
|
||||
uint32_t cnt_ : 31;
|
||||
uint32_t row_size_;
|
||||
char payload_[0];
|
||||
} __attribute__((packed));
|
||||
|
||||
struct Block {
|
||||
static const int64_t MAGIC = 0x35f4451b9b56eb12;
|
||||
typedef uint32_t row_idx_t;
|
||||
const static int64_t ROW_HEAD_SIZE = sizeof(StoredRow);
|
||||
const static int64_t ROW_INDEX_SIZE = sizeof(row_idx_t);
|
||||
|
||||
Block() : magic_(MAGIC), row_id_(0), rows_(0), idx_off_(0)
|
||||
{}
|
||||
|
||||
static int64_t inline min_buf_size(const int64_t row_store_size)
|
||||
{
|
||||
return sizeof(Block) + row_store_size;
|
||||
}
|
||||
static int inline row_store_size(
|
||||
const common::ObIArray<ObExpr*>& exprs, ObEvalCtx& ctx, int64_t& size, uint32_t row_extend_size = 0)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
size = 0;
|
||||
if (OB_FAIL(row_copy_size(exprs, ctx, size))) {
|
||||
SQL_ENG_LOG(WARN, "failed to calc store row size", K(ret));
|
||||
} else {
|
||||
size += ROW_HEAD_SIZE + ROW_INDEX_SIZE + row_extend_size;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
static int inline row_store_size(
|
||||
const common::ObIArray<common::ObDatum>& datums, int64_t& size, uint32_t row_extend_size = 0)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
size = 0;
|
||||
if (OB_FAIL(row_copy_size(datums, size))) {
|
||||
SQL_ENG_LOG(WARN, "failed to calc store row size", K(ret));
|
||||
} else {
|
||||
size += ROW_HEAD_SIZE + ROW_INDEX_SIZE + row_extend_size;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int add_row(ShrinkBuffer& buf, const common::ObIArray<ObExpr*>& exprs, ObEvalCtx& ctx, const int64_t row_size,
|
||||
uint32_t row_extend_size, StoredRow** stored_row = NULL);
|
||||
int add_row(ShrinkBuffer& buf, const common::ObIArray<common::ObDatum>& datums, const int64_t row_size,
|
||||
uint32_t row_extend_size, StoredRow** stored_row = NULL);
|
||||
int copy_stored_row(ShrinkBuffer& buf, const StoredRow& stored_row, StoredRow** dst_sr);
|
||||
int compact(ShrinkBuffer& buf);
|
||||
int to_copyable();
|
||||
const row_idx_t* indexes() const
|
||||
{
|
||||
return reinterpret_cast<const row_idx_t*>(payload_ + idx_off_);
|
||||
}
|
||||
inline bool contain(const int64_t row_id) const
|
||||
{
|
||||
return row_id_ <= row_id && row_id < row_id_ + rows_;
|
||||
}
|
||||
|
||||
int get_store_row(const int64_t row_id, const StoredRow*& sr);
|
||||
|
||||
TO_STRING_KV(K_(magic), K_(row_id), K_(rows), K_(idx_off));
|
||||
|
||||
int64_t magic_;
|
||||
int64_t row_id_;
|
||||
int32_t rows_;
|
||||
int32_t idx_off_;
|
||||
char payload_[0];
|
||||
} __attribute__((packed));
|
||||
|
||||
struct IndexBlock;
|
||||
struct BlockIndex {
|
||||
static bool compare(const BlockIndex& bi, const int64_t row_id)
|
||||
{
|
||||
return bi.row_id_ < row_id;
|
||||
}
|
||||
TO_STRING_KV(K_(is_idx_block), K_(on_disk), K_(row_id), K_(offset), K_(length));
|
||||
|
||||
uint64_t is_idx_block_ : 1;
|
||||
uint64_t on_disk_ : 1;
|
||||
uint64_t row_id_ : 62;
|
||||
union {
|
||||
IndexBlock* idx_blk_;
|
||||
Block* blk_;
|
||||
int64_t offset_;
|
||||
};
|
||||
int32_t length_;
|
||||
} __attribute__((packed));
|
||||
|
||||
struct IndexBlock {
|
||||
const static int64_t MAGIC = 0x4847bcb053c3703f;
|
||||
const static int64_t INDEX_BLOCK_SIZE = (64 << 10) - sizeof(LinkNode);
|
||||
constexpr static inline int64_t capacity()
|
||||
{
|
||||
return (INDEX_BLOCK_SIZE - sizeof(IndexBlock)) / sizeof(BlockIndex);
|
||||
}
|
||||
|
||||
IndexBlock() : magic_(MAGIC), cnt_(0)
|
||||
{}
|
||||
|
||||
inline int64_t buffer_size() const
|
||||
{
|
||||
return sizeof(*this) + sizeof(BlockIndex) * cnt_;
|
||||
}
|
||||
inline bool is_full() const
|
||||
{
|
||||
return cnt_ == capacity();
|
||||
}
|
||||
|
||||
// may return false when row in position (false negative),
|
||||
// since block index only contain start row_id, we can not detect right boundary.
|
||||
inline bool row_in_pos(const int64_t row_id, const int64_t pos);
|
||||
|
||||
void reset()
|
||||
{
|
||||
cnt_ = 0;
|
||||
}
|
||||
|
||||
TO_STRING_KV(K_(magic), K_(cnt));
|
||||
|
||||
int64_t magic_;
|
||||
int32_t cnt_;
|
||||
BlockIndex block_indexes_[0];
|
||||
} __attribute__((packed));
|
||||
|
||||
struct BlockBuffer {
|
||||
BlockBuffer() : buf_(), blk_()
|
||||
{}
|
||||
|
||||
void reset()
|
||||
{
|
||||
buf_.reset();
|
||||
blk_ = NULL;
|
||||
}
|
||||
ShrinkBuffer buf_;
|
||||
Block* blk_;
|
||||
};
|
||||
|
||||
// Reader must be deleted before ObRADatumStore
|
||||
class Reader {
|
||||
friend class ObRADatumStore;
|
||||
|
||||
public:
|
||||
explicit Reader(ObRADatumStore& store) : store_(store), file_size_(0), idx_blk_(NULL), ib_pos_(0), blk_(NULL)
|
||||
{}
|
||||
virtual ~Reader()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
inline int64_t get_row_cnt() const
|
||||
{
|
||||
return store_.get_row_cnt();
|
||||
}
|
||||
// row memory will hold until the next get_row called or non-const method of ObRADatumStore called
|
||||
int get_row(const int64_t row_id, const StoredRow*& sr);
|
||||
|
||||
void reset();
|
||||
void reuse();
|
||||
|
||||
private:
|
||||
void reset_cursor(const int64_t file_size);
|
||||
|
||||
private:
|
||||
ObRADatumStore& store_;
|
||||
// idx_blk_, blk_ may point to the writing block,
|
||||
// we need to invalid the pointers if file_size_ change.
|
||||
int64_t file_size_;
|
||||
IndexBlock* idx_blk_;
|
||||
int64_t ib_pos_; // current block index position in index block
|
||||
Block* blk_;
|
||||
|
||||
ShrinkBuffer buf_;
|
||||
ShrinkBuffer idx_buf_;
|
||||
|
||||
DISALLOW_COPY_AND_ASSIGN(Reader);
|
||||
};
|
||||
|
||||
public:
|
||||
const static int64_t BLOCK_SIZE = (64L << 10) - sizeof(LinkNode);
|
||||
const static int64_t BIG_BLOCK_SIZE = (256L << 10) - sizeof(LinkNode);
|
||||
// alloc first index block after store 1MB data.
|
||||
const static int64_t DEFAULT_BLOCK_CNT = (1L << 20) / BLOCK_SIZE;
|
||||
static const int32_t DATUM_SIZE = sizeof(common::ObDatum);
|
||||
|
||||
explicit ObRADatumStore(common::ObIAllocator* alloc = NULL);
|
||||
virtual ~ObRADatumStore()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
int init(int64_t mem_limit, uint64_t tenant_id = common::OB_SERVER_TENANT_ID,
|
||||
int64_t mem_ctx_id = common::ObCtxIds::DEFAULT_CTX_ID, const char* label = common::ObModIds::OB_SQL_ROW_STORE);
|
||||
|
||||
void reset();
|
||||
// Keeping one memory block, reader must call reuse() too.
|
||||
void reuse();
|
||||
|
||||
int add_row(const common::ObIArray<ObExpr*>& exprs, ObEvalCtx* ctx, StoredRow** stored_row = nullptr);
|
||||
int add_row(const common::ObIArray<common::ObDatum>& datums, StoredRow** stored_row = nullptr);
|
||||
int add_row(const StoredRow& src_stored_row, StoredRow** stored_row = nullptr);
|
||||
int finish_add_row();
|
||||
// row memory will hold until the next non-const method of ObRADatumStore called.
|
||||
int get_row(const int64_t row_id, const StoredRow*& sr)
|
||||
{
|
||||
return inner_reader_.get_row(row_id, sr);
|
||||
}
|
||||
|
||||
bool is_inited() const
|
||||
{
|
||||
return inited_;
|
||||
}
|
||||
bool is_file_open() const
|
||||
{
|
||||
return fd_ >= 0;
|
||||
}
|
||||
|
||||
void set_tenant_id(const uint64_t tenant_id)
|
||||
{
|
||||
tenant_id_ = tenant_id;
|
||||
}
|
||||
void set_mem_ctx_id(const int64_t ctx_id)
|
||||
{
|
||||
ctx_id_ = ctx_id;
|
||||
}
|
||||
void set_mem_limit(const int64_t limit)
|
||||
{
|
||||
mem_limit_ = limit;
|
||||
}
|
||||
|
||||
inline int64_t get_row_cnt() const
|
||||
{
|
||||
return row_cnt_;
|
||||
}
|
||||
inline int64_t get_mem_hold() const
|
||||
{
|
||||
return mem_hold_;
|
||||
}
|
||||
inline int64_t get_file_size() const
|
||||
{
|
||||
return file_size_;
|
||||
}
|
||||
|
||||
TO_STRING_KV(
|
||||
K_(tenant_id), K_(label), K_(ctx_id), K_(mem_limit), K_(save_row_cnt), K_(row_cnt), K_(fd), K_(file_size));
|
||||
|
||||
private:
|
||||
static int get_timeout(int64_t& timeout_ms);
|
||||
void* alloc_blk_mem(const int64_t size);
|
||||
void free_blk_mem(void* mem, const int64_t size = 0);
|
||||
int setup_block(BlockBuffer& blkbuf) const;
|
||||
int alloc_block(BlockBuffer& blkbuf, const int64_t min_size);
|
||||
// new block is not needed if %min_size is zero. (finish add row)
|
||||
int switch_block(const int64_t min_size);
|
||||
int add_block_idx(const BlockIndex& bi);
|
||||
int alloc_idx_block(IndexBlock*& ib);
|
||||
int build_idx_block();
|
||||
int switch_idx_block(bool finish_add = false);
|
||||
|
||||
int load_block(Reader& reader, const int64_t row_id);
|
||||
int load_idx_block(Reader& reader, IndexBlock*& ib, const BlockIndex& bi);
|
||||
int find_block_idx(Reader& reader, BlockIndex& bi, const int64_t row_id);
|
||||
|
||||
int get_store_row(Reader& reader, const int64_t row_id, const StoredRow*& sr);
|
||||
|
||||
int ensure_reader_buffer(ShrinkBuffer& buf, const int64_t size);
|
||||
|
||||
int write_file(BlockIndex& bi, void* buf, int64_t size);
|
||||
int read_file(void* buf, const int64_t size, const int64_t offset);
|
||||
|
||||
bool need_dump();
|
||||
|
||||
private:
|
||||
bool inited_;
|
||||
uint64_t tenant_id_;
|
||||
const char* label_;
|
||||
int64_t ctx_id_;
|
||||
int64_t mem_limit_;
|
||||
|
||||
BlockBuffer blkbuf_;
|
||||
IndexBlock* idx_blk_;
|
||||
|
||||
int64_t save_row_cnt_;
|
||||
int64_t row_cnt_;
|
||||
|
||||
int64_t fd_;
|
||||
int64_t dir_id_;
|
||||
int64_t file_size_;
|
||||
|
||||
Reader inner_reader_;
|
||||
|
||||
common::ObDList<LinkNode> blk_mem_list_;
|
||||
common::ObSEArray<BlockIndex, DEFAULT_BLOCK_CNT> blocks_;
|
||||
|
||||
int64_t mem_hold_;
|
||||
common::DefaultPageAllocator inner_allocator_;
|
||||
common::ObIAllocator& allocator_;
|
||||
|
||||
uint32_t row_extend_size_;
|
||||
|
||||
DISALLOW_COPY_AND_ASSIGN(ObRADatumStore);
|
||||
};
|
||||
|
||||
inline int ObRADatumStore::ShrinkBuffer::fill_head(int64_t size)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
if (size < -head_) {
|
||||
ret = common::OB_INVALID_ARGUMENT;
|
||||
SQL_ENG_LOG(WARN, "invalid argument", K(size), K_(head));
|
||||
} else if (size > remain()) {
|
||||
ret = common::OB_BUF_NOT_ENOUGH;
|
||||
SQL_ENG_LOG(WARN, "buffer not enough", K(size), "remain", remain());
|
||||
} else {
|
||||
head_ += size;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
inline int ObRADatumStore::ShrinkBuffer::fill_tail(int64_t size)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
if (size < -tail_size()) {
|
||||
ret = common::OB_INVALID_ARGUMENT;
|
||||
SQL_ENG_LOG(WARN, "invalid argument", K(size), "tail_size", tail_size());
|
||||
} else if (size > remain()) {
|
||||
ret = common::OB_BUF_NOT_ENOUGH;
|
||||
SQL_ENG_LOG(WARN, "buffer not enough", K(size), "remain", remain());
|
||||
} else {
|
||||
tail_ -= size;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
inline int ObRADatumStore::ShrinkBuffer::compact()
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
if (!is_inited()) {
|
||||
ret = common::OB_NOT_INIT;
|
||||
SQL_ENG_LOG(WARN, "not inited", K(ret));
|
||||
} else {
|
||||
MEMMOVE(head(), tail(), tail_size());
|
||||
head_ += tail_size();
|
||||
tail_ = cap_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
inline bool ObRADatumStore::IndexBlock::row_in_pos(const int64_t row_id, const int64_t pos)
|
||||
{
|
||||
bool in_pos = false;
|
||||
if (cnt_ > 0 && pos >= 0 && pos < cnt_) {
|
||||
if (pos + 1 == cnt_) {
|
||||
in_pos = block_indexes_[pos].row_id_ == row_id;
|
||||
} else {
|
||||
in_pos = block_indexes_[pos].row_id_ <= row_id && row_id < block_indexes_[pos + 1].row_id_;
|
||||
}
|
||||
}
|
||||
return in_pos;
|
||||
}
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_BASIC_OB_RA_DATUM_STORE_H_
|
||||
1108
src/sql/engine/basic/ob_ra_row_store.cpp
Normal file
1108
src/sql/engine/basic/ob_ra_row_store.cpp
Normal file
File diff suppressed because it is too large
Load Diff
455
src/sql/engine/basic/ob_ra_row_store.h
Normal file
455
src/sql/engine/basic/ob_ra_row_store.h
Normal file
@ -0,0 +1,455 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_RA_ROW_STORE_H_
|
||||
#define OCEANBASE_BASIC_OB_RA_ROW_STORE_H_
|
||||
|
||||
#include "share/ob_define.h"
|
||||
#include "lib/container/ob_se_array.h"
|
||||
#include "lib/allocator/page_arena.h"
|
||||
#include "lib/utility/ob_print_utils.h"
|
||||
#include "lib/list/ob_dlist.h"
|
||||
#include "common/row/ob_row.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
// Random access row store, support disk store.
|
||||
// All row must have same cell count and projector.
|
||||
class ObRARowStore {
|
||||
public:
|
||||
class ShrinkBuffer {
|
||||
public:
|
||||
ShrinkBuffer() : data_(NULL), head_(0), tail_(0), cap_(0)
|
||||
{}
|
||||
|
||||
int init(char* buf, const int64_t buf_size);
|
||||
inline int64_t remain() const
|
||||
{
|
||||
return tail_ - head_;
|
||||
}
|
||||
inline char* data()
|
||||
{
|
||||
return data_;
|
||||
}
|
||||
inline char* head() const
|
||||
{
|
||||
return data_ + head_;
|
||||
}
|
||||
inline int64_t head_size() const
|
||||
{
|
||||
return head_;
|
||||
}
|
||||
inline char* tail() const
|
||||
{
|
||||
return data_ + tail_;
|
||||
}
|
||||
inline int64_t tail_size() const
|
||||
{
|
||||
return cap_ - tail_;
|
||||
}
|
||||
inline int64_t capacity() const
|
||||
{
|
||||
return cap_;
|
||||
}
|
||||
inline bool is_inited() const
|
||||
{
|
||||
return NULL != data_;
|
||||
}
|
||||
|
||||
inline void reset()
|
||||
{
|
||||
*this = ShrinkBuffer();
|
||||
}
|
||||
inline void reuse()
|
||||
{
|
||||
head_ = 0;
|
||||
tail_ = cap_;
|
||||
}
|
||||
inline int fill_head(int64_t size);
|
||||
inline int fill_tail(int64_t size);
|
||||
inline int compact();
|
||||
|
||||
TO_STRING_KV(KP_(data), K_(head), K_(tail), K_(cap));
|
||||
|
||||
private:
|
||||
char* data_;
|
||||
int64_t head_;
|
||||
int64_t tail_;
|
||||
int64_t cap_;
|
||||
};
|
||||
|
||||
class LinkNode : public common::ObDLinkBase<LinkNode> {};
|
||||
|
||||
struct StoreRow {
|
||||
StoreRow() : readable_(true), cnt_(0)
|
||||
{}
|
||||
|
||||
static int64_t row_copy_size(const common::ObNewRow& row);
|
||||
int copy_row(const common::ObNewRow& row, char* buf, const int64_t size);
|
||||
common::ObObj* cells()
|
||||
{
|
||||
return reinterpret_cast<common::ObObj*>(payload_);
|
||||
}
|
||||
const common::ObObj* cells() const
|
||||
{
|
||||
return reinterpret_cast<const common::ObObj*>(payload_);
|
||||
}
|
||||
|
||||
int to_copyable();
|
||||
int to_readable();
|
||||
|
||||
uint32_t readable_ : 1;
|
||||
uint32_t cnt_ : 31;
|
||||
char payload_[0];
|
||||
} __attribute__((packed));
|
||||
|
||||
struct Block {
|
||||
static const int64_t MAGIC = 0x35f4451b9b56eb12;
|
||||
typedef uint32_t row_idx_t;
|
||||
const static int64_t ROW_INDEX_SIZE = sizeof(row_idx_t);
|
||||
|
||||
Block() : magic_(MAGIC), row_id_(0), rows_(0), idx_off_(0)
|
||||
{}
|
||||
|
||||
static int64_t inline min_buf_size(const common::ObNewRow& row)
|
||||
{
|
||||
return sizeof(Block) + row_store_size(row);
|
||||
}
|
||||
static int64_t inline row_store_size(const common::ObNewRow& row)
|
||||
{
|
||||
return sizeof(StoreRow) + ROW_INDEX_SIZE + StoreRow::row_copy_size(row);
|
||||
}
|
||||
int add_row(ShrinkBuffer& buf, const common::ObNewRow& row, const int64_t row_size);
|
||||
int compact(ShrinkBuffer& buf);
|
||||
int to_copyable();
|
||||
const row_idx_t* indexes() const
|
||||
{
|
||||
return reinterpret_cast<const row_idx_t*>(payload_ + idx_off_);
|
||||
}
|
||||
inline bool contain(const int64_t row_id) const
|
||||
{
|
||||
return row_id_ <= row_id && row_id < row_id_ + rows_;
|
||||
}
|
||||
|
||||
int get_store_row(const int64_t row_id, const StoreRow*& sr);
|
||||
|
||||
TO_STRING_KV(K_(magic), K_(row_id), K_(rows), K_(idx_off));
|
||||
|
||||
int64_t magic_;
|
||||
int64_t row_id_;
|
||||
int32_t rows_;
|
||||
int32_t idx_off_;
|
||||
char payload_[0];
|
||||
} __attribute__((packed));
|
||||
|
||||
struct IndexBlock;
|
||||
struct BlockIndex {
|
||||
static bool compare(const BlockIndex& bi, const int64_t row_id)
|
||||
{
|
||||
return bi.row_id_ < row_id;
|
||||
}
|
||||
TO_STRING_KV(K_(is_idx_block), K_(on_disk), K_(row_id), K_(offset), K_(length));
|
||||
|
||||
uint64_t is_idx_block_ : 1;
|
||||
uint64_t on_disk_ : 1;
|
||||
uint64_t row_id_ : 62;
|
||||
union {
|
||||
IndexBlock* idx_blk_;
|
||||
Block* blk_;
|
||||
int64_t offset_;
|
||||
};
|
||||
int32_t length_;
|
||||
} __attribute__((packed));
|
||||
|
||||
struct IndexBlock {
|
||||
const static int64_t MAGIC = 0x4847bcb053c3703f;
|
||||
const static int64_t INDEX_BLOCK_SIZE = (64 << 10) - sizeof(LinkNode);
|
||||
constexpr static inline int64_t capacity()
|
||||
{
|
||||
return (INDEX_BLOCK_SIZE - sizeof(IndexBlock)) / sizeof(BlockIndex);
|
||||
}
|
||||
|
||||
IndexBlock() : magic_(MAGIC), cnt_(0)
|
||||
{}
|
||||
|
||||
inline int64_t buffer_size() const
|
||||
{
|
||||
return sizeof(*this) + sizeof(BlockIndex) * cnt_;
|
||||
}
|
||||
inline bool is_full() const
|
||||
{
|
||||
return cnt_ == capacity();
|
||||
}
|
||||
|
||||
// may return false when row in position (false negative),
|
||||
// since block index only contain start row_id, we can not detect right boundary.
|
||||
inline bool row_in_pos(const int64_t row_id, const int64_t pos);
|
||||
|
||||
void reset()
|
||||
{
|
||||
cnt_ = 0;
|
||||
}
|
||||
|
||||
TO_STRING_KV(K_(magic), K_(cnt));
|
||||
|
||||
int64_t magic_;
|
||||
int32_t cnt_;
|
||||
BlockIndex block_indexes_[0];
|
||||
} __attribute__((packed));
|
||||
|
||||
struct BlockBuf {
|
||||
BlockBuf() : buf_(), blk_()
|
||||
{}
|
||||
|
||||
void reset()
|
||||
{
|
||||
buf_.reset();
|
||||
blk_ = NULL;
|
||||
}
|
||||
ShrinkBuffer buf_;
|
||||
Block* blk_;
|
||||
};
|
||||
|
||||
// Reader must be deleted before ObRARowStore
|
||||
class Reader {
|
||||
friend class ObRARowStore;
|
||||
|
||||
public:
|
||||
explicit Reader(ObRARowStore& store) : store_(store), file_size_(0), idx_blk_(NULL), ib_pos_(0), blk_(NULL)
|
||||
{}
|
||||
virtual ~Reader()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
inline int64_t get_row_cnt() const
|
||||
{
|
||||
return store_.get_row_cnt();
|
||||
}
|
||||
// row memory will hold until the next get_row called or non-const method of ObRARowStore called
|
||||
int get_row(const int64_t row_id, const common::ObNewRow*& row);
|
||||
// only copy data to %row.cells_, keep other filed of %row unchanged.
|
||||
int get_row(const int64_t row_id, const common::ObNewRow& row);
|
||||
|
||||
void reset();
|
||||
void reuse();
|
||||
|
||||
private:
|
||||
void reset_cursor(const int64_t file_size);
|
||||
|
||||
private:
|
||||
ObRARowStore& store_;
|
||||
// idx_blk_, blk_ may point to the writing block,
|
||||
// we need to invalid the pointers if file_size_ change.
|
||||
int64_t file_size_;
|
||||
IndexBlock* idx_blk_;
|
||||
int64_t ib_pos_; // current block index position in index block
|
||||
Block* blk_;
|
||||
|
||||
ShrinkBuffer buf_;
|
||||
ShrinkBuffer idx_buf_;
|
||||
common::ObNewRow row_;
|
||||
|
||||
DISALLOW_COPY_AND_ASSIGN(Reader);
|
||||
};
|
||||
|
||||
public:
|
||||
const static int64_t BLOCK_SIZE = (64L << 10) - sizeof(LinkNode);
|
||||
const static int64_t BIG_BLOCK_SIZE = (256L << 10) - sizeof(LinkNode);
|
||||
// alloc first index block after store 1MB data.
|
||||
const static int64_t DEFAULT_BLOCK_CNT = (1L << 20) / BLOCK_SIZE;
|
||||
|
||||
explicit ObRARowStore(common::ObIAllocator* alloc = NULL, bool keep_projector_ = false);
|
||||
virtual ~ObRARowStore()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
int init(int64_t mem_limit, uint64_t tenant_id = common::OB_SERVER_TENANT_ID,
|
||||
int64_t mem_ctx_id = common::ObCtxIds::DEFAULT_CTX_ID, const char* label = common::ObModIds::OB_SQL_ROW_STORE);
|
||||
|
||||
void reset();
|
||||
// Keeping one memory block, reader must call reuse() too.
|
||||
void reuse();
|
||||
|
||||
int add_row(const common::ObNewRow& row);
|
||||
int finish_add_row();
|
||||
// row memory will hold until the next non-const method of ObRARowStore called.
|
||||
int get_row(const int64_t row_id, const common::ObNewRow*& row)
|
||||
{
|
||||
return inner_reader_.get_row(row_id, row);
|
||||
}
|
||||
// only copy data to %row.cells_, keep other filed of %row unchanged.
|
||||
int get_row(const int64_t row_id, const common::ObNewRow& row)
|
||||
{
|
||||
return inner_reader_.get_row(row_id, row);
|
||||
}
|
||||
|
||||
bool is_inited() const
|
||||
{
|
||||
return inited_;
|
||||
}
|
||||
bool is_file_open() const
|
||||
{
|
||||
return fd_ >= 0;
|
||||
}
|
||||
|
||||
void set_tenant_id(const uint64_t tenant_id)
|
||||
{
|
||||
tenant_id_ = tenant_id;
|
||||
}
|
||||
void set_mem_ctx_id(const int64_t ctx_id)
|
||||
{
|
||||
ctx_id_ = ctx_id;
|
||||
}
|
||||
void set_mem_limit(const int64_t limit)
|
||||
{
|
||||
mem_limit_ = limit;
|
||||
}
|
||||
|
||||
inline int64_t get_row_cnt() const
|
||||
{
|
||||
return row_cnt_;
|
||||
}
|
||||
inline int64_t get_mem_hold() const
|
||||
{
|
||||
return mem_hold_;
|
||||
}
|
||||
inline int64_t get_file_size() const
|
||||
{
|
||||
return file_size_;
|
||||
}
|
||||
|
||||
TO_STRING_KV(
|
||||
K_(tenant_id), K_(label), K_(ctx_id), K_(mem_limit), K_(save_row_cnt), K_(row_cnt), K_(fd), K_(file_size));
|
||||
|
||||
private:
|
||||
static int get_timeout(int64_t& timeout_ms);
|
||||
void* alloc_blk_mem(const int64_t size);
|
||||
void free_blk_mem(void* mem, const int64_t size = 0);
|
||||
int setup_block(BlockBuf& blkbuf) const;
|
||||
int alloc_block(BlockBuf& blkbuf, const int64_t min_size);
|
||||
// new block is not needed if %min_size is zero. (finish add row)
|
||||
int switch_block(const int64_t min_size);
|
||||
int add_block_idx(const BlockIndex& bi);
|
||||
int alloc_idx_block(IndexBlock*& ib);
|
||||
int build_idx_block();
|
||||
int switch_idx_block(bool finish_add = false);
|
||||
|
||||
int load_block(Reader& reader, const int64_t row_id);
|
||||
int load_idx_block(Reader& reader, IndexBlock*& ib, const BlockIndex& bi);
|
||||
int find_block_idx(Reader& reader, BlockIndex& bi, const int64_t row_id);
|
||||
|
||||
int get_store_row(Reader& reader, const int64_t row_id, const StoreRow*& sr);
|
||||
|
||||
int ensure_reader_buffer(ShrinkBuffer& buf, const int64_t size);
|
||||
|
||||
int write_file(BlockIndex& bi, void* buf, int64_t size);
|
||||
int read_file(void* buf, const int64_t size, const int64_t offset);
|
||||
|
||||
bool need_dump();
|
||||
|
||||
private:
|
||||
bool inited_;
|
||||
uint64_t tenant_id_;
|
||||
const char* label_;
|
||||
int64_t ctx_id_;
|
||||
int64_t mem_limit_;
|
||||
|
||||
BlockBuf blkbuf_;
|
||||
IndexBlock* idx_blk_;
|
||||
|
||||
int64_t save_row_cnt_;
|
||||
int64_t row_cnt_;
|
||||
|
||||
int64_t fd_;
|
||||
int64_t dir_id_;
|
||||
int64_t file_size_;
|
||||
|
||||
Reader inner_reader_;
|
||||
|
||||
common::ObDList<LinkNode> blk_mem_list_;
|
||||
common::ObSEArray<BlockIndex, DEFAULT_BLOCK_CNT> blocks_;
|
||||
|
||||
int64_t mem_hold_;
|
||||
common::DefaultPageAllocator inner_allocator_;
|
||||
common::ObIAllocator& allocator_;
|
||||
|
||||
bool keep_projector_;
|
||||
int32_t* projector_;
|
||||
int64_t projector_size_;
|
||||
|
||||
DISALLOW_COPY_AND_ASSIGN(ObRARowStore);
|
||||
};
|
||||
|
||||
inline int ObRARowStore::ShrinkBuffer::fill_head(int64_t size)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
if (size < -head_) {
|
||||
ret = common::OB_INVALID_ARGUMENT;
|
||||
SQL_ENG_LOG(WARN, "invalid argument", K(size), K_(head));
|
||||
} else if (size > remain()) {
|
||||
ret = common::OB_BUF_NOT_ENOUGH;
|
||||
SQL_ENG_LOG(WARN, "buffer not enough", K(size), "remain", remain());
|
||||
} else {
|
||||
head_ += size;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
inline int ObRARowStore::ShrinkBuffer::fill_tail(int64_t size)
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
if (size < -tail_size()) {
|
||||
ret = common::OB_INVALID_ARGUMENT;
|
||||
SQL_ENG_LOG(WARN, "invalid argument", K(size), "tail_size", tail_size());
|
||||
} else if (size > remain()) {
|
||||
ret = common::OB_BUF_NOT_ENOUGH;
|
||||
SQL_ENG_LOG(WARN, "buffer not enough", K(size), "remain", remain());
|
||||
} else {
|
||||
tail_ -= size;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
inline int ObRARowStore::ShrinkBuffer::compact()
|
||||
{
|
||||
int ret = common::OB_SUCCESS;
|
||||
if (!is_inited()) {
|
||||
ret = common::OB_NOT_INIT;
|
||||
SQL_ENG_LOG(WARN, "not inited", K(ret));
|
||||
} else {
|
||||
MEMMOVE(head(), tail(), tail_size());
|
||||
head_ += tail_size();
|
||||
tail_ = cap_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
inline bool ObRARowStore::IndexBlock::row_in_pos(const int64_t row_id, const int64_t pos)
|
||||
{
|
||||
bool in_pos = false;
|
||||
if (cnt_ > 0 && pos >= 0 && pos < cnt_) {
|
||||
if (pos + 1 == cnt_) {
|
||||
in_pos = block_indexes_[pos].row_id_ == row_id;
|
||||
} else {
|
||||
in_pos = block_indexes_[pos].row_id_ <= row_id && row_id < block_indexes_[pos + 1].row_id_;
|
||||
}
|
||||
}
|
||||
return in_pos;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_BASIC_OB_RA_ROW_STORE_H_
|
||||
427
src/sql/engine/basic/ob_select_into.cpp
Normal file
427
src/sql/engine/basic/ob_select_into.cpp
Normal file
@ -0,0 +1,427 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "ob_select_into.h"
|
||||
#include "lib/file/ob_file.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/expr/ob_sql_expression.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/engine/cmd/ob_variable_set_executor.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
class ObSelectInto::ObSelectIntoCtx : public ObPhyOperatorCtx {
|
||||
friend class ObSelectInto;
|
||||
|
||||
public:
|
||||
explicit ObSelectIntoCtx(ObExecContext& ctx)
|
||||
: ObPhyOperatorCtx(ctx), top_limit_cnt_(INT64_MAX), file_appender_(), is_first_(true)
|
||||
{}
|
||||
virtual ~ObSelectIntoCtx()
|
||||
{}
|
||||
|
||||
virtual void destroy()
|
||||
{
|
||||
file_appender_.~ObFileAppender();
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
|
||||
void reset()
|
||||
{
|
||||
is_first_ = true;
|
||||
file_appender_.close();
|
||||
}
|
||||
|
||||
private:
|
||||
int64_t top_limit_cnt_;
|
||||
ObFileAppender file_appender_;
|
||||
bool is_first_;
|
||||
ObObj filed_str_;
|
||||
ObObj line_str_;
|
||||
ObObj file_name_;
|
||||
};
|
||||
|
||||
ObSelectInto::ObSelectInto(ObIAllocator& alloc)
|
||||
: ObSingleChildPhyOperator(alloc),
|
||||
into_type_(T_INTO_OUTFILE),
|
||||
user_vars_(),
|
||||
outfile_name_(),
|
||||
filed_str_(),
|
||||
line_str_(),
|
||||
closed_cht_(0),
|
||||
is_optional_(false)
|
||||
{}
|
||||
|
||||
ObSelectInto::~ObSelectInto()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
void ObSelectInto::reset()
|
||||
{
|
||||
ObSingleChildPhyOperator::reset();
|
||||
user_vars_.reset();
|
||||
into_type_ = T_INTO_OUTFILE;
|
||||
}
|
||||
|
||||
void ObSelectInto::reuse()
|
||||
{
|
||||
ObSingleChildPhyOperator::reset();
|
||||
user_vars_.reuse();
|
||||
into_type_ = T_INTO_OUTFILE;
|
||||
}
|
||||
|
||||
int ObSelectInto::rescan(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(ctx);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSelectInto::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhyOperatorCtx* op_ctx = NULL;
|
||||
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObSelectIntoCtx, ctx, get_id(), get_type(), op_ctx))) {
|
||||
LOG_WARN("failed to create LimitCtx", K(ret));
|
||||
} else if (OB_ISNULL(op_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("op_ctx is null");
|
||||
} else if (OB_FAIL(init_cur_row(*op_ctx, need_copy_row_for_compute()))) {
|
||||
LOG_WARN("init current row failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSelectInto::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSelectIntoCtx* into_ctx = NULL;
|
||||
ObPhysicalPlanCtx* phy_plan_ctx = NULL;
|
||||
ObSQLSessionInfo* session = NULL;
|
||||
ObExprCtx expr_ctx;
|
||||
if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("init operator context failed", K(ret));
|
||||
} else if (OB_ISNULL(into_ctx = GET_PHY_OPERATOR_CTX(ObSelectIntoCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("opt_ctx is null", K(ret));
|
||||
} else if (OB_ISNULL(phy_plan_ctx = ctx.get_physical_plan_ctx())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get phy_plan_ctx failed", K(ret));
|
||||
} else if (OB_ISNULL(session = ctx.get_my_session())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get session failed", K(ret));
|
||||
} else {
|
||||
into_ctx->file_name_ = outfile_name_;
|
||||
into_ctx->filed_str_ = filed_str_;
|
||||
into_ctx->line_str_ = line_str_;
|
||||
}
|
||||
|
||||
int64_t row_count = 0;
|
||||
if (OB_SUCC(ret)) {
|
||||
bool need_check = false;
|
||||
const ObNewRow* row = NULL;
|
||||
if (OB_FAIL(ObSQLUtils::get_param_value(
|
||||
outfile_name_, phy_plan_ctx->get_param_store(), into_ctx->file_name_, need_check))) {
|
||||
LOG_WARN("get param value failed", K(ret));
|
||||
} else if (OB_FAIL(ObSQLUtils::get_param_value(
|
||||
filed_str_, phy_plan_ctx->get_param_store(), into_ctx->filed_str_, need_check))) {
|
||||
LOG_WARN("get param value failed", K(ret));
|
||||
} else if (OB_FAIL(ObSQLUtils::get_param_value(
|
||||
line_str_, phy_plan_ctx->get_param_store(), into_ctx->line_str_, need_check))) {
|
||||
LOG_WARN("get param value failed", K(ret));
|
||||
} else if (OB_FAIL(session->get_sql_select_limit(into_ctx->top_limit_cnt_))) {
|
||||
LOG_WARN("fail tp get sql select limit", K(ret));
|
||||
} else if (OB_FAIL(wrap_expr_ctx(ctx, into_ctx->expr_ctx_))) {
|
||||
LOG_WARN("wrap expr ctx failed", K(ret));
|
||||
} else if (OB_FAIL(init_cur_row(*into_ctx, true))) {
|
||||
LOG_WARN("fail to init cur row", K(ret));
|
||||
}
|
||||
|
||||
while (OB_SUCC(ret) && row_count < into_ctx->top_limit_cnt_) {
|
||||
if (OB_FAIL(get_next_row(ctx, row))) {
|
||||
if (OB_LIKELY(OB_ITER_END == ret)) {
|
||||
} else {
|
||||
LOG_WARN("get next row failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
LOG_DEBUG("copy cur row", K(*row));
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (T_INTO_VARIABLES == into_type_) {
|
||||
if (OB_FAIL(into_varlist(*row, into_ctx->expr_ctx_))) {
|
||||
LOG_WARN("into varlist failed", K(ret));
|
||||
}
|
||||
} else if (T_INTO_OUTFILE == into_type_) {
|
||||
if (OB_FAIL(into_outfile(*into_ctx, *row))) {
|
||||
LOG_WARN("into outfile failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(into_dumpfile(*into_ctx, *row))) {
|
||||
LOG_WARN("into dumpfile failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) { // if into user variables or into dumpfile, must be one row
|
||||
++row_count;
|
||||
if ((T_INTO_VARIABLES == into_type_ || T_INTO_DUMPFILE == into_type_) && row_count > 1) {
|
||||
ret = OB_ERR_TOO_MANY_ROWS;
|
||||
LOG_WARN("more than one row for into variables or into dumpfile", K(ret), K(row_count));
|
||||
}
|
||||
}
|
||||
} // end while
|
||||
|
||||
if (OB_ITER_END == ret) {
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret) || OB_ITER_END == ret) { // set affected rows
|
||||
phy_plan_ctx->set_affected_rows(row_count);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSelectInto::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSelectIntoCtx* into_ctx = NULL;
|
||||
if (OB_ISNULL(into_ctx = GET_PHY_OPERATOR_CTX(ObSelectIntoCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get into ctx failed", K(ret));
|
||||
} else if (OB_FAIL(child_op_->get_next_row(ctx, row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get next row failed", "op_type", ob_phy_operator_type_str(child_op_->get_type()), K(ret));
|
||||
} else {
|
||||
// OB_ITER_END
|
||||
}
|
||||
} else if (OB_ISNULL(row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("row is null", K(ret));
|
||||
} else if (OB_FAIL(copy_cur_row(*into_ctx, row))) {
|
||||
LOG_WARN("copy current row failed", K(ret));
|
||||
} else {
|
||||
LOG_DEBUG("succ inner get next row", K(*row));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
/*
|
||||
int ObSelectInto::get_next_row(ObExecContext &ctx, const ObNewRow *&row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSelectIntoCtx *into_ctx = NULL;
|
||||
ObPhysicalPlanCtx *phy_plan_ctx = NULL;
|
||||
int64_t row_count = 0;
|
||||
if (OB_ISNULL(into_ctx = GET_PHY_OPERATOR_CTX(ObSelectIntoCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get into ctx failed", K(ret));
|
||||
} else if (OB_ISNULL(phy_plan_ctx = ctx.get_physical_plan_ctx())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get plan ctx failed", K(ret));
|
||||
} else {
|
||||
// do nothing
|
||||
}
|
||||
|
||||
while (OB_SUCC(ret) && row_count < into_ctx->top_limit_cnt_) {
|
||||
if (OB_FAIL(child_op_->get_next_row(ctx, row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get next row failed", "op_type", ob_phy_operator_type_str(child_op_->get_type()), K(ret));
|
||||
} else {
|
||||
// OB_ITER_END
|
||||
}
|
||||
} else if (OB_ISNULL(row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("row is null", K(ret));
|
||||
} else if (T_INTO_VARIABLES == into_type_) {
|
||||
if (OB_FAIL(into_varlist(*row, into_ctx->expr_ctx_))) {
|
||||
LOG_WARN("into varlist failed", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(into_outfile(*into_ctx, *row))) {
|
||||
LOG_WARN("into outfile failed", K(ret));
|
||||
}
|
||||
if (OB_SUCC(ret)) { // if into user variables or into dumpfile, must be one row
|
||||
++row_count;
|
||||
if ((T_INTO_VARIABLES == into_type_ || T_INTO_DUMPFILE == into_type_) && row_count > 1) {
|
||||
ret = OB_ERR_TOO_MANY_ROWS;
|
||||
LOG_WARN("more than one row for into variables or into dumpfile", K(ret), K(row_count));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret) || OB_ITER_END == ret) { // set affected rows
|
||||
phy_plan_ctx->set_affected_rows(row_count);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
*/
|
||||
int ObSelectInto::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(ctx);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSelectInto::get_row_str(const ObNewRow& row, const int64_t buf_len, bool is_first_row, char* buf, int64_t& pos,
|
||||
ObSelectIntoCtx& into_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObObj& filed_str = into_ctx.filed_str_;
|
||||
if (!is_first_row && into_ctx.line_str_.is_varying_len_char_type()) { // lines terminated by "a"
|
||||
ret = databuff_printf(
|
||||
buf, buf_len, pos, "%.*s", into_ctx.line_str_.get_varchar().length(), into_ctx.line_str_.get_varchar().ptr());
|
||||
}
|
||||
for (int i = 0; OB_SUCC(ret) && i < row.get_count(); i++) {
|
||||
const ObObj& cell = row.get_cell(i);
|
||||
if (0 != closed_cht_ && (!is_optional_ || cell.is_string_type())) {
|
||||
// closed by "a" (for all cell) or optionally by "a" (for string cell)
|
||||
if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%c", closed_cht_))) {
|
||||
LOG_WARN("print closed character failed", K(ret), K(closed_cht_));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(cell.print_plain_str_literal(buf, buf_len, pos))) { // cell value
|
||||
LOG_WARN("print sql failed", K(ret), K(cell));
|
||||
} else if (0 != closed_cht_ && (!is_optional_ || cell.is_string_type())) {
|
||||
if (OB_FAIL(databuff_printf(buf, buf_len, pos, "%c", closed_cht_))) {
|
||||
LOG_WARN("print closed character failed", K(ret), K(closed_cht_));
|
||||
}
|
||||
}
|
||||
// filed terminated by "a"
|
||||
if (OB_SUCC(ret) && i != row.get_count() - 1 && filed_str.is_varying_len_char_type()) {
|
||||
if (OB_FAIL(databuff_printf(
|
||||
buf, buf_len, pos, "%.*s", filed_str.get_varchar().length(), filed_str.get_varchar().ptr()))) {
|
||||
LOG_WARN("print filed str failed", K(ret), K(filed_str));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSelectInto::into_outfile(ObSelectIntoCtx& into_ctx, const ObNewRow& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (row.get_count() != 1) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid row in select into outfile", K(row.get_count()), K(ret));
|
||||
} else if (into_ctx.is_first_) { // create file
|
||||
if (OB_FAIL(into_ctx.file_appender_.create(into_ctx.file_name_.get_varchar(), true))) {
|
||||
LOG_WARN("create dumpfile failed", K(ret), K(into_ctx.file_name_));
|
||||
} else {
|
||||
into_ctx.is_first_ = false;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
const ObObj& cell = row.get_cell(0);
|
||||
if (OB_FAIL(into_ctx.file_appender_.append(cell.get_string_ptr(), cell.get_string_len(), false))) {
|
||||
LOG_WARN("failed to append file");
|
||||
} else {
|
||||
// do nothing
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSelectInto::into_dumpfile(ObSelectIntoCtx& into_ctx, const ObNewRow& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
char buf[MAX_VALUE_LENGTH];
|
||||
int64_t buf_len = MAX_VALUE_LENGTH;
|
||||
int64_t pos = 0;
|
||||
if (OB_FAIL(get_row_str(row, buf_len, into_ctx.is_first_, buf, pos, into_ctx))) {
|
||||
LOG_WARN("get row str failed", K(ret));
|
||||
} else if (into_ctx.is_first_) { // create file
|
||||
if (OB_FAIL(into_ctx.file_appender_.create(into_ctx.file_name_.get_varchar(), true))) {
|
||||
LOG_WARN("create dumpfile failed", K(ret), K(into_ctx.file_name_));
|
||||
} else {
|
||||
into_ctx.is_first_ = false;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(into_ctx.file_appender_.append(buf, pos, false))) {
|
||||
LOG_WARN("failed to append file");
|
||||
} else {
|
||||
// do nothing
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSelectInto::into_varlist(const ObNewRow& row, const ObExprCtx& expr_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (row.get_count() != user_vars_.count()) {
|
||||
ret = OB_ERR_COLUMN_SIZE;
|
||||
LOG_WARN("user vars count should be equal to cell count", K(row.get_count()), K(user_vars_.count()));
|
||||
} else {
|
||||
for (int i = 0; i < user_vars_.count(); ++i) {
|
||||
const ObString& var_name = user_vars_.at(i);
|
||||
if (OB_FAIL(ObVariableSetExecutor::set_user_variable(row.get_cell(i), var_name, expr_ctx))) {
|
||||
LOG_WARN("set user variable failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSelectInto::set_user_vars(const ObIArray<common::ObString>& user_vars, ObIAllocator& phy_alloc)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObString var;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < user_vars.count(); ++i) {
|
||||
var.reset();
|
||||
if (OB_FAIL(ob_write_string(phy_alloc, user_vars.at(i), var))) {
|
||||
LOG_WARN("fail to deep copy string", K(user_vars.at(i)), K(ret));
|
||||
} else if (OB_FAIL(user_vars_.push_back(var))) {
|
||||
LOG_WARN("fail to push back var", K(var), K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSelectInto::set_outfile_name(const common::ObObj& outfile_name, common::ObIAllocator& phy_alloc)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(deep_copy_obj(phy_alloc, outfile_name, outfile_name_))) {
|
||||
LOG_WARN("fail to deep copy obj", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSelectInto::set_filed_str(const common::ObObj& filed_str, common::ObIAllocator& phy_alloc)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(deep_copy_obj(phy_alloc, filed_str, filed_str_))) {
|
||||
LOG_WARN("fail to deep copy obj", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSelectInto::set_line_str(const common::ObObj& line_str, common::ObIAllocator& phy_alloc)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(deep_copy_obj(phy_alloc, line_str, line_str_))) {
|
||||
LOG_WARN("fail to deep copy obj", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObSelectInto, ObSingleChildPhyOperator), into_type_, closed_cht_, is_optional_, outfile_name_,
|
||||
filed_str_, line_str_, user_vars_);
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
105
src/sql/engine/basic/ob_select_into.h
Normal file
105
src/sql/engine/basic/ob_select_into.h
Normal file
@ -0,0 +1,105 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef SRC_SQL_ENGINE_BASIC_OB_SELECT_INTO_H_
|
||||
#define SRC_SQL_ENGINE_BASIC_OB_SELECT_INTO_H_
|
||||
|
||||
#include "sql/engine/ob_single_child_phy_operator.h"
|
||||
#include "sql/engine/expr/ob_expr_operator.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/parser/ob_item_type.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObSelectInto : public ObSingleChildPhyOperator {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
private:
|
||||
class ObSelectIntoCtx;
|
||||
|
||||
public:
|
||||
explicit ObSelectInto(common::ObIAllocator& alloc);
|
||||
virtual ~ObSelectInto();
|
||||
virtual void reset();
|
||||
virtual void reuse();
|
||||
virtual int rescan(ObExecContext& ctx) const;
|
||||
|
||||
void set_into_type(ObItemType into_type)
|
||||
{
|
||||
into_type_ = into_type;
|
||||
}
|
||||
|
||||
int set_user_vars(const common::ObIArray<common::ObString>& user_vars, common::ObIAllocator& phy_alloc);
|
||||
|
||||
int set_outfile_name(const common::ObObj& outfile_name, common::ObIAllocator& phy_alloc);
|
||||
|
||||
int set_filed_str(const common::ObObj& filed_str, common::ObIAllocator& phy_alloc);
|
||||
|
||||
int set_line_str(const common::ObObj& line_str, common::ObIAllocator& phy_alloc);
|
||||
|
||||
void set_is_optional(bool is_optional)
|
||||
{
|
||||
is_optional_ = is_optional;
|
||||
}
|
||||
void set_closed_cht(char closed_cht)
|
||||
{
|
||||
closed_cht_ = closed_cht;
|
||||
}
|
||||
|
||||
private:
|
||||
/**
|
||||
* @brief init operator context, will create a physical operator context (and a current row space)
|
||||
* @param ctx[in], execute context
|
||||
* @return if success, return OB_SUCCESS, otherwise, return errno
|
||||
*/
|
||||
virtual int init_op_ctx(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief: called by get_next_row(), get a row from the child operator or row_store
|
||||
* @param: ctx[in], execute context
|
||||
* @param: row[out], ObSqlRow an obj array and row_size
|
||||
*/
|
||||
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
/**
|
||||
* @brief open operator, not including children operators.
|
||||
* called by open.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_open(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief close operator, not including children operators.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_close(ObExecContext& ctx) const;
|
||||
// int get_next_row(ObExecContext &ctx, const ObNewRow *&row) const;
|
||||
// into outfile 'a.test'
|
||||
int into_outfile(ObSelectIntoCtx& into_ctx, const common::ObNewRow& row) const;
|
||||
// into dumpfile 'a.test'
|
||||
int into_dumpfile(ObSelectIntoCtx& into_ctx, const common::ObNewRow& row) const;
|
||||
// into @a,@b
|
||||
int into_varlist(const common::ObNewRow& row, const ObExprCtx& expr_ctx) const;
|
||||
int get_row_str(const common::ObNewRow& row, const int64_t buf_len, bool is_first_row, char* buf, int64_t& pos,
|
||||
ObSelectIntoCtx& into_ctx) const;
|
||||
DISALLOW_COPY_AND_ASSIGN(ObSelectInto);
|
||||
|
||||
private:
|
||||
ObItemType into_type_;
|
||||
common::ObSEArray<common::ObString, 16> user_vars_;
|
||||
common::ObObj outfile_name_;
|
||||
common::ObObj filed_str_;
|
||||
common::ObObj line_str_;
|
||||
char closed_cht_;
|
||||
bool is_optional_;
|
||||
};
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
#endif /* SRC_SQL_ENGINE_BASIC_OB_SELECT_INTO_H_ */
|
||||
31
src/sql/engine/basic/ob_sql_mem_callback.h
Normal file
31
src/sql/engine/basic/ob_sql_mem_callback.h
Normal file
@ -0,0 +1,31 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_SQL_MEMM_CALLBACK_H_
|
||||
#define OCEANBASE_BASIC_OB_SQL_MEMM_CALLBACK_H_
|
||||
|
||||
#include "share/ob_define.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObSqlMemoryCallback {
|
||||
public:
|
||||
virtual void alloc(int64_t size) = 0;
|
||||
virtual void free(int64_t size) = 0;
|
||||
virtual void dumped(int64_t size) = 0;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_BASIC_OB_SQL_MEMM_CALLBACK_H_
|
||||
422
src/sql/engine/basic/ob_temp_table_access.cpp
Normal file
422
src/sql/engine/basic/ob_temp_table_access.cpp
Normal file
@ -0,0 +1,422 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "sql/engine/basic/ob_temp_table_access.h"
|
||||
#include "sql/dtl/ob_dtl_interm_result_manager.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
ObTempTableAccessInput::ObTempTableAccessInput()
|
||||
: deserialize_allocator_(NULL), closed_count_(0), unfinished_count_ptr_(0), interm_result_ids_()
|
||||
{}
|
||||
|
||||
ObTempTableAccessInput::~ObTempTableAccessInput()
|
||||
{
|
||||
deserialize_allocator_ = NULL;
|
||||
unfinished_count_ptr_ = 0;
|
||||
closed_count_ = 0;
|
||||
interm_result_ids_.reset();
|
||||
}
|
||||
|
||||
void ObTempTableAccessInput::reset()
|
||||
{
|
||||
deserialize_allocator_ = NULL;
|
||||
unfinished_count_ptr_ = 0;
|
||||
closed_count_ = 0;
|
||||
interm_result_ids_.reset();
|
||||
}
|
||||
|
||||
ObPhyOperatorType ObTempTableAccessInput::get_phy_op_type() const
|
||||
{
|
||||
return PHY_TEMP_TABLE_ACCESS;
|
||||
}
|
||||
|
||||
void ObTempTableAccessInput::set_deserialize_allocator(common::ObIAllocator* allocator)
|
||||
{
|
||||
deserialize_allocator_ = allocator;
|
||||
}
|
||||
|
||||
int ObTempTableAccessInput::init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(ctx);
|
||||
UNUSED(task_info);
|
||||
UNUSED(op);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccessInput::check_closed_finish(bool& is_end)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
uint64_t closed_cnt = 0;
|
||||
uint64_t* count_ptr = reinterpret_cast<uint64_t*>(closed_count_);
|
||||
if (NULL != count_ptr) {
|
||||
closed_cnt = ATOMIC_AAF(count_ptr, -1);
|
||||
if (0 >= closed_cnt) {
|
||||
is_end = true;
|
||||
} else {
|
||||
is_end = false;
|
||||
}
|
||||
} else {
|
||||
is_end = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccessInput::check_finish(bool& is_end, int64_t& index)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
uint64_t* count_ptr = reinterpret_cast<uint64_t*>(unfinished_count_ptr_);
|
||||
if (NULL != count_ptr) {
|
||||
index = ATOMIC_AAF(count_ptr, -1);
|
||||
if (-1 >= index) {
|
||||
is_end = true;
|
||||
} else {
|
||||
is_end = false;
|
||||
}
|
||||
} else {
|
||||
is_end = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccessInput::assign_ids(common::ObIArray<uint64_t>& interm_res_ids)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(interm_result_ids_.assign(interm_res_ids))) {
|
||||
LOG_WARN("failed to assign to interm result ids.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE(ObTempTableAccessInput)
|
||||
{
|
||||
int ret = OK_;
|
||||
UNF_UNUSED_SER;
|
||||
BASE_SER(ObTempTableAccessInput);
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, closed_count_);
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, unfinished_count_ptr_);
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(serialization::encode_vi64(buf, buf_len, pos, interm_result_ids_.count()))) {
|
||||
LOG_WARN("fail to encode key ranges count", K(ret), K(interm_result_ids_));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < interm_result_ids_.count(); ++i) {
|
||||
if (OB_FAIL(serialization::encode_vi64(buf, buf_len, pos, interm_result_ids_.at(i)))) {
|
||||
LOG_WARN("fail to serialize key range", K(ret), K(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObTempTableAccessInput)
|
||||
{
|
||||
int ret = OK_;
|
||||
UNF_UNUSED_DES;
|
||||
BASE_DESER(ObTempTableAccessInput);
|
||||
LST_DO_CODE(OB_UNIS_DECODE, closed_count_);
|
||||
LST_DO_CODE(OB_UNIS_DECODE, unfinished_count_ptr_);
|
||||
if (OB_SUCC(ret)) {
|
||||
int64_t count = 0;
|
||||
interm_result_ids_.reset();
|
||||
if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &count))) {
|
||||
LOG_WARN("fail to decode key ranges count", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < count; i++) {
|
||||
int64_t interm_result_id = 0;
|
||||
if (OB_ISNULL(deserialize_allocator_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("deserialize allocator is NULL", K(ret));
|
||||
} else if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &interm_result_id))) {
|
||||
LOG_WARN("failed to decode vi64.", K(ret));
|
||||
} else if (OB_FAIL(interm_result_ids_.push_back(interm_result_id))) {
|
||||
LOG_WARN("failed to push back into interm result ids.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE_SIZE(ObTempTableAccessInput)
|
||||
{
|
||||
int64_t len = 0;
|
||||
BASE_ADD_LEN(ObTempTableAccessInput);
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, closed_count_);
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, unfinished_count_ptr_);
|
||||
len += serialization::encoded_length_vi64(interm_result_ids_.count());
|
||||
for (int64_t i = 0; i < interm_result_ids_.count(); ++i) {
|
||||
len += serialization::encoded_length_vi64(interm_result_ids_.at(i));
|
||||
}
|
||||
return len;
|
||||
}
|
||||
|
||||
int ObTempTableAccess::ObTempTableAccessCtx::locate_interm_result(dtl::ObDTLIntermResultKey& dtl_int_key)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
dtl::ObDTLIntermResultInfo result_info;
|
||||
if (OB_FAIL(dtl::ObDTLIntermResultManager::getInstance().get_interm_result_info(dtl_int_key, result_info))) {
|
||||
LOG_WARN("failed to create row store.", K(ret));
|
||||
} else if (FALSE_IT(row_store_ = result_info.row_store_)) {
|
||||
} else if (OB_FAIL(row_store_->begin(row_store_it_))) {
|
||||
LOG_WARN("failed to begin chunk row store.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObTempTableAccess::reset()
|
||||
{
|
||||
ObNoChildrenPhyOperator::reset();
|
||||
}
|
||||
|
||||
void ObTempTableAccess::reuse()
|
||||
{
|
||||
ObNoChildrenPhyOperator::reuse();
|
||||
}
|
||||
|
||||
int ObTempTableAccess::rescan(ObExecContext& exec_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(exec_ctx);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccess::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("failed to init op ctx.", K(ret));
|
||||
} else if (OB_FAIL(handle_op_ctx(ctx))) {
|
||||
LOG_WARN("failed to handle operator context.", K(ret));
|
||||
} else { /* do nothing. */
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccess::destory_interm_results(ObExecContext& exec_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool is_end = false;
|
||||
ObTempTableAccessCtx* tta_ctx = NULL;
|
||||
ObTempTableAccessInput* tta_input = NULL;
|
||||
if (OB_ISNULL(tta_ctx = GET_PHY_OPERATOR_CTX(ObTempTableAccessCtx, exec_ctx, get_id()))) {
|
||||
LOG_WARN("fail to create phy op ctx", K(ret), K(get_id()), K(get_type()));
|
||||
} else if (OB_ISNULL(tta_input = tta_ctx->tta_input_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("temp table access input is null.", K(ret));
|
||||
} else if (is_distributed_) {
|
||||
if (!need_release_) {
|
||||
// do nothing.
|
||||
} else if (OB_FAIL(tta_input->check_closed_finish(is_end))) {
|
||||
LOG_WARN("failed to check closed finish.", K(ret));
|
||||
} else if (is_end) {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < tta_input->interm_result_ids_.count(); ++i) {
|
||||
dtl::ObDTLIntermResultKey dtl_int_key;
|
||||
dtl_int_key.channel_id_ = tta_input->interm_result_ids_.at(i);
|
||||
if (OB_FAIL(dtl::ObDTLIntermResultManager::getInstance().erase_interm_result_info(dtl_int_key))) {
|
||||
LOG_WARN("failed to erase interm result info in manager.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
}
|
||||
} else if (need_release_) {
|
||||
dtl::ObDTLIntermResultKey dtl_int_key;
|
||||
dtl_int_key.channel_id_ = temp_table_id_;
|
||||
if (OB_FAIL(dtl::ObDTLIntermResultManager::getInstance().erase_interm_result_info(dtl_int_key))) {
|
||||
LOG_WARN("failed to erase interm result info in manager.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccess::inner_close(ObExecContext& exec_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(destory_interm_results(exec_ctx))) {
|
||||
LOG_WARN("failed to destory interm results.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccess::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObTempTableAccessCtx* op_ctx = NULL;
|
||||
ObTempTableAccessInput* op_input = NULL;
|
||||
if (OB_UNLIKELY(filter_exprs_.get_size() > 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("filter exprs should be empty", K(ret), K(filter_exprs_.get_size()));
|
||||
} else if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObTempTableAccessCtx, ctx, get_id(), get_type(), op_ctx))) {
|
||||
LOG_WARN("fail to create phy op ctx", K(ret), K(get_id()), K(get_type()));
|
||||
} else if (OB_ISNULL(op_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("op ctx is NULL", K(ret));
|
||||
} else if (OB_FAIL(init_cur_row(*op_ctx, true))) {
|
||||
LOG_WARN("fail to int cur row", K(ret));
|
||||
} else if (OB_ISNULL(op_input = GET_PHY_OP_INPUT(ObTempTableAccessInput, ctx, get_id()))) {
|
||||
if (OB_FAIL(create_operator_input(ctx))) {
|
||||
LOG_WARN("fail to create operator input", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_ISNULL(op_input = GET_PHY_OP_INPUT(ObTempTableAccessInput, ctx, get_id()))) {
|
||||
LOG_WARN("failed to get op input.", K(ret));
|
||||
} else {
|
||||
op_ctx->tta_input_ = op_input;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccess::construct_row_from_interm_row(
|
||||
ObExecContext& ctx, ObNewRow* input_row, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObTempTableAccessCtx* tta_ctx = NULL;
|
||||
if (OB_ISNULL(tta_ctx = GET_PHY_OPERATOR_CTX(ObTempTableAccessCtx, ctx, get_id()))) {
|
||||
LOG_WARN("fail to create phy op ctx", K(ret), K(get_id()), K(get_type()));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < output_indexs_.count(); ++i) {
|
||||
int64_t col_idx = output_indexs_.at(i);
|
||||
if (OB_UNLIKELY(i >= tta_ctx->get_cur_row().count_) || OB_UNLIKELY(col_idx >= input_row->get_count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("row not match", K(i), K(*input_row));
|
||||
} else {
|
||||
tta_ctx->get_cur_row().cells_[i] = input_row->get_cell(col_idx);
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
row = &(tta_ctx->get_cur_row());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccess::inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool is_end = false;
|
||||
common::ObNewRow* input_row = NULL;
|
||||
ObTempTableAccessCtx* tta_ctx = NULL;
|
||||
if (OB_ISNULL(tta_ctx = GET_PHY_OPERATOR_CTX(ObTempTableAccessCtx, ctx, get_id()))) {
|
||||
LOG_WARN("fail to create phy op ctx", K(ret), K(get_id()), K(get_type()));
|
||||
} else if (!tta_ctx->is_started_ && OB_FAIL(locate_next_interm_result(ctx, is_end))) {
|
||||
LOG_WARN("failed to locate next iterm result.", K(ret));
|
||||
} else if (!tta_ctx->is_started_ && is_end) {
|
||||
ret = OB_ITER_END;
|
||||
} else if (OB_FAIL(tta_ctx->row_store_it_.get_next_row(input_row))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
if (OB_FAIL(locate_next_interm_result(ctx, is_end))) {
|
||||
LOG_WARN("failed to locate next iterm result.", K(ret));
|
||||
} else if (is_end) {
|
||||
ret = OB_ITER_END;
|
||||
} else if (OB_FAIL(tta_ctx->row_store_it_.get_next_row(input_row))) {
|
||||
LOG_WARN("failed to get next row.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
} else {
|
||||
LOG_WARN("failed to get next row.", K(ret));
|
||||
}
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(construct_row_from_interm_row(ctx, input_row, row))) {
|
||||
LOG_WARN("failed to construct row from interm row.", K(ret));
|
||||
} else {
|
||||
tta_ctx->is_started_ = true;
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
if (OB_ITER_END != ret && OB_FAIL(destory_interm_results(ctx))) {
|
||||
LOG_WARN("failed to destory interm results.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccess::get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ret = ObPhyOperator::get_next_row(ctx, row);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccess::create_operator_input(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObIPhyOperatorInput* input = NULL;
|
||||
if (OB_FAIL(CREATE_PHY_OP_INPUT(ObTempTableAccessInput, ctx, get_id(), get_type(), input))) {
|
||||
LOG_WARN("failed to create phy op input.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
UNUSED(input);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccess::locate_next_interm_result(ObExecContext& ctx, bool& is_end) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t index = 0;
|
||||
ObTempTableAccessCtx* tta_ctx = NULL;
|
||||
dtl::ObDTLIntermResultKey dtl_int_key;
|
||||
if (OB_ISNULL(tta_ctx = GET_PHY_OPERATOR_CTX(ObTempTableAccessCtx, ctx, get_id()))) {
|
||||
LOG_WARN("fail to create phy op ctx", K(ret), K(get_id()), K(get_type()));
|
||||
} else if (!is_distributed_) {
|
||||
if (!tta_ctx->is_started_) {
|
||||
dtl_int_key.channel_id_ = temp_table_id_;
|
||||
if (OB_FAIL(tta_ctx->locate_interm_result(dtl_int_key))) {
|
||||
LOG_WARN("failed to locate interm result.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
} else {
|
||||
is_end = true;
|
||||
}
|
||||
} else if (OB_FAIL(tta_ctx->tta_input_->check_finish(is_end, index))) {
|
||||
LOG_WARN("failed to check finish.", K(ret));
|
||||
} else if (is_end) {
|
||||
/*do nothing.*/
|
||||
} else {
|
||||
dtl_int_key.channel_id_ = tta_ctx->tta_input_->interm_result_ids_.at(index);
|
||||
if (OB_FAIL(tta_ctx->locate_interm_result(dtl_int_key))) {
|
||||
LOG_WARN("failed to locate interm result.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_SERIALIZE_MEMBER(
|
||||
(ObTempTableAccess, ObNoChildrenPhyOperator), output_indexs_, temp_table_id_, is_distributed_, need_release_);
|
||||
|
||||
int64_t ObTempTableAccess::to_string(char* buf, const int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
J_OBJ_START();
|
||||
J_KV("output_index", output_indexs_);
|
||||
J_OBJ_END();
|
||||
return pos;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
151
src/sql/engine/basic/ob_temp_table_access.h
Normal file
151
src/sql/engine/basic/ob_temp_table_access.h
Normal file
@ -0,0 +1,151 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_ACCESS_H_
|
||||
#define OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_ACCESS_H_
|
||||
|
||||
#include "sql/engine/ob_phy_operator.h"
|
||||
#include "sql/engine/ob_no_children_phy_operator.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/engine/ob_single_child_phy_operator.h"
|
||||
#include "sql/engine/basic/ob_ra_row_store.h"
|
||||
#include "sql/engine/basic/ob_chunk_row_store.h"
|
||||
#include "sql/engine/ob_sql_mem_mgr_processor.h"
|
||||
#include "sql/engine/ob_tenant_sql_memory_manager.h"
|
||||
#include "sql/executor/ob_interm_result_manager.h"
|
||||
#include "sql/dtl/ob_dtl_interm_result_manager.h"
|
||||
#include "sql/engine/ob_physical_plan_ctx.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObExecContext;
|
||||
class ObTempTableAccessInput : public ObIPhyOperatorInput {
|
||||
friend class ObTempTableAccess;
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObTempTableAccessInput();
|
||||
virtual ~ObTempTableAccessInput();
|
||||
virtual void reset() override;
|
||||
virtual ObPhyOperatorType get_phy_op_type() const;
|
||||
virtual void set_deserialize_allocator(common::ObIAllocator* allocator);
|
||||
virtual int init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op);
|
||||
int check_finish(bool& is_end, int64_t& index);
|
||||
int check_closed_finish(bool& is_end);
|
||||
int assign_ids(common::ObIArray<uint64_t>& interm_res_ids);
|
||||
|
||||
protected:
|
||||
common::ObIAllocator* deserialize_allocator_;
|
||||
DISALLOW_COPY_AND_ASSIGN(ObTempTableAccessInput);
|
||||
|
||||
public:
|
||||
uint64_t closed_count_;
|
||||
uint64_t unfinished_count_ptr_;
|
||||
common::ObSEArray<uint64_t, 8> interm_result_ids_;
|
||||
};
|
||||
|
||||
class ObTempTableAccess : public ObNoChildrenPhyOperator {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
protected:
|
||||
class ObTempTableAccessCtx : public ObPhyOperatorCtx {
|
||||
friend class ObTempTableAccess;
|
||||
|
||||
public:
|
||||
explicit ObTempTableAccessCtx(ObExecContext& ctx)
|
||||
: ObPhyOperatorCtx(ctx), row_store_(NULL), tta_input_(NULL), is_started_(false)
|
||||
{}
|
||||
virtual ~ObTempTableAccessCtx()
|
||||
{}
|
||||
virtual void destroy()
|
||||
{
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
int locate_interm_result(dtl::ObDTLIntermResultKey& dtl_int_key);
|
||||
|
||||
private:
|
||||
ObChunkRowStore* row_store_;
|
||||
ObChunkRowStore::Iterator row_store_it_;
|
||||
ObTempTableAccessInput* tta_input_;
|
||||
bool is_started_;
|
||||
};
|
||||
|
||||
public:
|
||||
ObTempTableAccess(common::ObIAllocator& alloc)
|
||||
: ObNoChildrenPhyOperator(alloc),
|
||||
output_indexs_(alloc),
|
||||
temp_table_id_(0),
|
||||
is_distributed_(false),
|
||||
need_release_(false)
|
||||
{}
|
||||
virtual ~ObTempTableAccess()
|
||||
{}
|
||||
void reset();
|
||||
void reuse();
|
||||
int rescan(ObExecContext& exec_ctx) const;
|
||||
|
||||
DECLARE_VIRTUAL_TO_STRING;
|
||||
virtual int inner_open(ObExecContext& exec_ctx) const;
|
||||
virtual int inner_close(ObExecContext& exec_ctx) const;
|
||||
virtual int init_op_ctx(ObExecContext& ctx) const;
|
||||
virtual int inner_get_next_row(ObExecContext& exec_ctx, const common::ObNewRow*& row) const;
|
||||
int get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
int create_operator_input(ObExecContext& ctx) const;
|
||||
int locate_next_interm_result(ObExecContext& ctx, bool& is_end) const;
|
||||
int destory_interm_results(ObExecContext& exec_ctx) const;
|
||||
int construct_row_from_interm_row(ObExecContext& ctx, ObNewRow* input_row, const ObNewRow*& row) const;
|
||||
void set_temp_table_id(uint64_t temp_table_id)
|
||||
{
|
||||
temp_table_id_ = temp_table_id;
|
||||
}
|
||||
uint64_t get_table_id() const
|
||||
{
|
||||
return temp_table_id_;
|
||||
}
|
||||
void set_distributed(bool is_distributed)
|
||||
{
|
||||
is_distributed_ = is_distributed;
|
||||
}
|
||||
bool is_distributed() const
|
||||
{
|
||||
return is_distributed_;
|
||||
}
|
||||
void set_need_release(bool need_release)
|
||||
{
|
||||
need_release_ = need_release;
|
||||
}
|
||||
bool is_need_release() const
|
||||
{
|
||||
return need_release_;
|
||||
}
|
||||
int add_output_index(int64_t index)
|
||||
{
|
||||
return output_indexs_.push_back(index);
|
||||
}
|
||||
int init_output_index(int64_t count)
|
||||
{
|
||||
return init_array_size<>(output_indexs_, count);
|
||||
}
|
||||
|
||||
DISALLOW_COPY_AND_ASSIGN(ObTempTableAccess);
|
||||
|
||||
private:
|
||||
common::ObFixedArray<int64_t, common::ObIAllocator> output_indexs_;
|
||||
uint64_t temp_table_id_;
|
||||
bool is_distributed_;
|
||||
bool need_release_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_ACCESS_H_ */
|
||||
396
src/sql/engine/basic/ob_temp_table_access_op.cpp
Normal file
396
src/sql/engine/basic/ob_temp_table_access_op.cpp
Normal file
@ -0,0 +1,396 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "ob_temp_table_access_op.h"
|
||||
#include "sql/engine/ob_operator_reg.h"
|
||||
#include "sql/dtl/ob_dtl_interm_result_manager.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
using namespace storage;
|
||||
using namespace share;
|
||||
using namespace share::schema;
|
||||
namespace sql {
|
||||
|
||||
#define USE_MULTI_GET_ARRAY_BINDING 1
|
||||
|
||||
ObTempTableAccessOpInput::ObTempTableAccessOpInput(ObExecContext& ctx, const ObOpSpec& spec)
|
||||
: ObOpInput(ctx, spec),
|
||||
deserialize_allocator_(NULL),
|
||||
closed_count_(0),
|
||||
unfinished_count_ptr_(0),
|
||||
interm_result_ids_()
|
||||
{}
|
||||
|
||||
ObTempTableAccessOpInput::~ObTempTableAccessOpInput()
|
||||
{
|
||||
deserialize_allocator_ = NULL;
|
||||
closed_count_ = 0;
|
||||
unfinished_count_ptr_ = 0;
|
||||
interm_result_ids_.reset();
|
||||
}
|
||||
|
||||
void ObTempTableAccessOpInput::reset()
|
||||
{
|
||||
deserialize_allocator_ = NULL;
|
||||
closed_count_ = 0;
|
||||
unfinished_count_ptr_ = 0;
|
||||
interm_result_ids_.reset();
|
||||
}
|
||||
|
||||
ObPhyOperatorType ObTempTableAccessOpInput::get_phy_op_type() const
|
||||
{
|
||||
return PHY_TEMP_TABLE_ACCESS;
|
||||
}
|
||||
|
||||
void ObTempTableAccessOpInput::set_deserialize_allocator(common::ObIAllocator* allocator)
|
||||
{
|
||||
deserialize_allocator_ = allocator;
|
||||
}
|
||||
|
||||
int ObTempTableAccessOpInput::init(ObTaskInfo& task_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(task_info);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccessOpInput::init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(ctx);
|
||||
UNUSED(task_info);
|
||||
UNUSED(op);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccessOpInput::check_finish(bool& is_end, int64_t& index)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
uint64_t* count_ptr = reinterpret_cast<uint64_t*>(unfinished_count_ptr_);
|
||||
if (NULL != count_ptr) {
|
||||
index = ATOMIC_AAF(count_ptr, -1);
|
||||
if (-1 >= index) {
|
||||
is_end = true;
|
||||
} else {
|
||||
is_end = false;
|
||||
}
|
||||
} else {
|
||||
is_end = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccessOpInput::check_closed_finish(bool& is_end)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
uint64_t closed_cnt = 0;
|
||||
uint64_t* count_ptr = reinterpret_cast<uint64_t*>(closed_count_);
|
||||
if (NULL != count_ptr) {
|
||||
closed_cnt = ATOMIC_AAF(count_ptr, -1);
|
||||
if (0 >= closed_cnt) {
|
||||
is_end = true;
|
||||
} else {
|
||||
is_end = false;
|
||||
}
|
||||
} else {
|
||||
is_end = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE(ObTempTableAccessOpInput)
|
||||
{
|
||||
int ret = OK_;
|
||||
UNF_UNUSED_SER;
|
||||
BASE_SER(ObTempTableAccessOpInput);
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, closed_count_);
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, unfinished_count_ptr_);
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(serialization::encode_vi64(buf, buf_len, pos, interm_result_ids_.count()))) {
|
||||
LOG_WARN("fail to encode key ranges count", K(ret), K(interm_result_ids_));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < interm_result_ids_.count(); ++i) {
|
||||
if (OB_FAIL(serialization::encode_vi64(buf, buf_len, pos, interm_result_ids_.at(i)))) {
|
||||
LOG_WARN("fail to serialize key range", K(ret), K(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObTempTableAccessOpInput)
|
||||
{
|
||||
int ret = OK_;
|
||||
UNF_UNUSED_DES;
|
||||
BASE_DESER(ObTempTableAccessOpInput);
|
||||
LST_DO_CODE(OB_UNIS_DECODE, closed_count_);
|
||||
LST_DO_CODE(OB_UNIS_DECODE, unfinished_count_ptr_);
|
||||
if (OB_SUCC(ret)) {
|
||||
int64_t count = 0;
|
||||
interm_result_ids_.reset();
|
||||
if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &count))) {
|
||||
LOG_WARN("fail to decode key ranges count", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < count; i++) {
|
||||
int64_t interm_result_id = 0;
|
||||
if (OB_ISNULL(deserialize_allocator_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("deserialize allocator is NULL", K(ret));
|
||||
} else if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &interm_result_id))) {
|
||||
LOG_WARN("failed to decode vi64.", K(ret));
|
||||
} else if (OB_FAIL(interm_result_ids_.push_back(interm_result_id))) {
|
||||
LOG_WARN("failed to push back into interm result ids.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE_SIZE(ObTempTableAccessOpInput)
|
||||
{
|
||||
int64_t len = 0;
|
||||
BASE_ADD_LEN(ObTempTableAccessOpInput);
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, closed_count_);
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, unfinished_count_ptr_);
|
||||
len += serialization::encoded_length_vi64(interm_result_ids_.count());
|
||||
for (int64_t i = 0; i < interm_result_ids_.count(); ++i) {
|
||||
len += serialization::encoded_length_vi64(interm_result_ids_.at(i));
|
||||
}
|
||||
return len;
|
||||
}
|
||||
|
||||
DEF_TO_STRING(ObTempTableAccessOpSpec)
|
||||
{
|
||||
int64_t pos = 0;
|
||||
J_OBJ_START();
|
||||
J_NAME("op_spec");
|
||||
J_COLON();
|
||||
pos += ObOpSpec::to_string(buf + pos, buf_len - pos);
|
||||
J_COMMA();
|
||||
J_KV(K(temp_table_id_), K(is_distributed_), K(output_indexs_), K(access_exprs_));
|
||||
J_OBJ_END();
|
||||
return pos;
|
||||
}
|
||||
|
||||
OB_SERIALIZE_MEMBER_INHERIT(
|
||||
ObTempTableAccessOpSpec, ObOpSpec, output_indexs_, temp_table_id_, is_distributed_, need_release_, access_exprs_);
|
||||
|
||||
int ObTempTableAccessOp::rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccessOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccessOp::inner_close()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(destory_interm_results())) {
|
||||
LOG_WARN("failed to destroy interm results.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccessOp::destory_interm_results()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool is_end = false;
|
||||
if (MY_SPEC.is_distributed_) {
|
||||
if (!MY_SPEC.need_release_) {
|
||||
// do nothing.
|
||||
} else if (OB_FAIL(MY_INPUT.check_closed_finish(is_end))) {
|
||||
LOG_WARN("failed to check closed finish.", K(ret));
|
||||
} else if (is_end) {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < MY_INPUT.interm_result_ids_.count(); ++i) {
|
||||
dtl::ObDTLIntermResultKey dtl_int_key;
|
||||
dtl_int_key.channel_id_ = MY_INPUT.interm_result_ids_.at(i);
|
||||
if (OB_FAIL(dtl::ObDTLIntermResultManager::getInstance().erase_interm_result_info(dtl_int_key))) {
|
||||
LOG_WARN("failed to erase interm result info in manager.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
}
|
||||
} else if (MY_SPEC.need_release_) {
|
||||
dtl::ObDTLIntermResultKey dtl_int_key;
|
||||
dtl_int_key.channel_id_ = MY_SPEC.temp_table_id_;
|
||||
if (OB_FAIL(dtl::ObDTLIntermResultManager::getInstance().erase_interm_result_info(dtl_int_key))) {
|
||||
LOG_WARN("failed to erase interm result info in manager.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObTempTableAccessOp::destroy()
|
||||
{
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
int ObTempTableAccessOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
clear_evaluated_flag();
|
||||
bool is_end = false;
|
||||
const ObChunkDatumStore::StoredRow* tmp_sr = NULL;
|
||||
if (OB_FAIL(THIS_WORKER.check_status())) {
|
||||
LOG_WARN("check physical plan status failed", K(ret));
|
||||
} else if (!is_started_ && OB_FAIL(locate_next_interm_result(is_end))) {
|
||||
LOG_WARN("failed to locate next interm result.", K(ret));
|
||||
} else if (!is_started_ && is_end) {
|
||||
ret = OB_ITER_END;
|
||||
} else if (OB_FAIL(datum_store_it_.get_next_row(tmp_sr))) {
|
||||
if (OB_ITER_END == ret) {
|
||||
if (OB_FAIL(locate_next_interm_result(is_end))) {
|
||||
LOG_WARN("failed to locate next iterm result.", K(ret));
|
||||
} else if (is_end && ctx_.is_restart_plan()) {
|
||||
ret = OB_ITER_END;
|
||||
datum_store_it_.reset();
|
||||
} else if (OB_FAIL(datum_store_it_.get_next_row(tmp_sr))) {
|
||||
LOG_WARN("failed to get next row.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
} else {
|
||||
LOG_WARN("failed to get next row.", K(ret));
|
||||
}
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_ISNULL(tmp_sr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("temp src row is null.", K(ret));
|
||||
} else if (OB_UNLIKELY(MY_SPEC.output_indexs_.count() != MY_SPEC.access_exprs_.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(ret), K(MY_SPEC.output_indexs_.count()), K(MY_SPEC.access_exprs_.count()));
|
||||
} else {
|
||||
is_started_ = true;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.output_indexs_.count(); ++i) {
|
||||
int64_t col_idx = MY_SPEC.output_indexs_.at(i);
|
||||
if (OB_UNLIKELY(col_idx >= tmp_sr->cnt_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected error", K(ret), K(col_idx), K(tmp_sr->cnt_));
|
||||
} else if (OB_ISNULL(MY_SPEC.access_exprs_.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("spec output is null.", K(ret));
|
||||
} else {
|
||||
MY_SPEC.access_exprs_.at(i)->locate_expr_datum(eval_ctx_) = tmp_sr->cells()[col_idx];
|
||||
MY_SPEC.access_exprs_.at(i)->get_eval_info(eval_ctx_).evaluated_ = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
if (OB_ITER_END != ret && OB_FAIL(destory_interm_results())) {
|
||||
LOG_WARN("failed to destory interm results.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccessOp::locate_next_interm_result(bool& is_end)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t index = 0;
|
||||
dtl::ObDTLIntermResultKey dtl_int_key;
|
||||
if ((!MY_SPEC.is_distributed_)) {
|
||||
if (!is_started_) {
|
||||
dtl_int_key.channel_id_ = MY_SPEC.temp_table_id_;
|
||||
if (OB_FAIL(locate_interm_result(dtl_int_key))) {
|
||||
LOG_WARN("failed to locate interm result.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
} else {
|
||||
is_end = true;
|
||||
}
|
||||
} else if (OB_FAIL(MY_INPUT.check_finish(is_end, index))) {
|
||||
LOG_WARN("failed to check finish.", K(ret));
|
||||
} else if (is_end) {
|
||||
/*do nothing.*/
|
||||
} else {
|
||||
dtl_int_key.channel_id_ = MY_INPUT.interm_result_ids_.at(index);
|
||||
if (OB_FAIL(locate_interm_result(dtl_int_key))) {
|
||||
LOG_WARN("failed to locate interm result.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccessOp::locate_interm_result(dtl::ObDTLIntermResultKey& dtl_int_key)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
dtl::ObDTLIntermResultInfo result_info;
|
||||
if (OB_FAIL(dtl::ObDTLIntermResultManager::getInstance().get_interm_result_info(dtl_int_key, result_info))) {
|
||||
LOG_WARN("failed to create row store.", K(ret));
|
||||
} else {
|
||||
datum_store_ = result_info.datum_store_;
|
||||
if (OB_ISNULL(datum_store_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("datum store is null.", K(ret));
|
||||
} else if (OB_FAIL(datum_store_->begin(datum_store_it_))) {
|
||||
LOG_WARN("failed to begin chunk row store.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableAccessOp::prepare_scan_param()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
dtl::ObDTLIntermResultKey dtl_int_key;
|
||||
dtl::ObDTLIntermResultInfo result_info;
|
||||
if (!MY_SPEC.is_distributed_) {
|
||||
dtl_int_key.channel_id_ = MY_SPEC.temp_table_id_;
|
||||
} else {
|
||||
uint64_t index = 0;
|
||||
uint64_t* count_ptr = reinterpret_cast<uint64_t*>(MY_INPUT.unfinished_count_ptr_);
|
||||
if (MY_INPUT.interm_result_ids_.count() == 0) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("interm result ids count is zero.", K(ret));
|
||||
} else if (NULL != count_ptr) {
|
||||
index = ATOMIC_AAF(count_ptr, -1);
|
||||
dtl_int_key.channel_id_ = MY_INPUT.interm_result_ids_.at(index);
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(dtl::ObDTLIntermResultManager::getInstance().get_interm_result_info(dtl_int_key, result_info))) {
|
||||
LOG_WARN("failed to create row store.", K(ret));
|
||||
} else if (FALSE_IT(datum_store_ = result_info.datum_store_)) {
|
||||
} else if (OB_ISNULL(datum_store_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("datum store is null.", K(ret));
|
||||
} else if (OB_FAIL(datum_store_->begin(datum_store_it_))) {
|
||||
LOG_WARN("failed to begin chunk row store.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
151
src/sql/engine/basic/ob_temp_table_access_op.h
Normal file
151
src/sql/engine/basic/ob_temp_table_access_op.h
Normal file
@ -0,0 +1,151 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_ACCESS_OP_H_
|
||||
#define OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_ACCESS_OP_H_
|
||||
|
||||
#include "sql/engine/ob_operator.h"
|
||||
#include "sql/engine/ob_no_children_phy_operator.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/engine/ob_single_child_phy_operator.h"
|
||||
#include "sql/engine/basic/ob_ra_row_store.h"
|
||||
#include "sql/engine/basic/ob_chunk_row_store.h"
|
||||
#include "sql/engine/ob_sql_mem_mgr_processor.h"
|
||||
#include "sql/engine/ob_tenant_sql_memory_manager.h"
|
||||
#include "sql/executor/ob_interm_result_manager.h"
|
||||
#include "sql/dtl/ob_dtl_interm_result_manager.h"
|
||||
#include "sql/engine/ob_physical_plan_ctx.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObExecContext;
|
||||
class ObTempTableAccessOp;
|
||||
class ObTempTableAccessOpInput : public ObOpInput {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
friend class ObTempTableAccessOp;
|
||||
|
||||
public:
|
||||
ObTempTableAccessOpInput(ObExecContext& ctx, const ObOpSpec& spec);
|
||||
virtual ~ObTempTableAccessOpInput();
|
||||
virtual void reset() override;
|
||||
virtual ObPhyOperatorType get_phy_op_type() const;
|
||||
virtual void set_deserialize_allocator(common::ObIAllocator* allocator);
|
||||
virtual int init(ObTaskInfo& task_info);
|
||||
virtual int init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op);
|
||||
int check_finish(bool& is_end, int64_t& interm_res_ids);
|
||||
int check_closed_finish(bool& is_end);
|
||||
|
||||
protected:
|
||||
common::ObIAllocator* deserialize_allocator_;
|
||||
DISALLOW_COPY_AND_ASSIGN(ObTempTableAccessOpInput);
|
||||
|
||||
public:
|
||||
uint64_t closed_count_;
|
||||
uint64_t unfinished_count_ptr_;
|
||||
common::ObSEArray<uint64_t, 8> interm_result_ids_;
|
||||
};
|
||||
|
||||
class ObTempTableAccessOpSpec : public ObOpSpec {
|
||||
public:
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObTempTableAccessOpSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type)
|
||||
: ObOpSpec(alloc, type),
|
||||
output_indexs_(alloc),
|
||||
temp_table_id_(0),
|
||||
is_distributed_(false),
|
||||
need_release_(false),
|
||||
access_exprs_(alloc)
|
||||
{}
|
||||
virtual ~ObTempTableAccessOpSpec()
|
||||
{}
|
||||
void set_temp_table_id(uint64_t temp_table_id)
|
||||
{
|
||||
temp_table_id_ = temp_table_id;
|
||||
}
|
||||
uint64_t get_table_id() const
|
||||
{
|
||||
return temp_table_id_;
|
||||
}
|
||||
void set_distributed(bool is_distributed)
|
||||
{
|
||||
is_distributed_ = is_distributed;
|
||||
}
|
||||
bool is_distributed() const
|
||||
{
|
||||
return is_distributed_;
|
||||
}
|
||||
void set_need_release(bool need_release)
|
||||
{
|
||||
need_release_ = need_release;
|
||||
}
|
||||
bool is_need_release() const
|
||||
{
|
||||
return need_release_;
|
||||
}
|
||||
int add_output_index(int64_t index)
|
||||
{
|
||||
return output_indexs_.push_back(index);
|
||||
}
|
||||
int init_output_index(int64_t count)
|
||||
{
|
||||
return output_indexs_.init(count);
|
||||
}
|
||||
int add_access_expr(ObExpr* access_expr)
|
||||
{
|
||||
return access_exprs_.push_back(access_expr);
|
||||
}
|
||||
int init_access_exprs(int64_t count)
|
||||
{
|
||||
return access_exprs_.init(count);
|
||||
}
|
||||
|
||||
DECLARE_VIRTUAL_TO_STRING;
|
||||
|
||||
public:
|
||||
common::ObFixedArray<int64_t, common::ObIAllocator> output_indexs_;
|
||||
uint64_t temp_table_id_;
|
||||
bool is_distributed_;
|
||||
bool need_release_;
|
||||
// Operator access exprs expressions
|
||||
ExprFixedArray access_exprs_;
|
||||
};
|
||||
|
||||
class ObTempTableAccessOp : public ObOperator {
|
||||
public:
|
||||
ObTempTableAccessOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObOperator(exec_ctx, spec, input), datum_store_(NULL), datum_store_it_(), is_started_(false)
|
||||
{}
|
||||
~ObTempTableAccessOp()
|
||||
{}
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int rescan() override;
|
||||
virtual int inner_get_next_row() override;
|
||||
virtual int inner_close() override;
|
||||
virtual void destroy() override;
|
||||
int prepare_scan_param();
|
||||
int locate_next_interm_result(bool& is_end);
|
||||
int locate_interm_result(dtl::ObDTLIntermResultKey& dtl_int_key);
|
||||
int destory_interm_results();
|
||||
|
||||
private:
|
||||
ObChunkDatumStore* datum_store_;
|
||||
ObChunkDatumStore::Iterator datum_store_it_;
|
||||
bool is_started_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_ACCESS_OP_H_ */
|
||||
334
src/sql/engine/basic/ob_temp_table_insert.cpp
Normal file
334
src/sql/engine/basic/ob_temp_table_insert.cpp
Normal file
@ -0,0 +1,334 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "sql/engine/basic/ob_temp_table_insert.h"
|
||||
#include "sql/dtl/ob_dtl_interm_result_manager.h"
|
||||
#include "sql/engine/basic/ob_chunk_row_store.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
using dtl::ObDTLIntermResultInfo;
|
||||
|
||||
ObTempTableInsertInput::ObTempTableInsertInput()
|
||||
: deserialize_allocator_(NULL), unfinished_count_ptr_(0), interm_result_ids_()
|
||||
{}
|
||||
|
||||
ObTempTableInsertInput::~ObTempTableInsertInput()
|
||||
{
|
||||
deserialize_allocator_ = NULL;
|
||||
unfinished_count_ptr_ = 0;
|
||||
interm_result_ids_.reset();
|
||||
}
|
||||
|
||||
void ObTempTableInsertInput::reset()
|
||||
{
|
||||
deserialize_allocator_ = NULL;
|
||||
unfinished_count_ptr_ = 0;
|
||||
interm_result_ids_.reset();
|
||||
}
|
||||
|
||||
ObPhyOperatorType ObTempTableInsertInput::get_phy_op_type() const
|
||||
{
|
||||
return PHY_TEMP_TABLE_INSERT;
|
||||
}
|
||||
|
||||
void ObTempTableInsertInput::set_deserialize_allocator(common::ObIAllocator* allocator)
|
||||
{
|
||||
deserialize_allocator_ = allocator;
|
||||
}
|
||||
|
||||
int ObTempTableInsertInput::init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(ctx);
|
||||
UNUSED(task_info);
|
||||
UNUSED(op);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableInsertInput::assign_ids(common::ObIArray<uint64_t>& interm_res_ids)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(interm_result_ids_.assign(interm_res_ids))) {
|
||||
LOG_WARN("failed to assign to interm result ids.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE(ObTempTableInsertInput)
|
||||
{
|
||||
int ret = OK_;
|
||||
UNF_UNUSED_SER;
|
||||
BASE_SER(ObTempTableInsertInput);
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, unfinished_count_ptr_);
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(serialization::encode_vi64(buf, buf_len, pos, interm_result_ids_.count()))) {
|
||||
LOG_WARN("fail to encode key ranges count", K(ret), K(interm_result_ids_));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < interm_result_ids_.count(); ++i) {
|
||||
if (OB_FAIL(serialization::encode_vi64(buf, buf_len, pos, interm_result_ids_.at(i)))) {
|
||||
LOG_WARN("fail to serialize key range", K(ret), K(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObTempTableInsertInput)
|
||||
{
|
||||
int ret = OK_;
|
||||
UNF_UNUSED_DES;
|
||||
BASE_DESER(ObTempTableInsertInput);
|
||||
LST_DO_CODE(OB_UNIS_DECODE, unfinished_count_ptr_);
|
||||
if (OB_SUCC(ret)) {
|
||||
int64_t count = 0;
|
||||
interm_result_ids_.reset();
|
||||
if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &count))) {
|
||||
LOG_WARN("fail to decode key ranges count", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < count; i++) {
|
||||
int64_t interm_result_id = 0;
|
||||
if (OB_ISNULL(deserialize_allocator_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("deserialize allocator is NULL", K(ret));
|
||||
} else if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &interm_result_id))) {
|
||||
LOG_WARN("failed to decode vi64.", K(ret));
|
||||
} else if (OB_FAIL(interm_result_ids_.push_back(interm_result_id))) {
|
||||
LOG_WARN("failed to push back into interm result ids.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE_SIZE(ObTempTableInsertInput)
|
||||
{
|
||||
int64_t len = 0;
|
||||
BASE_ADD_LEN(ObTempTableInsertInput);
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, unfinished_count_ptr_);
|
||||
len += serialization::encoded_length_vi64(interm_result_ids_.count());
|
||||
for (int64_t i = 0; i < interm_result_ids_.count(); ++i) {
|
||||
len += serialization::encoded_length_vi64(interm_result_ids_.at(i));
|
||||
}
|
||||
return len;
|
||||
}
|
||||
|
||||
void ObTempTableInsert::reset()
|
||||
{
|
||||
ObSingleChildPhyOperator::reset();
|
||||
}
|
||||
|
||||
void ObTempTableInsert::reuse()
|
||||
{
|
||||
ObSingleChildPhyOperator::reuse();
|
||||
}
|
||||
|
||||
int ObTempTableInsert::rescan(ObExecContext& exec_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObSingleChildPhyOperator::rescan(exec_ctx))) {
|
||||
LOG_WARN("failed to rescan.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableInsert::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObTempTableInsertInput* tti_input = NULL;
|
||||
if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("failed to init op ctx.", K(ret));
|
||||
} else if (OB_FAIL(handle_op_ctx(ctx))) {
|
||||
LOG_WARN("failed to handle operator context.", K(ret));
|
||||
} else if (OB_FAIL(prepare_interm_result(ctx))) {
|
||||
LOG_WARN("failed to prepare interm result.", K(ret));
|
||||
} else if (OB_ISNULL(ctx.get_my_session())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get my session.", K(ret));
|
||||
} else if (is_distributed_ && OB_ISNULL(tti_input = GET_PHY_OP_INPUT(ObTempTableInsertInput, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get op input", K(ret));
|
||||
} else {
|
||||
const common::ObNewRow* row = NULL;
|
||||
ObPhysicalPlanCtx* phy_plan_ctx = NULL;
|
||||
ObTempTableInsertCtx* tti_ctx = NULL;
|
||||
dtl::ObDTLIntermResultKey dtl_int_key;
|
||||
ObDTLIntermResultInfo chunk_row_store;
|
||||
uint64_t tenant_id = ctx.get_my_session()->get_effective_tenant_id();
|
||||
int64_t chuck_cnt = is_distributed_ ? tti_input->interm_result_ids_.count() : 1;
|
||||
int64_t chuck_dump_size = 0;
|
||||
if (chuck_cnt > 0) {
|
||||
chuck_dump_size = lib::get_tenant_memory_limit(tenant_id) / (2 * chuck_cnt);
|
||||
}
|
||||
ObMemAttr mem_attr(tenant_id, ObModIds::OB_SQL_SORT_ROW, ObCtxIds::WORK_AREA);
|
||||
if (OB_FAIL(dtl::ObDTLIntermResultManager::getInstance().create_interm_result_info(mem_attr, chunk_row_store))) {
|
||||
LOG_WARN("failed to create row store.", K(ret));
|
||||
} else if (chunk_row_store.row_store_->init(chuck_dump_size,
|
||||
ctx.get_my_session()->get_effective_tenant_id(),
|
||||
ObCtxIds::WORK_AREA,
|
||||
"SqlTempTableRowSt",
|
||||
true,
|
||||
ObChunkRowStore::WITHOUT_PROJECTOR,
|
||||
sizeof(uint64_t))) {
|
||||
LOG_WARN("failed to init the chunk row store.", K(ret));
|
||||
} else if (OB_ISNULL(tti_ctx = GET_PHY_OPERATOR_CTX(ObTempTableInsertCtx, ctx, get_id()))) {
|
||||
LOG_WARN("fail to create phy op ctx", K(ret), K(get_id()), K(get_type()));
|
||||
} else if (OB_ISNULL(phy_plan_ctx = GET_PHY_PLAN_CTX(ctx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("phy_plan_ctx is NULL", K(ret));
|
||||
} else {
|
||||
dtl_int_key.channel_id_ = tti_ctx->interm_result_id_;
|
||||
dtl_int_key.start_time_ = oceanbase::common::ObTimeUtility::current_time();
|
||||
dtl_int_key.time_us_ = phy_plan_ctx->get_timeout_timestamp();
|
||||
while (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(child_op_->get_next_row(ctx, row))) {
|
||||
if (OB_UNLIKELY(OB_ITER_END != ret)) {
|
||||
LOG_WARN("fail to get next row from child.", K(ret), K(child_op_->get_type()));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
} else if (OB_ISNULL(row)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("row is NULL", K(ret));
|
||||
} else if (OB_FAIL(chunk_row_store.row_store_->add_row(*row))) {
|
||||
LOG_WARN("failed to add row to chunk row store.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
if (OB_ITER_END != ret) {
|
||||
chunk_row_store.row_store_->reset();
|
||||
} else {
|
||||
ret = OB_SUCCESS;
|
||||
LOG_DEBUG("all rows are fetched");
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(
|
||||
dtl::ObDTLIntermResultManager::getInstance().insert_interm_result_info(dtl_int_key, chunk_row_store))) {
|
||||
LOG_WARN("failed to insert row store.", K(ret), K(dtl_int_key.channel_id_));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableInsert::inner_close(ObExecContext& exec_ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(exec_ctx);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableInsert::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhyOperatorCtx* op_ctx = NULL;
|
||||
uint64_t interm_r_id = 0;
|
||||
if (OB_UNLIKELY(calc_exprs_.get_size() > 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("calc exprs should be empty", K(ret), K(calc_exprs_.get_size()));
|
||||
} else if (OB_UNLIKELY(filter_exprs_.get_size() > 0)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("filter exprs should be empty", K(ret), K(filter_exprs_.get_size()));
|
||||
} else if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObTempTableInsertCtx, ctx, get_id(), get_type(), op_ctx))) {
|
||||
LOG_WARN("fail to create phy op ctx", K(ret), K(get_id()), K(get_type()));
|
||||
} else if (OB_ISNULL(op_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("op ctx is NULL", K(ret));
|
||||
} else {
|
||||
op_ctx->set_op_id(interm_r_id);
|
||||
if (OB_FAIL(init_cur_row(*op_ctx, false))) {
|
||||
LOG_WARN("fail to int cur row", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableInsert::inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(row);
|
||||
UNUSED(ctx);
|
||||
if (OB_ISNULL(child_op_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("child op is null");
|
||||
} else {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableInsert::get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ret = ObPhyOperator::get_next_row(ctx, row);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableInsert::create_operator_input(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObIPhyOperatorInput* input = NULL;
|
||||
if (OB_FAIL(CREATE_PHY_OP_INPUT(ObTempTableInsertInput, ctx, get_id(), get_type(), input))) {
|
||||
LOG_WARN("fail to create phy op input", K(ret), K(get_id()), K(get_type()));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
UNUSED(input);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableInsert::prepare_interm_result(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObTempTableInsertInput* insert_input = NULL;
|
||||
ObTempTableInsertCtx* tti_ctx = NULL;
|
||||
if (OB_ISNULL(tti_ctx = GET_PHY_OPERATOR_CTX(ObTempTableInsertCtx, ctx, get_id()))) {
|
||||
LOG_WARN("fail to create phy op ctx", K(ret), K(get_id()), K(get_type()));
|
||||
} else if (!is_distributed_) {
|
||||
tti_ctx->set_iterm_result_id(temp_table_id_);
|
||||
} else if (OB_ISNULL(insert_input = GET_PHY_OP_INPUT(ObTempTableInsertInput, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get op input", K(ret));
|
||||
} else {
|
||||
uint64_t index = 0;
|
||||
uint64_t* count_ptr = reinterpret_cast<uint64_t*>(insert_input->unfinished_count_ptr_);
|
||||
if (NULL != count_ptr) {
|
||||
index = ATOMIC_AAF(count_ptr, -1);
|
||||
tti_ctx->set_iterm_result_id(insert_input->interm_result_ids_.at(index));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObTempTableInsert, ObSingleChildPhyOperator), temp_table_id_, is_distributed_);
|
||||
|
||||
int64_t ObTempTableInsert::to_string(char* buf, const int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
J_OBJ_START();
|
||||
J_OBJ_END();
|
||||
return pos;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
121
src/sql/engine/basic/ob_temp_table_insert.h
Normal file
121
src/sql/engine/basic/ob_temp_table_insert.h
Normal file
@ -0,0 +1,121 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_INSERT_H_
|
||||
#define OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_INSERT_H_
|
||||
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/engine/ob_single_child_phy_operator.h"
|
||||
#include "sql/engine/basic/ob_ra_row_store.h"
|
||||
#include "sql/engine/ob_physical_plan_ctx.h"
|
||||
#include "sql/engine/basic/ob_chunk_row_store.h"
|
||||
#include "sql/engine/ob_sql_mem_mgr_processor.h"
|
||||
#include "sql/engine/ob_tenant_sql_memory_manager.h"
|
||||
#include "sql/executor/ob_interm_result_manager.h"
|
||||
#include "sql/dtl/ob_dtl_interm_result_manager.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObExecContext;
|
||||
class ObTempTableInsertInput : public ObIPhyOperatorInput {
|
||||
friend class ObTempTableInsert;
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObTempTableInsertInput();
|
||||
virtual ~ObTempTableInsertInput();
|
||||
virtual void reset() override;
|
||||
virtual ObPhyOperatorType get_phy_op_type() const;
|
||||
virtual void set_deserialize_allocator(common::ObIAllocator* allocator);
|
||||
virtual int init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op);
|
||||
int assign_ids(common::ObIArray<uint64_t>& interm_res_ids);
|
||||
|
||||
protected:
|
||||
common::ObIAllocator* deserialize_allocator_;
|
||||
DISALLOW_COPY_AND_ASSIGN(ObTempTableInsertInput);
|
||||
|
||||
public:
|
||||
uint64_t unfinished_count_ptr_;
|
||||
common::ObSEArray<uint64_t, 8> interm_result_ids_;
|
||||
};
|
||||
|
||||
class ObTempTableInsert : public ObSingleChildPhyOperator {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
protected:
|
||||
class ObTempTableInsertCtx : public ObPhyOperatorCtx {
|
||||
friend class ObTempTableInsert;
|
||||
|
||||
public:
|
||||
explicit ObTempTableInsertCtx(ObExecContext& ctx) : ObPhyOperatorCtx(ctx), interm_result_id_(0)
|
||||
{}
|
||||
virtual ~ObTempTableInsertCtx()
|
||||
{}
|
||||
virtual void destroy()
|
||||
{
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
void set_iterm_result_id(uint64_t id)
|
||||
{
|
||||
interm_result_id_ = id;
|
||||
}
|
||||
|
||||
public:
|
||||
uint64_t interm_result_id_;
|
||||
};
|
||||
|
||||
public:
|
||||
ObTempTableInsert(common::ObIAllocator& alloc)
|
||||
: ObSingleChildPhyOperator(alloc), temp_table_id_(0), is_distributed_(false)
|
||||
{}
|
||||
virtual ~ObTempTableInsert()
|
||||
{}
|
||||
void reset();
|
||||
void reuse();
|
||||
int rescan(ObExecContext& exec_ctx) const;
|
||||
|
||||
DECLARE_VIRTUAL_TO_STRING;
|
||||
virtual int inner_open(ObExecContext& exec_ctx) const;
|
||||
virtual int inner_close(ObExecContext& exec_ctx) const;
|
||||
virtual int init_op_ctx(ObExecContext& ctx) const;
|
||||
virtual int inner_get_next_row(ObExecContext& exec_ctx, const common::ObNewRow*& row) const;
|
||||
int get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
int create_operator_input(ObExecContext& ctx) const;
|
||||
int prepare_interm_result(ObExecContext& ctx) const;
|
||||
void set_temp_table_id(uint64_t temp_table_id)
|
||||
{
|
||||
temp_table_id_ = temp_table_id;
|
||||
}
|
||||
uint64_t get_table_id() const
|
||||
{
|
||||
return temp_table_id_;
|
||||
}
|
||||
void set_distributed(bool is_distributed)
|
||||
{
|
||||
is_distributed_ = is_distributed;
|
||||
}
|
||||
bool is_distributed() const
|
||||
{
|
||||
return is_distributed_;
|
||||
}
|
||||
|
||||
DISALLOW_COPY_AND_ASSIGN(ObTempTableInsert);
|
||||
|
||||
private:
|
||||
uint64_t temp_table_id_;
|
||||
bool is_distributed_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_INSERT_H_ */
|
||||
273
src/sql/engine/basic/ob_temp_table_insert_op.cpp
Normal file
273
src/sql/engine/basic/ob_temp_table_insert_op.cpp
Normal file
@ -0,0 +1,273 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "ob_temp_table_insert_op.h"
|
||||
#include "sql/engine/ob_operator_reg.h"
|
||||
#include "sql/dtl/ob_dtl_interm_result_manager.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
using namespace storage;
|
||||
using namespace share;
|
||||
using namespace share::schema;
|
||||
namespace sql {
|
||||
|
||||
#define USE_MULTI_GET_ARRAY_BINDING 1
|
||||
|
||||
ObTempTableInsertOpInput::ObTempTableInsertOpInput(ObExecContext& ctx, const ObOpSpec& spec)
|
||||
: ObOpInput(ctx, spec), deserialize_allocator_(NULL), unfinished_count_ptr_(0), interm_result_ids_()
|
||||
{}
|
||||
|
||||
ObTempTableInsertOpInput::~ObTempTableInsertOpInput()
|
||||
{
|
||||
deserialize_allocator_ = NULL;
|
||||
unfinished_count_ptr_ = 0;
|
||||
interm_result_ids_.reset();
|
||||
}
|
||||
|
||||
void ObTempTableInsertOpInput::reset()
|
||||
{
|
||||
deserialize_allocator_ = NULL;
|
||||
unfinished_count_ptr_ = 0;
|
||||
interm_result_ids_.reset();
|
||||
}
|
||||
|
||||
ObPhyOperatorType ObTempTableInsertOpInput::get_phy_op_type() const
|
||||
{
|
||||
return PHY_TEMP_TABLE_INSERT;
|
||||
}
|
||||
|
||||
void ObTempTableInsertOpInput::set_deserialize_allocator(common::ObIAllocator* allocator)
|
||||
{
|
||||
deserialize_allocator_ = allocator;
|
||||
}
|
||||
|
||||
int ObTempTableInsertOpInput::init(ObTaskInfo& task_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(task_info);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableInsertOpInput::init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(ctx);
|
||||
UNUSED(task_info);
|
||||
UNUSED(op);
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE(ObTempTableInsertOpInput)
|
||||
{
|
||||
int ret = OK_;
|
||||
UNF_UNUSED_SER;
|
||||
BASE_SER(ObTempTableInsertOpInput);
|
||||
LST_DO_CODE(OB_UNIS_ENCODE, unfinished_count_ptr_);
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(serialization::encode_vi64(buf, buf_len, pos, interm_result_ids_.count()))) {
|
||||
LOG_WARN("fail to encode key ranges count", K(ret), K(interm_result_ids_));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < interm_result_ids_.count(); ++i) {
|
||||
if (OB_FAIL(serialization::encode_vi64(buf, buf_len, pos, interm_result_ids_.at(i)))) {
|
||||
LOG_WARN("fail to serialize key range", K(ret), K(i));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObTempTableInsertOpInput)
|
||||
{
|
||||
int ret = OK_;
|
||||
UNF_UNUSED_DES;
|
||||
BASE_DESER(ObTempTableInsertOpInput);
|
||||
LST_DO_CODE(OB_UNIS_DECODE, unfinished_count_ptr_);
|
||||
if (OB_SUCC(ret)) {
|
||||
int64_t count = 0;
|
||||
interm_result_ids_.reset();
|
||||
if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &count))) {
|
||||
LOG_WARN("fail to decode key ranges count", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < count; i++) {
|
||||
int64_t interm_result_id = 0;
|
||||
if (OB_ISNULL(deserialize_allocator_)) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("deserialize allocator is NULL", K(ret));
|
||||
} else if (OB_FAIL(serialization::decode_vi64(buf, data_len, pos, &interm_result_id))) {
|
||||
LOG_WARN("failed to decode vi64.", K(ret));
|
||||
} else if (OB_FAIL(interm_result_ids_.push_back(interm_result_id))) {
|
||||
LOG_WARN("failed to push back into interm result ids.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE_SIZE(ObTempTableInsertOpInput)
|
||||
{
|
||||
int64_t len = 0;
|
||||
BASE_ADD_LEN(ObTempTableInsertOpInput);
|
||||
LST_DO_CODE(OB_UNIS_ADD_LEN, unfinished_count_ptr_);
|
||||
len += serialization::encoded_length_vi64(interm_result_ids_.count());
|
||||
for (int64_t i = 0; i < interm_result_ids_.count(); ++i) {
|
||||
len += serialization::encoded_length_vi64(interm_result_ids_.at(i));
|
||||
}
|
||||
return len;
|
||||
}
|
||||
|
||||
DEF_TO_STRING(ObTempTableInsertOpSpec)
|
||||
{
|
||||
int64_t pos = 0;
|
||||
J_OBJ_START();
|
||||
J_NAME("op_spec");
|
||||
J_COLON();
|
||||
pos += ObOpSpec::to_string(buf + pos, buf_len - pos);
|
||||
J_COMMA();
|
||||
J_KV(K(temp_table_id_), K(is_distributed_));
|
||||
J_OBJ_END();
|
||||
return pos;
|
||||
}
|
||||
|
||||
OB_SERIALIZE_MEMBER_INHERIT(ObTempTableInsertOpSpec, ObOpSpec, temp_table_id_, is_distributed_);
|
||||
|
||||
int ObTempTableInsertOp::rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObOperator::rescan())) {
|
||||
LOG_WARN("failed to rescan.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableInsertOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(ctx_.get_my_session())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("ctx session is null");
|
||||
} else if (OB_FAIL(prepare_scan_param())) {
|
||||
LOG_WARN("failed to prepare scan param.", K(ret));
|
||||
} else {
|
||||
dtl::ObDTLIntermResultKey dtl_int_key;
|
||||
dtl::ObDTLIntermResultInfo chunk_row_store;
|
||||
ObPhysicalPlanCtx* phy_plan_ctx = NULL;
|
||||
uint64_t tenant_id = ctx_.get_my_session()->get_effective_tenant_id();
|
||||
int64_t chuck_cnt = MY_SPEC.is_distributed_ ? MY_INPUT.interm_result_ids_.count() : 1;
|
||||
int64_t chuck_dump_size = 0;
|
||||
if (chuck_cnt > 0) {
|
||||
chuck_dump_size = lib::get_tenant_memory_limit(tenant_id) / (2 * chuck_cnt);
|
||||
}
|
||||
ObMemAttr mem_attr(tenant_id, ObModIds::OB_SQL_SORT_ROW, ObCtxIds::WORK_AREA);
|
||||
chunk_row_store.is_datum_ = true;
|
||||
if (OB_FAIL(dtl::ObDTLIntermResultManager::getInstance().create_interm_result_info(mem_attr, chunk_row_store))) {
|
||||
LOG_WARN("failed to create row store.", K(ret));
|
||||
} else if (OB_ISNULL(chunk_row_store.datum_store_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("datum store is null");
|
||||
} else if (chunk_row_store.datum_store_->init(chuck_dump_size,
|
||||
ctx_.get_my_session()->get_effective_tenant_id(),
|
||||
ObCtxIds::WORK_AREA,
|
||||
"SqlTempTableRowSt",
|
||||
true,
|
||||
sizeof(uint64_t))) {
|
||||
LOG_WARN("failed to init the chunk row store.", K(ret));
|
||||
} else if (OB_ISNULL(child_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("child operator is null");
|
||||
} else if (OB_ISNULL(phy_plan_ctx = GET_PHY_PLAN_CTX(ctx_))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("phy plan ctx is null.", K(ret));
|
||||
} else {
|
||||
dtl_int_key.channel_id_ = interm_result_id_;
|
||||
dtl_int_key.start_time_ = oceanbase::common::ObTimeUtility::current_time();
|
||||
dtl_int_key.time_us_ = phy_plan_ctx->get_timeout_timestamp();
|
||||
while (OB_SUCC(ret)) {
|
||||
clear_evaluated_flag();
|
||||
if (OB_FAIL(child_->get_next_row())) {
|
||||
if (OB_UNLIKELY(OB_ITER_END != ret)) {
|
||||
LOG_WARN("fail to get next row from child.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
} else if (OB_FAIL(chunk_row_store.datum_store_->add_row(child_->get_spec().output_, &eval_ctx_))) {
|
||||
LOG_WARN("failed to add row to chunk row store.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
if (OB_ITER_END != ret) {
|
||||
chunk_row_store.datum_store_->reset();
|
||||
} else {
|
||||
ret = OB_SUCCESS;
|
||||
LOG_DEBUG("all rows are fetched");
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(dtl::ObDTLIntermResultManager::getInstance().insert_interm_result_info(
|
||||
dtl_int_key, chunk_row_store))) {
|
||||
LOG_WARN("failed to insert row store.", K(ret), K(dtl_int_key));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableInsertOp::inner_close()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
interm_result_id_ = 0;
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObTempTableInsertOp::destroy()
|
||||
{
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
int ObTempTableInsertOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(THIS_WORKER.check_status())) {
|
||||
LOG_WARN("check physical plan status failed.", K(ret));
|
||||
} else {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableInsertOp::prepare_scan_param()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (!MY_SPEC.is_distributed_) {
|
||||
interm_result_id_ = MY_SPEC.temp_table_id_;
|
||||
} else {
|
||||
uint64_t index = 0;
|
||||
uint64_t* count_ptr = reinterpret_cast<uint64_t*>(MY_INPUT.unfinished_count_ptr_);
|
||||
if (NULL != count_ptr) {
|
||||
index = ATOMIC_AAF(count_ptr, -1);
|
||||
interm_result_id_ = MY_INPUT.interm_result_ids_.at(index);
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
110
src/sql/engine/basic/ob_temp_table_insert_op.h
Normal file
110
src/sql/engine/basic/ob_temp_table_insert_op.h
Normal file
@ -0,0 +1,110 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_INSERT_OP_H_
|
||||
#define OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_INSERT_OP_H_
|
||||
|
||||
#include "sql/engine/ob_operator.h"
|
||||
#include "sql/engine/ob_no_children_phy_operator.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/engine/ob_single_child_phy_operator.h"
|
||||
#include "sql/engine/basic/ob_ra_row_store.h"
|
||||
#include "sql/engine/basic/ob_chunk_row_store.h"
|
||||
#include "sql/engine/ob_sql_mem_mgr_processor.h"
|
||||
#include "sql/engine/ob_tenant_sql_memory_manager.h"
|
||||
#include "sql/executor/ob_interm_result_manager.h"
|
||||
#include "sql/dtl/ob_dtl_interm_result_manager.h"
|
||||
#include "sql/engine/ob_physical_plan_ctx.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObExecContext;
|
||||
class ObTempTableInsertOp;
|
||||
class ObTempTableInsertOpInput : public ObOpInput {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
friend class ObTempTableInsertOp;
|
||||
|
||||
public:
|
||||
ObTempTableInsertOpInput(ObExecContext& ctx, const ObOpSpec& spec);
|
||||
virtual ~ObTempTableInsertOpInput();
|
||||
virtual void reset() override;
|
||||
virtual ObPhyOperatorType get_phy_op_type() const;
|
||||
virtual void set_deserialize_allocator(common::ObIAllocator* allocator);
|
||||
virtual int init(ObTaskInfo& task_info);
|
||||
virtual int init(ObExecContext& ctx, ObTaskInfo& task_info, const ObPhyOperator& op);
|
||||
|
||||
protected:
|
||||
common::ObIAllocator* deserialize_allocator_;
|
||||
DISALLOW_COPY_AND_ASSIGN(ObTempTableInsertOpInput);
|
||||
|
||||
public:
|
||||
uint64_t unfinished_count_ptr_;
|
||||
common::ObSEArray<uint64_t, 8> interm_result_ids_;
|
||||
};
|
||||
|
||||
class ObTempTableInsertOpSpec : public ObOpSpec {
|
||||
public:
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObTempTableInsertOpSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type)
|
||||
: ObOpSpec(alloc, type), temp_table_id_(0), is_distributed_(false)
|
||||
{}
|
||||
virtual ~ObTempTableInsertOpSpec()
|
||||
{}
|
||||
void set_temp_table_id(uint64_t temp_table_id)
|
||||
{
|
||||
temp_table_id_ = temp_table_id;
|
||||
}
|
||||
uint64_t get_table_id() const
|
||||
{
|
||||
return temp_table_id_;
|
||||
}
|
||||
void set_distributed(bool is_distributed)
|
||||
{
|
||||
is_distributed_ = is_distributed;
|
||||
}
|
||||
bool is_distributed() const
|
||||
{
|
||||
return is_distributed_;
|
||||
}
|
||||
|
||||
DECLARE_VIRTUAL_TO_STRING;
|
||||
|
||||
public:
|
||||
uint64_t temp_table_id_;
|
||||
bool is_distributed_;
|
||||
};
|
||||
|
||||
class ObTempTableInsertOp : public ObOperator {
|
||||
public:
|
||||
ObTempTableInsertOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObOperator(exec_ctx, spec, input), interm_result_id_(0)
|
||||
{}
|
||||
~ObTempTableInsertOp()
|
||||
{}
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int rescan() override;
|
||||
virtual int inner_get_next_row() override;
|
||||
virtual int inner_close() override;
|
||||
virtual void destroy() override;
|
||||
int prepare_scan_param();
|
||||
|
||||
private:
|
||||
uint64_t interm_result_id_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_INSERT_OP_H_ */
|
||||
123
src/sql/engine/basic/ob_temp_table_transformation.cpp
Normal file
123
src/sql/engine/basic/ob_temp_table_transformation.cpp
Normal file
@ -0,0 +1,123 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "sql/engine/basic/ob_temp_table_transformation.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
void ObTempTableTransformation::reset()
|
||||
{
|
||||
ObMultiChildrenPhyOperator::reset();
|
||||
}
|
||||
|
||||
void ObTempTableTransformation::reuse()
|
||||
{
|
||||
ObMultiChildrenPhyOperator::reuse();
|
||||
}
|
||||
|
||||
int ObTempTableTransformation::rescan(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObMultiChildrenPhyOperator::rescan(ctx))) {
|
||||
LOG_WARN("failed to rescan transformation.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableTransformation::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const common::ObNewRow* row = NULL;
|
||||
if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("failed to init op ctx.", K(ret));
|
||||
} else if (OB_FAIL(handle_op_ctx(ctx))) {
|
||||
LOG_WARN("failed to handle operator context.", K(ret));
|
||||
} else if (OB_ISNULL(child_array_[0])) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("child op is null");
|
||||
} else if (OB_FAIL(child_array_[0]->get_next_row(ctx, row))) {
|
||||
if (ret != OB_ITER_END) {
|
||||
LOG_WARN("failed to get next row.", K(ret));
|
||||
} else {
|
||||
LOG_DEBUG("all rows are fetched");
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableTransformation::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhyOperatorCtx* op_ctx = NULL;
|
||||
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObTempTableTransformationCtx, ctx, get_id(), get_type(), op_ctx))) {
|
||||
LOG_WARN("failed to create phy operator ctx.", K(ret));
|
||||
} else if (OB_FAIL(init_cur_row(*op_ctx, true))) {
|
||||
LOG_WARN("create current row failed", K(ret));
|
||||
} else { /* do nothing. */
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableTransformation::inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhyOperatorCtx* ttt_ctx = NULL;
|
||||
const ObNewRow* input_row = NULL;
|
||||
if (OB_ISNULL(child_array_[0])) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("child op is null");
|
||||
} else if (child_array_[0]->is_dml_operator()) {
|
||||
ret = OB_ITER_END;
|
||||
} else if (OB_ISNULL(ttt_ctx = GET_PHY_OPERATOR_CTX(ObTempTableTransformationCtx, ctx, get_id()))) {
|
||||
LOG_WARN("failed to create phy operator ctx.", K(ret));
|
||||
} else if (OB_FAIL(child_array_[1]->get_next_row(ctx, input_row))) {
|
||||
if (ret != OB_ITER_END) {
|
||||
LOG_WARN("failed to get next row from child array.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
} else {
|
||||
for (int64_t i = 0; i < input_row->get_count(); i++) {
|
||||
ttt_ctx->get_cur_row().cells_[i] = input_row->get_cell(i);
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
row = &ttt_ctx->get_cur_row();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableTransformation::get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ret = ObPhyOperator::get_next_row(ctx, row);
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableTransformation::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(ctx);
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObTempTableTransformation, ObMultiChildrenPhyOperator));
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
64
src/sql/engine/basic/ob_temp_table_transformation.h
Normal file
64
src/sql/engine/basic/ob_temp_table_transformation.h
Normal file
@ -0,0 +1,64 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_TRANSFORMATION_H_
|
||||
#define OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_TRANSFORMATION_H_
|
||||
|
||||
#include "sql/engine/ob_phy_operator.h"
|
||||
#include "sql/engine/ob_multi_children_phy_operator.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/dtl/ob_dtl_interm_result_manager.h"
|
||||
#include "common/row/ob_row.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObTempTableTransformation : public ObMultiChildrenPhyOperator {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
protected:
|
||||
class ObTempTableTransformationCtx : public ObPhyOperatorCtx {
|
||||
friend class ObTempTableTransformation;
|
||||
|
||||
public:
|
||||
explicit ObTempTableTransformationCtx(ObExecContext& ctx) : ObPhyOperatorCtx(ctx)
|
||||
{}
|
||||
virtual ~ObTempTableTransformationCtx()
|
||||
{}
|
||||
virtual void destroy()
|
||||
{
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
};
|
||||
|
||||
public:
|
||||
explicit ObTempTableTransformation(common::ObIAllocator& alloc) : ObMultiChildrenPhyOperator(alloc)
|
||||
{}
|
||||
~ObTempTableTransformation()
|
||||
{}
|
||||
|
||||
virtual void reset();
|
||||
virtual void reuse();
|
||||
|
||||
virtual int rescan(ObExecContext& ctx) const;
|
||||
virtual int inner_open(ObExecContext& ctx) const;
|
||||
virtual int init_op_ctx(ObExecContext& ctx) const;
|
||||
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
virtual int get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
virtual int inner_close(ObExecContext& ctx) const;
|
||||
|
||||
DISALLOW_COPY_AND_ASSIGN(ObTempTableTransformation);
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_TRANSFORMATION_H_ */
|
||||
102
src/sql/engine/basic/ob_temp_table_transformation_op.cpp
Normal file
102
src/sql/engine/basic/ob_temp_table_transformation_op.cpp
Normal file
@ -0,0 +1,102 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "ob_temp_table_transformation_op.h"
|
||||
#include "sql/engine/ob_operator_reg.h"
|
||||
#include "sql/dtl/ob_dtl_interm_result_manager.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/px/ob_px_util.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
using namespace storage;
|
||||
using namespace share;
|
||||
using namespace share::schema;
|
||||
namespace sql {
|
||||
#define USE_MULTI_GET_ARRAY_BINDING 1
|
||||
|
||||
DEF_TO_STRING(ObTempTableTransformationOpSpec)
|
||||
{
|
||||
int64_t pos = 0;
|
||||
J_OBJ_START();
|
||||
J_NAME("op_spec");
|
||||
J_COLON();
|
||||
pos += ObOpSpec::to_string(buf + pos, buf_len - pos);
|
||||
J_COMMA();
|
||||
J_OBJ_END();
|
||||
return pos;
|
||||
}
|
||||
|
||||
OB_SERIALIZE_MEMBER_INHERIT(ObTempTableTransformationOpSpec, ObOpSpec);
|
||||
|
||||
int ObTempTableTransformationOp::rescan()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(ObOperator::rescan())) {
|
||||
LOG_WARN("failed to rescan the operator.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableTransformationOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(children_[0])) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("child op is null");
|
||||
} else if (OB_FAIL(children_[0]->get_next_row())) {
|
||||
if (ret != OB_ITER_END) {
|
||||
LOG_WARN("failed to get next row.", K(ret));
|
||||
} else {
|
||||
LOG_DEBUG("all rows are fetched");
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableTransformationOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
clear_evaluated_flag();
|
||||
if (OB_ISNULL(children_[1])) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("child op is null");
|
||||
} else if (OB_FAIL(children_[1]->get_next_row())) {
|
||||
if (ret != OB_ITER_END) {
|
||||
LOG_WARN("failed to get next row.", K(ret));
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
} else { /*do nothing.*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTempTableTransformationOp::inner_close()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObTempTableTransformationOp::destroy()
|
||||
{
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
61
src/sql/engine/basic/ob_temp_table_transformation_op.h
Normal file
61
src/sql/engine/basic/ob_temp_table_transformation_op.h
Normal file
@ -0,0 +1,61 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_TRANSFORMATION_OP_H_
|
||||
#define OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_TRANSFORMATION_OP_H_
|
||||
|
||||
#include "sql/engine/ob_operator.h"
|
||||
#include "sql/engine/ob_no_children_phy_operator.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
#include "sql/engine/ob_single_child_phy_operator.h"
|
||||
#include "sql/engine/basic/ob_ra_row_store.h"
|
||||
#include "sql/engine/basic/ob_chunk_row_store.h"
|
||||
#include "sql/engine/ob_sql_mem_mgr_processor.h"
|
||||
#include "sql/engine/ob_tenant_sql_memory_manager.h"
|
||||
#include "sql/executor/ob_interm_result_manager.h"
|
||||
#include "sql/dtl/ob_dtl_interm_result_manager.h"
|
||||
#include "sql/engine/ob_physical_plan_ctx.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObExecContext;
|
||||
class ObTempTableTransformationOpSpec : public ObOpSpec {
|
||||
public:
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObTempTableTransformationOpSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type) : ObOpSpec(alloc, type)
|
||||
{}
|
||||
virtual ~ObTempTableTransformationOpSpec()
|
||||
{}
|
||||
DECLARE_VIRTUAL_TO_STRING;
|
||||
};
|
||||
|
||||
class ObTempTableTransformationOp : public ObOperator {
|
||||
public:
|
||||
ObTempTableTransformationOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObOperator(exec_ctx, spec, input)
|
||||
{}
|
||||
~ObTempTableTransformationOp()
|
||||
{}
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int rescan() override;
|
||||
virtual int inner_get_next_row() override;
|
||||
virtual int inner_close() override;
|
||||
virtual void destroy() override;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SRC_SQL_ENGINE_BASIC_OB_TEMP_TABLE_TRANSFORMATION_OP_H_ */
|
||||
390
src/sql/engine/basic/ob_topk.cpp
Normal file
390
src/sql/engine/basic/ob_topk.cpp
Normal file
@ -0,0 +1,390 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/basic/ob_topk.h"
|
||||
#include "lib/utility/utility.h"
|
||||
#include "share/object/ob_obj_cast.h"
|
||||
#include "sql/session/ob_sql_session_info.h"
|
||||
#include "sql/engine/ob_physical_plan.h"
|
||||
#include "sql/engine/expr/ob_sql_expression.h"
|
||||
#include "sql/engine/sort/ob_sort.h"
|
||||
#include "sql/engine/basic/ob_material.h"
|
||||
#include "sql/engine/aggregate/ob_hash_groupby.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
|
||||
class ObTopK::ObTopKCtx : public ObPhyOperatorCtx {
|
||||
public:
|
||||
explicit ObTopKCtx(ObExecContext& ctx) : ObPhyOperatorCtx(ctx), topk_final_count_(-1), output_count_(0)
|
||||
{}
|
||||
virtual void destroy()
|
||||
{
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
|
||||
private:
|
||||
int64_t topk_final_count_; // count of rows that need to be output upforward
|
||||
int64_t output_count_;
|
||||
friend class ObTopK;
|
||||
};
|
||||
|
||||
ObTopK::ObTopK(ObIAllocator& alloc)
|
||||
: ObSingleChildPhyOperator(alloc), minimum_row_count_(-1), topk_precision_(-1), org_limit_(NULL), org_offset_(NULL)
|
||||
{}
|
||||
|
||||
ObTopK::~ObTopK()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
void ObTopK::reset()
|
||||
{
|
||||
org_limit_ = NULL;
|
||||
org_offset_ = NULL;
|
||||
minimum_row_count_ = -1;
|
||||
topk_precision_ = -1;
|
||||
ObSingleChildPhyOperator::reset();
|
||||
}
|
||||
|
||||
void ObTopK::reuse()
|
||||
{
|
||||
reset();
|
||||
}
|
||||
|
||||
int ObTopK::set_topk_params(
|
||||
ObSqlExpression* limit, ObSqlExpression* offset, int64_t minimum_row_count, int64_t topk_precision)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(limit) || minimum_row_count < 0 || topk_precision < 0) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("invalid arguments", KP(limit), K(minimum_row_count), K(topk_precision), K(ret));
|
||||
} else {
|
||||
org_limit_ = limit;
|
||||
org_offset_ = offset;
|
||||
minimum_row_count_ = minimum_row_count;
|
||||
topk_precision_ = topk_precision;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTopK::get_int_value(
|
||||
ObExecContext& ctx, const ObSqlExpression* in_val, int64_t& out_val, bool& is_null_value) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObNewRow input_row;
|
||||
ObObj result;
|
||||
ObExprCtx expr_ctx;
|
||||
is_null_value = false;
|
||||
if (in_val != NULL && !in_val->is_empty()) {
|
||||
if (OB_FAIL(wrap_expr_ctx(ctx, expr_ctx))) {
|
||||
LOG_WARN("wrap expr context failed", K(ret));
|
||||
} else if (OB_FAIL(in_val->calc(expr_ctx, input_row, result))) {
|
||||
LOG_WARN("Failed to calculate expression", K(ret));
|
||||
} else if (OB_LIKELY(result.is_int())) {
|
||||
if (OB_FAIL(result.get_int(out_val))) {
|
||||
LOG_WARN("get_int error", K(ret), K(result));
|
||||
}
|
||||
} else if (result.is_null()) {
|
||||
out_val = 0;
|
||||
is_null_value = true;
|
||||
} else {
|
||||
EXPR_DEFINE_CAST_CTX(expr_ctx, CM_NONE);
|
||||
EXPR_GET_INT64_V2(result, out_val);
|
||||
if (OB_FAIL(ret)) {
|
||||
LOG_WARN("get_int error", K(ret), K(result));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTopK::get_topk_final_count(ObExecContext& ctx, int64_t& topk_final_count) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
||||
int64_t limit = -1;
|
||||
int64_t offset = 0;
|
||||
bool is_null_value = false;
|
||||
ObPhysicalPlanCtx* plan_ctx = ctx.get_physical_plan_ctx();
|
||||
if (OB_ISNULL(plan_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid plan ctx is NULL", K(ret));
|
||||
} else if (OB_ISNULL(child_op_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("Invalid child_op_ is NULL", K(ret));
|
||||
} else if (OB_FAIL(get_int_value(ctx, org_limit_, limit, is_null_value))) {
|
||||
LOG_WARN("Get limit value failed", K(ret));
|
||||
} else if (!is_null_value && OB_FAIL(get_int_value(ctx, org_offset_, offset, is_null_value))) {
|
||||
LOG_WARN("Get offset value failed", K(ret));
|
||||
} else {
|
||||
// revise limit, offset because rownum < -1 is rewritten as limit -1
|
||||
limit = (is_null_value || limit < 0) ? 0 : limit;
|
||||
offset = (is_null_value || offset < 0) ? 0 : offset;
|
||||
topk_final_count = std::max(minimum_row_count_, limit + offset);
|
||||
int64_t row_count = 0;
|
||||
ObPhyOperatorType op_type = child_op_->get_type();
|
||||
// TODO(): may be we should add one func to paremt class
|
||||
if (PHY_SORT == op_type) {
|
||||
ObSort* sort_op = static_cast<ObSort*>(child_op_);
|
||||
if (OB_ISNULL(sort_op)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("casted sort_op is NULL", K(ret));
|
||||
} else if (OB_FAIL(sort_op->get_sort_row_count(ctx, row_count))) {
|
||||
LOG_WARN("failed to get sort row count", K(ret));
|
||||
} else { /*do nothing*/
|
||||
}
|
||||
} else if (PHY_MATERIAL == op_type) {
|
||||
ObMaterial* material_op = static_cast<ObMaterial*>(child_op_);
|
||||
if (OB_ISNULL(material_op)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("casted material_op is NULL", K(ret));
|
||||
} else if (OB_FAIL(material_op->get_material_row_count(ctx, row_count))) {
|
||||
LOG_WARN("failed to get material row count", K(ret));
|
||||
} else { /*do nothing*/
|
||||
}
|
||||
} else if (PHY_HASH_GROUP_BY == op_type) {
|
||||
ObHashGroupBy* hash_groupby_op = static_cast<ObHashGroupBy*>(child_op_);
|
||||
if (OB_ISNULL(hash_groupby_op)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("casted hash_groupby_op is NULL", K(ret));
|
||||
} else if (OB_FAIL(hash_groupby_op->get_hash_groupby_row_count(ctx, row_count))) {
|
||||
LOG_WARN("failed to get material row count", K(ret));
|
||||
} else { /*do nothing*/
|
||||
}
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("Invalid child_op_", K(op_type), K(ret));
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
topk_final_count = std::max(topk_final_count, static_cast<int64_t>(row_count * topk_precision_ / 100));
|
||||
if (topk_final_count >= row_count) {
|
||||
plan_ctx->set_is_result_accurate(true);
|
||||
} else {
|
||||
plan_ctx->set_is_result_accurate(false);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObTopK::is_valid() const
|
||||
{
|
||||
return (get_column_count() > 0 && (NULL != org_limit_) && (NULL != child_op_) && child_op_->get_column_count() > 0);
|
||||
}
|
||||
|
||||
int ObTopK::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_UNLIKELY(!is_valid())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("limit operator is invalid");
|
||||
} else if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("initialize operator context failed", K(ret));
|
||||
} else { /*do nothing*/
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTopK::rescan(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObTopKCtx* topk_ctx = NULL;
|
||||
if (OB_FAIL(ObSingleChildPhyOperator::rescan(ctx))) {
|
||||
LOG_WARN("rescan child physical operator failed", K(ret));
|
||||
} else if (OB_ISNULL(topk_ctx = GET_PHY_OPERATOR_CTX(ObTopKCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("topk_ctx is null");
|
||||
} else {
|
||||
topk_ctx->output_count_ = 0;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTopK::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
UNUSED(ctx);
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
int ObTopK::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhyOperatorCtx* op_ctx = NULL;
|
||||
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObTopKCtx, ctx, get_id(), get_type(), op_ctx))) {
|
||||
LOG_WARN("failed to create TopKCtx", K(ret));
|
||||
} else if (OB_ISNULL(op_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("op_ctx is null");
|
||||
} else if (OB_FAIL(init_cur_row(*op_ctx, need_copy_row_for_compute()))) {
|
||||
LOG_WARN("init current row failed", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTopK::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObTopKCtx* topk_ctx = NULL;
|
||||
ObSQLSessionInfo* my_session = NULL;
|
||||
const ObNewRow* input_row = NULL;
|
||||
|
||||
if (OB_FAIL(!is_valid())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("limit operator is invalid");
|
||||
} else if (OB_ISNULL(topk_ctx = GET_PHY_OPERATOR_CTX(ObTopKCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get physical operator ctx failed");
|
||||
} else if (OB_ISNULL(my_session = GET_MY_SESSION(ctx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get my session", K(ret));
|
||||
} else {
|
||||
if (0 == topk_ctx->output_count_ || topk_ctx->output_count_ < topk_ctx->topk_final_count_) {
|
||||
if (OB_FAIL(child_op_->get_next_row(ctx, input_row))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN(
|
||||
"child_op failed to get next row", K_(topk_ctx->topk_final_count), K_(topk_ctx->output_count), K(ret));
|
||||
}
|
||||
} else {
|
||||
if (0 == topk_ctx->output_count_) {
|
||||
if (OB_FAIL(get_topk_final_count(ctx, topk_ctx->topk_final_count_))) {
|
||||
LOG_WARN("failed to get_topk_final_count", K(ret));
|
||||
} else if (OB_UNLIKELY(0 == topk_ctx->topk_final_count_)) {
|
||||
ret = OB_ITER_END;
|
||||
} else { /*do nothing*/
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
++topk_ctx->output_count_;
|
||||
row = input_row;
|
||||
if (OB_FAIL(copy_cur_row(*topk_ctx, row))) {
|
||||
LOG_WARN("copy current row failed", K(ret));
|
||||
} else {
|
||||
LOG_DEBUG("copy cur row", K(*row));
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObTopK::to_string_kv(char* buf, const int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
// TODO():macro define N_TOPK_PRECISION
|
||||
if (org_limit_ && org_offset_) {
|
||||
J_KV(N_LIMIT,
|
||||
org_limit_,
|
||||
N_OFFSET,
|
||||
org_offset_,
|
||||
"minimum_row_count",
|
||||
minimum_row_count_,
|
||||
"topk_precision",
|
||||
topk_precision_);
|
||||
} else if (NULL != org_limit_) {
|
||||
J_KV(N_LIMIT, org_limit_, "minimum_row_count", minimum_row_count_, "topk_precision", topk_precision_);
|
||||
} else if (NULL != org_offset_) {
|
||||
J_KV(N_OFFSET, org_offset_, "minimum_row_count", minimum_row_count_, "topk_precision", topk_precision_);
|
||||
} else { /*do nothing*/
|
||||
}
|
||||
return pos;
|
||||
}
|
||||
|
||||
int ObTopK::add_filter(ObSqlExpression* expr)
|
||||
{
|
||||
UNUSED(expr);
|
||||
LOG_ERROR("limit operator should have no filter expr");
|
||||
return OB_NOT_SUPPORTED;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE(ObTopK)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool has_limit_count = (org_limit_ != NULL && !org_limit_->is_empty());
|
||||
bool has_limit_offset = (org_offset_ != NULL && !org_offset_->is_empty());
|
||||
|
||||
OB_UNIS_ENCODE(minimum_row_count_);
|
||||
OB_UNIS_ENCODE(topk_precision_);
|
||||
OB_UNIS_ENCODE(has_limit_count);
|
||||
OB_UNIS_ENCODE(has_limit_offset);
|
||||
if (has_limit_count) {
|
||||
OB_UNIS_ENCODE(*org_limit_);
|
||||
}
|
||||
if (has_limit_offset) {
|
||||
OB_UNIS_ENCODE(*org_offset_);
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(ObSingleChildPhyOperator::serialize(buf, buf_len, pos))) {
|
||||
LOG_WARN("serialize child operator failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
OB_DEF_SERIALIZE_SIZE(ObTopK)
|
||||
{
|
||||
int64_t len = 0;
|
||||
bool has_limit_count = (org_limit_ != NULL && !org_limit_->is_empty());
|
||||
bool has_limit_offset = (org_offset_ != NULL && !org_offset_->is_empty());
|
||||
|
||||
OB_UNIS_ADD_LEN(minimum_row_count_);
|
||||
OB_UNIS_ADD_LEN(topk_precision_);
|
||||
OB_UNIS_ADD_LEN(has_limit_count);
|
||||
OB_UNIS_ADD_LEN(has_limit_offset);
|
||||
if (has_limit_count) {
|
||||
OB_UNIS_ADD_LEN(*org_limit_);
|
||||
}
|
||||
if (has_limit_offset) {
|
||||
OB_UNIS_ADD_LEN(*org_offset_);
|
||||
}
|
||||
len += ObSingleChildPhyOperator::get_serialize_size();
|
||||
return len;
|
||||
}
|
||||
|
||||
OB_DEF_DESERIALIZE(ObTopK)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool has_limit_count = false;
|
||||
bool has_limit_offset = false;
|
||||
|
||||
OB_UNIS_DECODE(minimum_row_count_);
|
||||
OB_UNIS_DECODE(topk_precision_);
|
||||
OB_UNIS_DECODE(has_limit_count);
|
||||
OB_UNIS_DECODE(has_limit_offset);
|
||||
|
||||
if (OB_SUCC(ret) && has_limit_count) {
|
||||
if (OB_FAIL(ObSqlExpressionUtil::make_sql_expr(my_phy_plan_, org_limit_))) {
|
||||
LOG_WARN("make sql expression failed", K(ret));
|
||||
} else if (OB_LIKELY(org_limit_ != NULL)) {
|
||||
OB_UNIS_DECODE(*org_limit_);
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && has_limit_offset) {
|
||||
if (OB_FAIL(ObSqlExpressionUtil::make_sql_expr(my_phy_plan_, org_offset_))) {
|
||||
LOG_WARN("make sql expression failed", K(ret));
|
||||
} else if (OB_LIKELY(org_offset_ != NULL)) {
|
||||
OB_UNIS_DECODE(*org_offset_);
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
ret = ObSingleChildPhyOperator::deserialize(buf, data_len, pos);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
85
src/sql/engine/basic/ob_topk.h
Normal file
85
src/sql/engine/basic/ob_topk.h
Normal file
@ -0,0 +1,85 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef _OB_TOPK_H
|
||||
#define _OB_TOPK_H 1
|
||||
#include "sql/engine/ob_single_child_phy_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObSqlExpression;
|
||||
|
||||
class ObTopK : public ObSingleChildPhyOperator {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
private:
|
||||
class ObTopKCtx;
|
||||
|
||||
public:
|
||||
explicit ObTopK(common::ObIAllocator& alloc);
|
||||
virtual ~ObTopK();
|
||||
|
||||
virtual void reset();
|
||||
virtual void reuse();
|
||||
int set_topk_params(
|
||||
ObSqlExpression* limit, ObSqlExpression* offset, int64_t minimum_row_count, int64_t topk_precision);
|
||||
virtual int rescan(ObExecContext& ctx) const;
|
||||
|
||||
private:
|
||||
bool is_valid() const;
|
||||
int get_int_value(ObExecContext& ctx, const ObSqlExpression* in_val, int64_t& out_val, bool& is_null_value) const;
|
||||
/**
|
||||
* @brief init operator context, will create a physical operator context (and a current row space)
|
||||
* @param ctx[in], execute context
|
||||
* @return if success, return OB_SUCCESS, otherwise, return errno
|
||||
*/
|
||||
int init_op_ctx(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief: called by get_next_row(), get a row from the child operator or row_store
|
||||
* @param: ctx[in], execute context
|
||||
* @param: row[out], ObSqlRow an obj array and row_size
|
||||
*/
|
||||
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
/**
|
||||
* @brief open operator, not including children operators.
|
||||
* called by open.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_open(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief close operator, not including children operators.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_close(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief for specified phy operator to print it's member variable with json key-value format
|
||||
* @param buf[in] to string buffer
|
||||
* @param buf_len[in] buffer length
|
||||
* @return if success, return the length used by print string, otherwise return 0
|
||||
*/
|
||||
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
|
||||
int add_filter(ObSqlExpression* expr);
|
||||
int get_topk_final_count(ObExecContext& ctx, int64_t& topk_final_count) const;
|
||||
// disallow copy
|
||||
DISALLOW_COPY_AND_ASSIGN(ObTopK);
|
||||
|
||||
private:
|
||||
// data members
|
||||
int64_t minimum_row_count_;
|
||||
int64_t topk_precision_;
|
||||
ObSqlExpression* org_limit_;
|
||||
ObSqlExpression* org_offset_;
|
||||
};
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
||||
#endif /* _OB_TOPK_H */
|
||||
140
src/sql/engine/basic/ob_topk_op.cpp
Normal file
140
src/sql/engine/basic/ob_topk_op.cpp
Normal file
@ -0,0 +1,140 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/basic/ob_topk_op.h"
|
||||
#include "sql/engine/basic/ob_limit_op.h"
|
||||
#include "sql/engine/sort/ob_sort_op.h"
|
||||
#include "sql/engine/aggregate/ob_hash_groupby_op.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
using namespace oceanbase::common;
|
||||
|
||||
ObTopKSpec::ObTopKSpec(ObIAllocator& alloc, const ObPhyOperatorType type)
|
||||
: ObOpSpec(alloc, type), minimum_row_count_(-1), topk_precision_(-1), org_limit_(NULL), org_offset_(NULL)
|
||||
{}
|
||||
|
||||
bool ObTopKSpec::is_valid() const
|
||||
{
|
||||
return (get_output_count() > 0) && (NULL != org_limit_) && (NULL != child_) && (child_->get_output_count() > 0);
|
||||
}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObTopKSpec, ObOpSpec), minimum_row_count_, topk_precision_, org_limit_, org_offset_);
|
||||
|
||||
ObTopKOp::ObTopKOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObOperator(exec_ctx, spec, input), topk_final_count_(-1), output_count_(0)
|
||||
{}
|
||||
|
||||
int ObTopKOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (!MY_SPEC.is_valid()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("topk operator is invalid", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTopKOp::rescan()
|
||||
{
|
||||
output_count_ = 0;
|
||||
return ObOperator::rescan();
|
||||
}
|
||||
|
||||
int ObTopKOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (0 == output_count_ || output_count_ < topk_final_count_) {
|
||||
if (OB_FAIL(child_->get_next_row())) {
|
||||
if (OB_ITER_END == ret) {
|
||||
LOG_WARN("child get next row", K(ret), K(output_count_), K(topk_final_count_));
|
||||
}
|
||||
} else {
|
||||
if (0 == output_count_) {
|
||||
if (OB_FAIL(get_topk_final_count())) {
|
||||
LOG_WARN("get topk count failed", K(ret));
|
||||
} else if (OB_UNLIKELY(0 == topk_final_count_)) {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
clear_evaluated_flag();
|
||||
++output_count_;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
ret = OB_ITER_END;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTopKOp::get_topk_final_count()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t limit = -1;
|
||||
int64_t offset = 0;
|
||||
bool is_null_value = false;
|
||||
ObPhysicalPlanCtx* plan_ctx = ctx_.get_physical_plan_ctx();
|
||||
if (OB_ISNULL(child_) || OB_ISNULL(plan_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("child_ or plan_ctx is NULL", K(ret), KP(child_), KP(plan_ctx));
|
||||
} else if (OB_FAIL(ObLimitOp::get_int_val(MY_SPEC.org_limit_, eval_ctx_, limit, is_null_value))) {
|
||||
LOG_WARN("get limit values failed", K(ret));
|
||||
} else if (!is_null_value && OB_FAIL(ObLimitOp::get_int_val(MY_SPEC.org_offset_, eval_ctx_, offset, is_null_value))) {
|
||||
LOG_WARN("get offset values failed", K(ret));
|
||||
} else {
|
||||
// revise limit, offset because rownum < -1 is rewritten as limit -1
|
||||
limit = (is_null_value || limit < 0) ? 0 : limit;
|
||||
offset = (is_null_value || offset < 0) ? 0 : offset;
|
||||
topk_final_count_ = std::max(MY_SPEC.minimum_row_count_, limit + offset);
|
||||
int64_t row_count = 0;
|
||||
ObPhyOperatorType op_type = child_->get_spec().get_type();
|
||||
switch (op_type) {
|
||||
case PHY_SORT: {
|
||||
ObSortOp* sort_op = static_cast<ObSortOp*>(child_);
|
||||
if (OB_FAIL(row_count = sort_op->get_sort_row_count())) {
|
||||
LOG_WARN("get sort row count failed", K(ret));
|
||||
}
|
||||
break;
|
||||
}
|
||||
case PHY_MATERIAL: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("material not implimented yet", K(ret));
|
||||
break;
|
||||
}
|
||||
case PHY_HASH_GROUP_BY: {
|
||||
ObHashGroupByOp* gby_op = static_cast<ObHashGroupByOp*>(child_);
|
||||
row_count = gby_op->get_hash_groupby_row_count();
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid child_ for topk operator", K(ret), K(op_type));
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret)) {
|
||||
topk_final_count_ = std::max(topk_final_count_, static_cast<int64_t>(row_count * MY_SPEC.topk_precision_ / 100));
|
||||
if (topk_final_count_ >= row_count) {
|
||||
plan_ctx->set_is_result_accurate(true);
|
||||
} else {
|
||||
plan_ctx->set_is_result_accurate(false);
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
67
src/sql/engine/basic/ob_topk_op.h
Normal file
67
src/sql/engine/basic/ob_topk_op.h
Normal file
@ -0,0 +1,67 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_TOPK_OP_H_
|
||||
#define OCEANBASE_BASIC_OB_TOPK_OP_H_
|
||||
|
||||
#include "sql/engine/ob_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
class ObTopKSpec : public ObOpSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObTopKSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
|
||||
bool is_valid() const;
|
||||
|
||||
INHERIT_TO_STRING_KV(
|
||||
"op_spec", ObOpSpec, K_(minimum_row_count), K_(topk_precision), KPC_(org_limit), KPC_(org_offset));
|
||||
|
||||
int64_t minimum_row_count_;
|
||||
int64_t topk_precision_;
|
||||
ObExpr* org_limit_;
|
||||
ObExpr* org_offset_;
|
||||
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObTopKSpec);
|
||||
};
|
||||
|
||||
class ObTopKOp : public ObOperator {
|
||||
public:
|
||||
ObTopKOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int rescan() override;
|
||||
|
||||
virtual int inner_get_next_row() override;
|
||||
|
||||
virtual void destroy() override
|
||||
{
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
private:
|
||||
int get_topk_final_count();
|
||||
|
||||
DISALLOW_COPY_AND_ASSIGN(ObTopKOp);
|
||||
|
||||
int64_t topk_final_count_;
|
||||
int64_t output_count_;
|
||||
};
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
|
||||
#endif // OCEANBASE_BASIC_OB_TOPK_OP_H_
|
||||
142
src/sql/engine/basic/ob_values.cpp
Normal file
142
src/sql/engine/basic/ob_values.cpp
Normal file
@ -0,0 +1,142 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
#include "sql/engine/basic/ob_values.h"
|
||||
|
||||
namespace oceanbase {
|
||||
using namespace common;
|
||||
namespace sql {
|
||||
OB_SERIALIZE_MEMBER((ObValues, ObNoChildrenPhyOperator), row_store_);
|
||||
|
||||
class ObValues::ObValuesCtx : public ObPhyOperatorCtx {
|
||||
public:
|
||||
explicit ObValuesCtx(ObExecContext& ctx) : ObPhyOperatorCtx(ctx), row_store_it_()
|
||||
{}
|
||||
virtual void destroy()
|
||||
{
|
||||
ObPhyOperatorCtx::destroy_base();
|
||||
}
|
||||
|
||||
private:
|
||||
ObRowStore::Iterator row_store_it_;
|
||||
friend class ObValues;
|
||||
};
|
||||
|
||||
void ObValues::reset()
|
||||
{
|
||||
row_store_.reset();
|
||||
ObNoChildrenPhyOperator::reset();
|
||||
}
|
||||
|
||||
void ObValues::reuse()
|
||||
{
|
||||
row_store_.reuse();
|
||||
ObNoChildrenPhyOperator::reuse();
|
||||
}
|
||||
// not used so far, may be use later
|
||||
int ObValues::add_row(const ObNewRow& row)
|
||||
{
|
||||
return row_store_.add_row(row);
|
||||
}
|
||||
|
||||
int ObValues::inner_open(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObValuesCtx* values_ctx = NULL;
|
||||
|
||||
if (OB_FAIL(init_op_ctx(ctx))) {
|
||||
LOG_WARN("fail to init operator context", K(ret));
|
||||
} else if (OB_ISNULL(values_ctx = GET_PHY_OPERATOR_CTX(ObValuesCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get physical operator context");
|
||||
} else {
|
||||
values_ctx->row_store_it_ = row_store_.begin();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
// not used so far, may be use later
|
||||
int ObValues::rescan(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObValuesCtx* values_ctx = NULL;
|
||||
if (OB_FAIL(ObNoChildrenPhyOperator::rescan(ctx))) {
|
||||
LOG_WARN("rescan ObNoChildrenPhyOperator failed", K(ret));
|
||||
} else if (OB_ISNULL(values_ctx = GET_PHY_OPERATOR_CTX(ObValuesCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("values_ctx is null");
|
||||
} else {
|
||||
values_ctx->row_store_it_ = row_store_.begin();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObValues::inner_close(ObExecContext& ctx) const
|
||||
{
|
||||
UNUSED(ctx);
|
||||
return OB_SUCCESS;
|
||||
}
|
||||
|
||||
int ObValues::init_op_ctx(ObExecContext& ctx) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObPhyOperatorCtx* op_ctx = NULL;
|
||||
if (OB_FAIL(CREATE_PHY_OPERATOR_CTX(ObValuesCtx, ctx, get_id(), get_type(), op_ctx))) {
|
||||
LOG_WARN("create physical operator context failed", K(ret));
|
||||
} else if (OB_ISNULL(op_ctx)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("op_ctx is null");
|
||||
} else if (OB_FAIL(op_ctx->create_cur_row(get_column_count(), projector_, projector_size_))) {
|
||||
LOG_WARN("create current row failed", K(ret), "#columns", get_column_count());
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObValues::inner_get_next_row(ObExecContext& ctx, const ObNewRow*& row) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObValuesCtx* values_ctx = NULL;
|
||||
|
||||
if (OB_ISNULL(values_ctx = GET_PHY_OPERATOR_CTX(ObValuesCtx, ctx, get_id()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("fail to get physical operator context");
|
||||
} else if (OB_FAIL(THIS_WORKER.check_status())) {
|
||||
LOG_WARN("check physical plan status failed", K(ret));
|
||||
} else if (OB_FAIL(values_ctx->row_store_it_.get_next_row(values_ctx->get_cur_row()))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("fail to get next row", K(ret));
|
||||
}
|
||||
} else {
|
||||
row = &values_ctx->get_cur_row();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int64_t ObValues::to_string_kv(char* buf, const int64_t buf_len) const
|
||||
{
|
||||
int64_t pos = 0;
|
||||
J_KV(N_ROW_STORE, row_store_);
|
||||
return pos;
|
||||
}
|
||||
|
||||
int ObValues::set_row_store(const common::ObRowStore& row_store)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(row_store_.assign(row_store))) {
|
||||
LOG_WARN("fail to assign row store", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
71
src/sql/engine/basic/ob_values.h
Normal file
71
src/sql/engine/basic/ob_values.h
Normal file
@ -0,0 +1,71 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_SQL_ENGINE_DML_OB_VALUES_H_
|
||||
#define OCEANBASE_SQL_ENGINE_DML_OB_VALUES_H_
|
||||
#include "common/row/ob_row_store.h"
|
||||
#include "sql/engine/ob_no_children_phy_operator.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
class ObValues : public ObNoChildrenPhyOperator {
|
||||
OB_UNIS_VERSION(1);
|
||||
class ObValuesCtx;
|
||||
|
||||
public:
|
||||
explicit ObValues(common::ObIAllocator& alloc) : ObNoChildrenPhyOperator(alloc), row_store_(alloc)
|
||||
{}
|
||||
~ObValues()
|
||||
{}
|
||||
|
||||
void reset();
|
||||
void reuse();
|
||||
int add_row(const common::ObNewRow& row);
|
||||
|
||||
int rescan(ObExecContext& ctx) const;
|
||||
int set_row_store(const common::ObRowStore& row_store);
|
||||
|
||||
private:
|
||||
/**
|
||||
* @brief init operator context, will create a physical operator context (and a current row space)
|
||||
* @param ctx[in], execute context
|
||||
* @return if success, return OB_SUCCESS, otherwise, return errno
|
||||
*/
|
||||
virtual int init_op_ctx(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief called by get_next_row(), get a row from the child operator or row_store
|
||||
* @param ctx[in], execute context
|
||||
* @param row[out], ObSqlRow an obj array and row_size
|
||||
*/
|
||||
virtual int inner_get_next_row(ObExecContext& ctx, const common::ObNewRow*& row) const;
|
||||
/**
|
||||
* @brief open operator, not including children operators.
|
||||
* called by open.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_open(ObExecContext& ctx) const;
|
||||
/**
|
||||
* @brief close operator, not including children operators.
|
||||
* Every op should implement this method.
|
||||
*/
|
||||
virtual int inner_close(ObExecContext& ctx) const;
|
||||
virtual int64_t to_string_kv(char* buf, const int64_t buf_len) const;
|
||||
DISALLOW_COPY_AND_ASSIGN(ObValues);
|
||||
|
||||
private:
|
||||
common::ObRowStore row_store_;
|
||||
};
|
||||
} // namespace sql
|
||||
} // namespace oceanbase
|
||||
|
||||
#endif /* OCEANBASE_SQL_ENGINE_DML_OB_VALUES_H_ */
|
||||
85
src/sql/engine/basic/ob_values_op.cpp
Normal file
85
src/sql/engine/basic/ob_values_op.cpp
Normal file
@ -0,0 +1,85 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#define USING_LOG_PREFIX SQL_ENG
|
||||
|
||||
#include "ob_values_op.h"
|
||||
#include "sql/engine/ob_exec_context.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
using namespace common;
|
||||
|
||||
ObValuesSpec::ObValuesSpec(ObIAllocator& alloc, const ObPhyOperatorType type) : ObOpSpec(alloc, type), row_store_(alloc)
|
||||
{}
|
||||
|
||||
OB_SERIALIZE_MEMBER((ObValuesSpec, ObOpSpec), row_store_);
|
||||
|
||||
ObValuesOp::ObValuesOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input)
|
||||
: ObOperator(exec_ctx, spec, input)
|
||||
{}
|
||||
|
||||
int ObValuesOp::inner_open()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
row_store_it_ = MY_SPEC.row_store_.begin();
|
||||
ObObj* cells = static_cast<ObObj*>(ctx_.get_allocator().alloc(sizeof(ObObj) * MY_SPEC.output_.count()));
|
||||
if (OB_ISNULL(cells)) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_WARN("alloc memory failed", K(ret));
|
||||
} else {
|
||||
for (int64_t i = 0; i < MY_SPEC.output_.count(); i++) {
|
||||
new (&cells[i]) ObObj();
|
||||
}
|
||||
cur_row_.cells_ = cells;
|
||||
cur_row_.count_ = MY_SPEC.output_.count();
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObValuesOp::rescan()
|
||||
{
|
||||
row_store_it_ = MY_SPEC.row_store_.begin();
|
||||
return ObOperator::rescan();
|
||||
}
|
||||
|
||||
int ObValuesOp::inner_get_next_row()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_FAIL(row_store_it_.get_next_row(cur_row_))) {
|
||||
if (OB_ITER_END != ret) {
|
||||
LOG_WARN("get row from row store failed", K(ret));
|
||||
}
|
||||
} else {
|
||||
clear_evaluated_flag();
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.output_.count(); i++) {
|
||||
const ObObj& cell = cur_row_.cells_[i];
|
||||
ObDatum& datum = MY_SPEC.output_.at(i)->locate_datum_for_write(eval_ctx_);
|
||||
ObEvalInfo& eval_info = MY_SPEC.output_.at(i)->get_eval_info(eval_ctx_);
|
||||
ObExpr* expr = MY_SPEC.output_.at(i);
|
||||
if (cell.is_null()) {
|
||||
datum.set_null();
|
||||
} else if (cell.get_type() != expr->datum_meta_.type_) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("type mismatch", K(ret), K(i), K(cell.get_type()), K(*expr));
|
||||
} else if (OB_FAIL(datum.from_obj(cell, expr->obj_datum_map_))) {
|
||||
LOG_WARN("convert obj to datum failed", K(ret));
|
||||
} else {
|
||||
eval_info.evaluated_ = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
55
src/sql/engine/basic/ob_values_op.h
Normal file
55
src/sql/engine/basic/ob_values_op.h
Normal file
@ -0,0 +1,55 @@
|
||||
/**
|
||||
* Copyright (c) 2021 OceanBase
|
||||
* OceanBase CE is licensed under Mulan PubL v2.
|
||||
* You can use this software according to the terms and conditions of the Mulan PubL v2.
|
||||
* You may obtain a copy of Mulan PubL v2 at:
|
||||
* http://license.coscl.org.cn/MulanPubL-2.0
|
||||
* THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND,
|
||||
* EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT,
|
||||
* MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE.
|
||||
* See the Mulan PubL v2 for more details.
|
||||
*/
|
||||
|
||||
#ifndef OCEANBASE_BASIC_OB_VALUES_OP_H_
|
||||
#define OCEANBASE_BASIC_OB_VALUES_OP_H_
|
||||
|
||||
#include "common/row/ob_row_store.h"
|
||||
#include "sql/engine/ob_operator.h"
|
||||
|
||||
namespace oceanbase {
|
||||
namespace sql {
|
||||
|
||||
//
|
||||
// Values operator is only use in explain statement to hold the explain result.
|
||||
//
|
||||
class ObValuesSpec : public ObOpSpec {
|
||||
OB_UNIS_VERSION_V(1);
|
||||
|
||||
public:
|
||||
ObValuesSpec(common::ObIAllocator& alloc, const ObPhyOperatorType type);
|
||||
|
||||
common::ObRowStore row_store_;
|
||||
};
|
||||
|
||||
class ObValuesOp : public ObOperator {
|
||||
public:
|
||||
ObValuesOp(ObExecContext& exec_ctx, const ObOpSpec& spec, ObOpInput* input);
|
||||
|
||||
virtual int inner_open() override;
|
||||
virtual int rescan() override;
|
||||
|
||||
virtual int inner_get_next_row() override;
|
||||
|
||||
virtual void destroy() override
|
||||
{
|
||||
ObOperator::destroy();
|
||||
}
|
||||
|
||||
private:
|
||||
common::ObNewRow cur_row_;
|
||||
common::ObRowStore::Iterator row_store_it_;
|
||||
};
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
#endif // OCEANBASE_BASIC_OB_VALUES_OP_H_
|
||||
Reference in New Issue
Block a user