init push

This commit is contained in:
oceanbase-admin
2021-05-31 22:56:52 +08:00
commit cea7de1475
7020 changed files with 5689869 additions and 0 deletions

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View 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_

View 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 */

View 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_ */

View 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

View 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_

View 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

View 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 */

View 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

View 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

View 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

View 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 */

View 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_ */

View 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;
}

File diff suppressed because it is too large Load Diff

View 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.
*/
#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;
}

File diff suppressed because it is too large Load Diff

View 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

View 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 */

View 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

View 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_

View 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

View 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_ */

View 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

View 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_ */

View 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

View 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

View 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

View 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_ */

View 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

View 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_

File diff suppressed because it is too large Load Diff

View 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_

File diff suppressed because it is too large Load Diff

View 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_

View 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

View 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_ */

View 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_

View 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

View 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_ */

View 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

View 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_ */

View 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

View 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_ */

View 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

View 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_ */

View 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

View 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_ */

View 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

View 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_ */

View 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

View 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 */

View 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

View 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_

View 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

View 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_ */

View 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

View 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_