[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:
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -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())) {
|
||||
|
||||
@ -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;
|
||||
}
|
||||
|
||||
@ -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_;
|
||||
};
|
||||
|
||||
@ -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;
|
||||
|
||||
@ -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_;
|
||||
|
||||
Reference in New Issue
Block a user