[FEAT MERGE]4_1_sql_feature

Co-authored-by: leslieyuchen <leslieyuchen@gmail.com>
Co-authored-by: Charles0429 <xiezhenjiang@gmail.com>
Co-authored-by: raywill <hustos@gmail.com>
This commit is contained in:
obdev
2023-01-28 16:01:26 +08:00
committed by ob-robot
parent 3080f2b66f
commit 2d19a9d8f5
846 changed files with 161957 additions and 116661 deletions

View File

@ -66,30 +66,9 @@ void point2pointer(T *&dst_pointer, B *dst_base, T *src_pointer, const B *src_ba
}
int ObChunkDatumStore::StoredRow::to_expr_skip_const(const ObIArray<ObExpr*> &exprs, ObEvalCtx &ctx) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(cnt_ != exprs.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("datum count mismatch", K(ret), K(cnt_), K(exprs.count()));
} else {
for (uint32_t i = 0; i < cnt_; ++i) {
const ObExpr *expr = exprs.at(i);
if (expr->is_const_expr()) { // T_QUESTIONMARK is included in dynamic_const
continue;
} else {
expr->locate_expr_datum(ctx) = cells()[i];
expr->set_evaluated_projected(ctx);
LOG_DEBUG("succ to_expr", K(cnt_), K(exprs.count()),
KPC(exprs.at(i)), K(cells()[i]), K(lbt()));
}
}
}
return ret;
}
int ObChunkDatumStore::StoredRow::to_expr(const common::ObIArray<ObExpr*> &exprs,
ObEvalCtx &ctx, int64_t count) const
ObEvalCtx &ctx,
int64_t count) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(cnt_ < count || exprs.count() < count)) {
@ -97,8 +76,12 @@ int ObChunkDatumStore::StoredRow::to_expr(const common::ObIArray<ObExpr*> &exprs
LOG_WARN("datum count mismatch", K(ret), K(cnt_), K(exprs.count()), K(count));
} else {
for (uint32_t i = 0; i < count; ++i) {
exprs.at(i)->locate_expr_datum(ctx) = cells()[i];
exprs.at(i)->set_evaluated_projected(ctx);
if (exprs.at(i)->is_const_expr()) {
continue;
} else {
exprs.at(i)->locate_expr_datum(ctx) = cells()[i];
exprs.at(i)->set_evaluated_projected(ctx);
}
}
}
return ret;
@ -2360,38 +2343,6 @@ int ObChunkDatumStore::Iterator::get_next_batch(const StoredRow **rows,
return ret;
};
int ObChunkDatumStore::Iterator::get_next_batch(
const common::ObIArray<ObExpr*> &exprs, ObEvalCtx &ctx,
const int64_t max_rows, int64_t &read_rows, const StoredRow **rows)
{
int ret = OB_SUCCESS;
int64_t max_batch_size = ctx.max_batch_size_;
const StoredRow **srows = rows;
if (NULL == rows) {
if (!chunk_it_.is_valid()) {
ret = OB_NOT_INIT;
LOG_WARN("not init", K(ret));
} else if (OB_FAIL(chunk_it_.get_store()->init_batch_ctx(exprs.count(), max_batch_size))) {
LOG_WARN("init batch ctx failed", K(ret), K(max_batch_size));
} else {
srows = const_cast<const StoredRow **>(chunk_it_.get_store()->batch_ctx_->stored_rows_);
}
}
CK(max_rows <= max_batch_size);
if (OB_FAIL(ret)) {
} else if (OB_FAIL(get_next_batch(srows, max_rows, read_rows))) {
if (OB_ITER_END != ret) {
LOG_WARN("get next batch failed", K(ret), K(max_rows));
} else {
read_rows = 0;
}
} else {
attach_rows(exprs, ctx, srows, read_rows);
}
return ret;
}
int ObChunkDatumStore::RowIterator::convert_to_row(
const StoredRow *sr, const common::ObIArray<ObExpr*> &exprs, ObEvalCtx &ctx)
{
@ -2456,18 +2407,22 @@ int ObChunkDatumStore::RowIterator::get_next_batch(const common::ObIArray<ObExpr
if (OB_SUCC(ret)) {
for (int64_t col_idx = 0; col_idx < exprs.count(); col_idx++) {
ObExpr *e = exprs.at(col_idx);
ObDatum *datums = e->locate_batch_datums(ctx);
if (!e->is_batch_result()) {
datums[0] = rows[0]->cells()[col_idx];
if (e->is_const_expr()) {
continue;
} else {
for (int64_t i = 0; i < read_rows; i++) {
datums[i] = rows[i]->cells()[col_idx];
ObDatum *datums = e->locate_batch_datums(ctx);
if (!e->is_batch_result()) {
datums[0] = rows[0]->cells()[col_idx];
} else {
for (int64_t i = 0; i < read_rows; i++) {
datums[i] = rows[i]->cells()[col_idx];
}
}
e->set_evaluated_projected(ctx);
ObEvalInfo &info = e->get_eval_info(ctx);
info.notnull_ = false;
info.point_to_frame_ = false;
}
e->set_evaluated_projected(ctx);
ObEvalInfo &info = e->get_eval_info(ctx);
info.notnull_ = false;
info.point_to_frame_ = false;
}
}

View File

@ -95,9 +95,7 @@ public:
char *buf, const int64_t size, const int64_t row_size, const uint32_t row_extend_size);
template <bool fill_invariable_res_buf = false>
int to_expr(const common::ObIArray<ObExpr*> &exprs, ObEvalCtx &ctx) const;
int to_expr_skip_const(const common::ObIArray<ObExpr*> &exprs, ObEvalCtx &ctx) const;
int to_expr(const common::ObIArray<ObExpr*> &exprs, ObEvalCtx &ctx, int64_t count) const;
inline common::ObDatum *cells() { return reinterpret_cast<common::ObDatum *>(payload_); }
inline const common::ObDatum *cells() const
{ return reinterpret_cast<const common::ObDatum *>(payload_); }
@ -772,24 +770,20 @@ public:
int get_next_row(common::ObDatum **datums);
int get_next_row(const StoredRow *&sr);
template <bool fill_invariable_res_buf = false>
int get_next_row_skip_const(ObEvalCtx &ctx, const common::ObIArray<ObExpr*> &exprs);
int get_next_row(ObEvalCtx &ctx, const common::ObIArray<ObExpr*> &exprs);
// read next batch rows
// return OB_ITER_END and set %read_rows to zero for iterate end.
int get_next_batch(const StoredRow **rows, const int64_t max_rows, int64_t &read_rows);
int get_next_batch(const common::ObIArray<ObExpr*> &exprs, ObEvalCtx &ctx,
const int64_t max_rows, int64_t &read_rows, const StoredRow **rows = NULL);
template <bool fill_invariable_res_buf = false>
int get_next_batch_skip_const(const common::ObIArray<ObExpr*> &exprs, ObEvalCtx &ctx,
const int64_t max_rows, int64_t &read_rows,
const StoredRow **rows = NULL);
int get_next_batch(const common::ObIArray<ObExpr*> &exprs, ObEvalCtx &ctx,
const int64_t max_rows, int64_t &read_rows,
const StoredRow **rows = NULL);
// attach read store rows to expressions
template <bool fill_invariable_res_buf = false>
static void attach_rows(const common::ObIArray<ObExpr*> &exprs, ObEvalCtx &ctx,
const StoredRow **srows, const int64_t read_rows,
bool skip_const = false);
const StoredRow **srows, const int64_t read_rows);
int convert_to_row(const StoredRow *sr, const common::ObIArray<ObExpr*> &exprs, ObEvalCtx &ctx)
{ return row_it_.convert_to_row(sr, exprs, ctx); }
@ -1162,26 +1156,30 @@ int ObChunkDatumStore::StoredRow::to_expr(
} else {
for (uint32_t i = 0; i < cnt_; ++i) {
ObExpr *expr = exprs.at(i);
const ObDatum &src = cells()[i];
if (OB_UNLIKELY(fill_invariable_res_buf && !exprs.at(i)->is_variable_res_buf())) {
ObDatum &dst = expr->locate_datum_for_write(ctx);
dst.pack_ = src.pack_;
MEMCPY(const_cast<char *>(dst.ptr_), src.ptr_, src.len_);
if (expr->is_const_expr()) {
continue;
} else {
ObDatum &dst = expr->locate_expr_datum(ctx);
dst = src;
const ObDatum &src = cells()[i];
if (OB_UNLIKELY(fill_invariable_res_buf && !expr->is_variable_res_buf())) {
ObDatum &dst = expr->locate_datum_for_write(ctx);
dst.pack_ = src.pack_;
MEMCPY(const_cast<char *>(dst.ptr_), src.ptr_, src.len_);
} else {
ObDatum &dst = expr->locate_expr_datum(ctx);
dst = src;
}
expr->set_evaluated_projected(ctx);
SQL_ENG_LOG(DEBUG, "succ to_expr", K(cnt_), K(exprs.count()),
KPC(exprs.at(i)), K(cells()[i]), K(lbt()));
}
expr->set_evaluated_projected(ctx);
SQL_ENG_LOG(DEBUG, "succ to_expr", K(cnt_), K(exprs.count()),
KPC(exprs.at(i)), K(cells()[i]), K(lbt()));
}
}
return ret;
}
template <bool fill_invariable_res_buf>
int ObChunkDatumStore::Iterator::get_next_row_skip_const(ObEvalCtx &ctx,
const common::ObIArray<ObExpr*> &exprs)
int ObChunkDatumStore::Iterator::get_next_row(
ObEvalCtx &ctx, const common::ObIArray<ObExpr*> &exprs)
{
int ret = OB_SUCCESS;
const ObChunkDatumStore::StoredRow *sr = NULL;
@ -1225,14 +1223,13 @@ int ObChunkDatumStore::Iterator::get_next_row_skip_const(ObEvalCtx &ctx,
}
template <bool fill_invariable_res_buf>
int ObChunkDatumStore::Iterator::get_next_batch_skip_const(
int ObChunkDatumStore::Iterator::get_next_batch(
const common::ObIArray<ObExpr*> &exprs, ObEvalCtx &ctx,
const int64_t max_rows, int64_t &read_rows, const StoredRow **rows)
{
int ret = OB_SUCCESS;
int64_t max_batch_size = ctx.max_batch_size_;
const StoredRow **srows = rows;
bool skip_const = true;
if (NULL == rows) {
if (!chunk_it_.is_valid()) {
ret = OB_NOT_INIT;
@ -1255,7 +1252,7 @@ int ObChunkDatumStore::Iterator::get_next_batch_skip_const(
read_rows = 0;
}
} else {
attach_rows<fill_invariable_res_buf>(exprs, ctx, srows, read_rows, skip_const);
attach_rows<fill_invariable_res_buf>(exprs, ctx, srows, read_rows);
}
return ret;
@ -1264,13 +1261,13 @@ int ObChunkDatumStore::Iterator::get_next_batch_skip_const(
template <bool fill_invariable_res_buf>
void ObChunkDatumStore::Iterator::attach_rows(
const common::ObIArray<ObExpr*> &exprs, ObEvalCtx &ctx,
const StoredRow **srows, const int64_t read_rows, bool skip_const)
const StoredRow **srows, const int64_t read_rows)
{
// FIXME bin.lb: change to row style?
if (NULL != srows) {
for (int64_t col_idx = 0; col_idx < exprs.count(); col_idx++) {
ObExpr *e = exprs.at(col_idx);
if (skip_const && e->is_const_expr()) {
if (e->is_const_expr()) {
continue;
}
if (OB_LIKELY(!fill_invariable_res_buf || e->is_variable_res_buf())) {

View File

@ -1404,28 +1404,68 @@ ObPushdownExprSpec::ObPushdownExprSpec(ObIAllocator &alloc)
access_exprs_(alloc),
max_batch_size_(0),
pushdown_filters_(alloc),
filters_before_index_back_(alloc),
pd_storage_flag_(0),
pd_storage_filters_(alloc),
pd_storage_index_back_filters_(alloc),
pd_storage_aggregate_output_(alloc)
{
}
OB_SERIALIZE_MEMBER(ObPushdownExprSpec,
calc_exprs_,
access_exprs_,
max_batch_size_,
pushdown_filters_,
filters_before_index_back_,
pd_storage_flag_,
pd_storage_filters_,
pd_storage_index_back_filters_,
pd_storage_aggregate_output_);
OB_DEF_SERIALIZE(ObPushdownExprSpec)
{
int ret = OB_SUCCESS;
ExprFixedArray fake_filters_before_index_back(CURRENT_CONTEXT->get_allocator());
ObPushdownFilter fake_pd_storage_index_back_filters(CURRENT_CONTEXT->get_allocator());
LST_DO_CODE(OB_UNIS_ENCODE,
calc_exprs_,
access_exprs_,
max_batch_size_,
pushdown_filters_,
fake_filters_before_index_back, //mock a fake filters to compatible with 4.0
pd_storage_flag_,
pd_storage_filters_,
fake_pd_storage_index_back_filters, //mock a fake filters to compatible with 4.0
pd_storage_aggregate_output_);
return ret;
}
OB_DEF_DESERIALIZE(ObPushdownExprSpec)
{
int ret = OB_SUCCESS;
ExprFixedArray fake_filters_before_index_back(CURRENT_CONTEXT->get_allocator());
ObPushdownFilter fake_pd_storage_index_back_filters(CURRENT_CONTEXT->get_allocator());
LST_DO_CODE(OB_UNIS_DECODE,
calc_exprs_,
access_exprs_,
max_batch_size_,
pushdown_filters_,
fake_filters_before_index_back, //mock a fake filters to compatible with 4.0
pd_storage_flag_,
pd_storage_filters_,
fake_pd_storage_index_back_filters, //mock a fake filters to compatible with 4.0
pd_storage_aggregate_output_);
return ret;
}
OB_DEF_SERIALIZE_SIZE(ObPushdownExprSpec)
{
int64_t len = 0;
ExprFixedArray fake_filters_before_index_back(CURRENT_CONTEXT->get_allocator());
ObPushdownFilter fake_pd_storage_index_back_filters(CURRENT_CONTEXT->get_allocator());
LST_DO_CODE(OB_UNIS_ADD_LEN,
calc_exprs_,
access_exprs_,
max_batch_size_,
pushdown_filters_,
fake_filters_before_index_back, //mock a fake filters to compatible with 4.0
pd_storage_flag_,
pd_storage_filters_,
fake_pd_storage_index_back_filters, //mock a fake filters to compatible with 4.0
pd_storage_aggregate_output_);
return len;
}
ObPushdownOperator::ObPushdownOperator(ObEvalCtx &eval_ctx, const ObPushdownExprSpec &expr_spec)
: pd_storage_filters_(nullptr),
pd_storage_index_back_filters_(nullptr),
eval_ctx_(eval_ctx),
expr_spec_(expr_spec)
{
@ -1446,17 +1486,6 @@ int ObPushdownOperator::init_pushdown_storage_filter()
LOG_WARN("filter executor is null", K(ret));
}
}
if (OB_SUCC(ret) && OB_NOT_NULL(expr_spec_.pd_storage_index_back_filters_.get_pushdown_filter())) {
if (OB_FAIL(filter_exec_constructor.apply(
expr_spec_.pd_storage_index_back_filters_.get_pushdown_filter(),
pd_storage_index_back_filters_,
*this))) {
LOG_WARN("failed to create filter executor", K(ret));
} else if (OB_ISNULL(pd_storage_index_back_filters_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("filter executor is null", K(ret));
}
}
}
return ret;
}

View File

@ -530,7 +530,6 @@ public:
K_(access_exprs),
K_(max_batch_size),
K_(pushdown_filters),
K_(filters_before_index_back),
K_(pd_storage_flag));
int set_calc_exprs(const ExprFixedArray &calc_exprs, int64_t max_batch_size)
@ -545,11 +544,9 @@ public:
// filters push down to storage.
ExprFixedArray pushdown_filters_;
ExprFixedArray filters_before_index_back_;
int32_t pd_storage_flag_;
ObPushdownFilter pd_storage_filters_;
ObPushdownFilter pd_storage_index_back_filters_;
// used to pushdown aggregate expression now.
ExprFixedArray pd_storage_aggregate_output_;
};
@ -577,7 +574,6 @@ public:
int deep_copy(const sql::ObExprPtrIArray *exprs, const int64_t batch_idx);
public:
ObPushdownFilterExecutor *pd_storage_filters_;
ObPushdownFilterExecutor *pd_storage_index_back_filters_;
ObEvalCtx &eval_ctx_;
const ObPushdownExprSpec &expr_spec_;
};

View File

@ -126,11 +126,11 @@ int ObRADatumStore::StoredRow::to_expr(const common::ObIArray<ObExpr*> &exprs,
ObEvalCtx &ctx) const
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(cnt_ != exprs.count())) {
if (OB_UNLIKELY(cnt_ < exprs.count())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("datum count mismatch", K(ret), K(cnt_), K(exprs.count()));
} else {
for (uint32_t i = 0; i < cnt_; ++i) {
for (uint32_t i = 0; i < exprs.count(); ++i) {
if (exprs.at(i)->is_const_expr()) {
continue;
} else {
@ -349,7 +349,7 @@ int ObRADatumStore::Block::to_copyable()
ObRADatumStore::ObRADatumStore(common::ObIAllocator *alloc /* = NULL */)
: inited_(false), tenant_id_(0), label_(nullptr), ctx_id_(0), mem_limit_(0),
idx_blk_(NULL), save_row_cnt_(0), row_cnt_(0), fd_(-1), dir_id_(-1), file_size_(0),
inner_reader_(*this), mem_hold_(0), allocator_(NULL == alloc ? inner_allocator_ : *alloc),
inner_reader_(*this), mem_hold_(0), allocator_(NULL == alloc ? &inner_allocator_ : alloc),
row_extend_size_(0), mem_stat_(NULL), io_observer_(NULL)
{
}
@ -357,7 +357,8 @@ ObRADatumStore::ObRADatumStore(common::ObIAllocator *alloc /* = NULL */)
int ObRADatumStore::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) */)
const char *label /* = common::ObModIds::OB_SQL_ROW_STORE) */,
uint32_t row_extend_size /* = 0 */)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(inited_)) {
@ -368,6 +369,7 @@ int ObRADatumStore::init(int64_t mem_limit,
ctx_id_ = mem_ctx_id;
label_ = label;
mem_limit_ = mem_limit;
row_extend_size_ = row_extend_size;
inited_ = true;
}
return ret;
@ -420,7 +422,7 @@ void ObRADatumStore::reset()
LinkNode *node = blk_mem_list_.remove_first();
if (NULL != node) {
node->~LinkNode();
allocator_.free(node);
allocator_->free(node);
}
}
blocks_.reset();
@ -434,7 +436,7 @@ void ObRADatumStore::reuse()
int ret = OB_SUCCESS;
save_row_cnt_ = 0;
row_cnt_ = 0;
inner_reader_.reuse();
inner_reader_.reset();
if (is_file_open()) {
if (OB_FAIL(FILE_MANAGER_INSTANCE_V2.remove(fd_))) {
LOG_WARN("remove file failed", K(ret), K_(fd));
@ -450,7 +452,7 @@ void ObRADatumStore::reuse()
if (&(*node) + 1 != static_cast<LinkNode *>(static_cast<void *>(blkbuf_.buf_.data()))) {
node->unlink();
node->~LinkNode();
allocator_.free(node);
allocator_->free(node);
}
}
if (NULL != blkbuf_.buf_.data()) {
@ -492,7 +494,7 @@ void *ObRADatumStore::alloc_blk_mem(const int64_t size)
LOG_WARN("invalid argument", K(size));
} else {
ObMemAttr attr(tenant_id_, label_, ctx_id_);
void *mem = allocator_.alloc(size + sizeof(LinkNode), attr);
void *mem = allocator_->alloc(size + sizeof(LinkNode), attr);
if (OB_UNLIKELY(NULL == mem)) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("alloc memory failed", K(ret), KP(mem));
@ -502,7 +504,7 @@ void *ObRADatumStore::alloc_blk_mem(const int64_t size)
ret = OB_ERR_UNEXPECTED;
LOG_WARN("add node to list failed", K(ret));
node->~LinkNode();
allocator_.free(mem);
allocator_->free(mem);
} else {
blk = static_cast<char *>(mem) + sizeof(LinkNode);
inc_mem_hold(size + sizeof(LinkNode));
@ -520,7 +522,7 @@ void ObRADatumStore::free_blk_mem(void *mem, const int64_t size /* = 0 */)
node->unlink();
}
node->~LinkNode();
allocator_.free(node);
allocator_->free(node);
inc_mem_hold(-(size + sizeof(LinkNode)));
}
}
@ -1047,11 +1049,7 @@ void ObRADatumStore::Reader::reset()
reset_cursor(file_size);
store_.free_blk_mem(buf_.data(), buf_.capacity());
buf_.reset();
while (NULL != try_free_list_) {
auto next = try_free_list_->next_;
store_.free_blk_mem(try_free_list_, try_free_list_->size_);
try_free_list_ = next;
}
free_all_blks();
store_.free_blk_mem(idx_buf_.data(), idx_buf_.capacity());
idx_buf_.reset();
}
@ -1059,6 +1057,7 @@ void ObRADatumStore::Reader::reset()
void ObRADatumStore::Reader::reuse()
{
reset_cursor(0);
free_all_blks();
buf_.reset();
idx_buf_.reset();
}
@ -1071,6 +1070,15 @@ void ObRADatumStore::Reader::reset_cursor(const int64_t file_size)
blk_ = NULL;
}
void ObRADatumStore::Reader::free_all_blks()
{
while (NULL != try_free_list_) {
auto next = try_free_list_->next_;
store_.free_blk_mem(try_free_list_, try_free_list_->size_);
try_free_list_ = next;
}
}
int ObRADatumStore::Reader::get_row(const int64_t row_id, const StoredRow *&sr)
{
int ret = OB_SUCCESS;

View File

@ -306,6 +306,7 @@ public:
private:
void reset_cursor(const int64_t file_size);
void free_all_blks();
private:
ObRADatumStore &store_;
@ -338,7 +339,8 @@ public:
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);
const char *label = common::ObModIds::OB_SQL_ROW_STORE,
uint32_t row_extend_size = 0);
void reset();
// Keeping one memory block, reader must call reuse() too.
@ -376,6 +378,8 @@ public:
void set_mem_hold(int64_t hold);
void inc_mem_hold(int64_t hold);
void set_allocator(common::ObIAllocator &alloc) { allocator_ = &alloc; }
void set_dir_id(int64_t dir_id) { dir_id_ = dir_id; }
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));
@ -431,7 +435,7 @@ private:
int64_t mem_hold_;
common::DefaultPageAllocator inner_allocator_;
common::ObIAllocator &allocator_;
common::ObIAllocator *allocator_;
uint32_t row_extend_size_;
ObSqlMemoryCallback *mem_stat_;