[scn] mv definition of SCN to dir of share

This commit is contained in:
obdev
2022-11-28 02:50:55 +00:00
committed by ob-robot
parent 68151eb87a
commit 99d4f56fec
716 changed files with 7746 additions and 6117 deletions

View File

@ -552,7 +552,8 @@ ObAggregateProcessor::ObAggregateProcessor(ObEvalCtx &eval_ctx,
dir_id_(-1),
tmp_store_row_(nullptr),
io_event_observer_(nullptr),
removal_info_()
removal_info_(),
support_fast_single_row_agg_(false)
{
}
@ -5793,5 +5794,218 @@ int ObAggregateProcessor::get_json_objectagg_result(const ObAggrInfo &aggr_info,
}
return ret;
}
int ObAggregateProcessor::single_row_agg(GroupRow &group_row, ObEvalCtx &eval_ctx)
{
int ret = OB_SUCCESS;
if (!support_fast_single_row_agg_) {
group_row.reuse();
if (OB_FAIL(prepare(group_row))) {
LOG_WARN("failed to prepare group row", K(ret));
} else if (OB_FAIL(collect_group_row(&group_row))) {
LOG_WARN("failed to collect group by row", K(ret));
}
} else if (OB_FAIL(fast_single_row_agg(eval_ctx))) {
LOG_WARN("failed to fill result", K(ret));
}
return ret;
}
int ObAggregateProcessor::single_row_agg_batch(GroupRow **group_row, ObEvalCtx &eval_ctx, const int64_t batch_size, const ObBitVector *skip)
{
int ret = OB_SUCCESS;
CK (OB_NOT_NULL(group_row) && OB_NOT_NULL(skip));
ObEvalCtx::BatchInfoScopeGuard batch_info_guard(eval_ctx);
batch_info_guard.set_batch_size(batch_size);
if (OB_FAIL(ret)) {
} else if (!support_fast_single_row_agg_) {
for (int64_t i = 0; OB_SUCC(ret) && i < batch_size; ++i) {
if (skip->at(i)) {
continue;
}
if (OB_ISNULL(group_row[i])) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("group row is not init", K(ret), K(i));
} else {
group_row[i]->reuse();
batch_info_guard.set_batch_idx(i);
if (OB_FAIL(prepare(*group_row[i]))) {
LOG_WARN("failed to prepare group row", K(ret));
} else if (OB_FAIL(collect_group_row(group_row[i]))) {
LOG_WARN("failed to collect group by row", K(ret));
}
}
}
} else if (OB_FAIL(fast_single_row_agg_batch(eval_ctx, batch_size, skip))) {
LOG_WARN("failed to fill result", K(ret));
}
return ret;
}
int ObAggregateProcessor::fast_single_row_agg(ObEvalCtx &eval_ctx)
{
int ret = OB_SUCCESS;
for (int64_t i = 0; OB_SUCC(ret) && i < aggr_infos_.count(); ++i) {
ObAggrInfo &aggr_info = aggr_infos_.at(i);
if (aggr_info.is_implicit_first_aggr()) {
continue;
}
const ObItemType aggr_fun = aggr_info.get_expr_type();
switch (aggr_fun) {
case T_FUN_COUNT: {
bool has_null = false;
ObDatum &result = aggr_info.expr_->locate_datum_for_write(eval_ctx);
for (int64_t j = 0; !has_null && j < aggr_info.param_exprs_.count(); ++j) {
has_null = aggr_info.param_exprs_.at(j)->locate_expr_datum(eval_ctx).is_null();
}
if (lib::is_mysql_mode()) {
result.set_int(has_null ? 0 : 1);
} else {
result.set_number(has_null ? ObNumber::get_zero() : ObNumber::get_positive_one());
}
break;
}
case T_FUN_SUM: {
ObDatum &result = aggr_info.expr_->locate_datum_for_write(eval_ctx);
const ObObjTypeClass tc = ob_obj_type_class(aggr_info.get_first_child_type());
if ((ObIntTC == tc || ObUIntTC == tc) && !aggr_info.param_exprs_.at(0)->locate_expr_datum(eval_ctx).is_null()) {
ObNumStackAllocator<2> tmp_alloc;
ObNumber result_nmb;
if (ObIntTC == tc) {
if (OB_FAIL(result_nmb.from(aggr_info.param_exprs_.at(0)->locate_expr_datum(eval_ctx).get_int(), tmp_alloc))) {
LOG_WARN("create number from int failed", K(ret), K(result_nmb), K(tc));
}
} else {
if (OB_FAIL(result_nmb.from(aggr_info.param_exprs_.at(0)->locate_expr_datum(eval_ctx).get_uint(), tmp_alloc))) {
LOG_WARN("create number from int failed", K(ret), K(result_nmb), K(tc));
}
}
OX (result.set_number(result_nmb));
} else {
result.set_datum(aggr_info.param_exprs_.at(0)->locate_expr_datum(eval_ctx));
}
break;
}
case T_FUN_MAX:
case T_FUN_MIN: {
ObDatum &result = aggr_info.expr_->locate_expr_datum(eval_ctx);
result.set_datum(aggr_info.param_exprs_.at(0)->locate_expr_datum(eval_ctx));
break;
}
default: {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unknown aggr function type", K(ret), K(aggr_fun), K(*aggr_info.expr_));
}
}
}
return ret;
}
int ObAggregateProcessor::fast_single_row_agg_batch(ObEvalCtx &eval_ctx, const int64_t batch_size, const ObBitVector *skip)
{
int ret = OB_SUCCESS;
CK (OB_NOT_NULL(skip));
for (int64_t i = 0; OB_SUCC(ret) && i < aggr_infos_.count(); ++i) {
ObAggrInfo &aggr_info = aggr_infos_.at(i);
if (aggr_info.is_implicit_first_aggr()) {
continue;
}
const ObItemType aggr_fun = aggr_info.get_expr_type();
switch (aggr_fun) {
case T_FUN_COUNT: {
bool has_null[batch_size];
MEMSET(has_null, false, sizeof(bool) * batch_size);
ObDatum *result = aggr_info.expr_->locate_datums_for_update(eval_ctx, batch_size);
for (int64_t j = 0; j < aggr_info.param_exprs_.count(); ++j) {
ObDatumVector param_vec = aggr_info.param_exprs_.at(j)->locate_expr_datumvector(eval_ctx);
for (int64_t batch_idx = 0; batch_idx < batch_size; ++batch_idx) {
if (skip->at(batch_idx) || has_null[batch_idx]) {
continue;
}
has_null[batch_idx] = param_vec.at(batch_idx)->is_null();
}
}
if (lib::is_mysql_mode()) {
for (int64_t batch_idx = 0; batch_idx < batch_size; ++batch_idx) {
if (skip->at(batch_idx)) {
continue;
}
result[batch_idx].set_int(has_null[batch_idx] ? 0 : 1);
}
} else {
for (int64_t batch_idx = 0; OB_SUCC(ret) && batch_idx < batch_size; ++batch_idx) {
if (skip->at(batch_idx)) {
continue;
}
result[batch_idx].set_number(has_null[batch_idx] ? ObNumber::get_zero() : ObNumber::get_positive_one());
}
}
break;
}
case T_FUN_SUM: {
const ObObjTypeClass tc = ob_obj_type_class(aggr_info.get_first_child_type());
ObDatum *result = aggr_info.expr_->locate_datums_for_update(eval_ctx, batch_size);
ObDatumVector param_vec = aggr_info.param_exprs_.at(0)->locate_expr_datumvector(eval_ctx);
if (ObIntTC == tc) {
for (int64_t batch_idx = 0; OB_SUCC(ret) && batch_idx < batch_size; ++batch_idx) {
if (skip->at(batch_idx)) {
continue;
}
ObNumStackAllocator<2> tmp_alloc;
ObNumber result_nmb;
if (param_vec.at(batch_idx)->is_null()) {
result[batch_idx].set_null();
} else if (OB_FAIL(result_nmb.from(param_vec.at(batch_idx)->get_int(), tmp_alloc))) {
LOG_WARN("create number from int failed", K(ret), K(result_nmb), K(tc));
} else {
result[batch_idx].set_number(result_nmb);
}
}
} else if (ObUIntTC == tc) {
for (int64_t batch_idx = 0; OB_SUCC(ret) && batch_idx < batch_size; ++batch_idx) {
if (skip->at(batch_idx)) {
continue;
}
ObNumStackAllocator<2> tmp_alloc;
ObNumber result_nmb;
if (param_vec.at(batch_idx)->is_null()) {
result[batch_idx].set_null();
} else if (OB_FAIL(result_nmb.from(param_vec.at(batch_idx)->get_uint64(), tmp_alloc))) {
LOG_WARN("create number from int failed", K(ret), K(result_nmb), K(tc));
} else {
result[batch_idx].set_number(result_nmb);
}
}
} else {
for (int64_t batch_idx = 0; batch_idx < batch_size; ++batch_idx) {
if (skip->at(batch_idx)) {
continue;
}
result[batch_idx].set_datum(*param_vec.at(batch_idx));
}
}
break;
}
case T_FUN_MAX:
case T_FUN_MIN: {
ObDatum *result = aggr_info.expr_->locate_batch_datums(eval_ctx);
ObDatumVector param_vec = aggr_info.param_exprs_.at(0)->locate_expr_datumvector(eval_ctx);
for (int64_t batch_idx = 0; batch_idx < batch_size; ++batch_idx) {
if (skip->at(batch_idx)) {
continue;
}
result[batch_idx].set_datum(*param_vec.at(batch_idx));
}
break;
}
default: {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unknown aggr function type", K(ret), K(aggr_fun), K(*aggr_info.expr_));
}
}
}
return ret;
}
} //namespace sql
} //namespace oceanbase

View File

@ -654,6 +654,12 @@ public:
{
io_event_observer_ = observer;
}
inline ObIArray<ObAggrInfo> &get_aggr_infos() { return aggr_infos_; }
int single_row_agg(GroupRow &group_row, ObEvalCtx &eval_ctx);
int single_row_agg_batch(GroupRow **group_rows, ObEvalCtx &eval_ctx, const int64_t batch_size, const ObBitVector *skip);
int fast_single_row_agg(ObEvalCtx &eval_ctx);
int fast_single_row_agg_batch(ObEvalCtx &eval_ctx, const int64_t batch_size, const ObBitVector *skip);
inline void set_support_fast_single_row_agg(const bool flag) { support_fast_single_row_agg_ = flag; }
private:
template <typename T>
int inner_process_batch(GroupRow &group_rows, T &selector, int64_t start_idx, int64_t end_idx);
@ -911,6 +917,7 @@ private:
ObChunkDatumStore::ShadowStoredRow *tmp_store_row_;
ObIOEventObserver *io_event_observer_;
RemovalInfo removal_info_;
bool support_fast_single_row_agg_;
};
struct ObAggregateCalcFunc

View File

@ -32,7 +32,8 @@ OB_SERIALIZE_MEMBER((ObGroupBySpec, ObOpSpec),
dist_aggr_group_idxes_,
aggr_code_idx_,
aggr_code_expr_,
by_pass_enabled_);
by_pass_enabled_,
support_fast_single_row_agg_);
DEF_TO_STRING(ObGroupBySpec)
{

View File

@ -34,7 +34,8 @@ public:
dist_aggr_group_idxes_(alloc),
aggr_code_idx_(OB_INVALID_INDEX_INT64),
aggr_code_expr_(nullptr),
by_pass_enabled_(false)
by_pass_enabled_(false),
support_fast_single_row_agg_(false)
{
}
DECLARE_VIRTUAL_TO_STRING;
@ -55,6 +56,8 @@ public:
int64_t aggr_code_idx_;
ObExpr *aggr_code_expr_;
bool by_pass_enabled_;
// COUNT/SUM/MIN/MAX can use fast single row agg
bool support_fast_single_row_agg_;
};
//modifiable

View File

@ -17,6 +17,7 @@
#include "sql/engine/px/ob_px_util.h"
#include "observer/omt/ob_tenant_config_mgr.h"
#include "lib/charset/ob_charset.h"
#include "src/sql/engine/expr/ob_expr_util.h"
namespace oceanbase
{
@ -169,7 +170,6 @@ int ObHashGroupByOp::inner_open()
int64_t est_hash_mem_size = 0;
int64_t estimate_mem_size = 0;
int64_t init_size = 0;
int64_t init_bkt_size = INIT_BKT_SIZE_FOR_ADAPTIVE_GBY;
ObMemAttr attr(ctx_.get_my_session()->get_effective_tenant_id(),
ObModIds::OB_HASH_NODE_GROUP_ROWS,
ObCtxIds::WORK_AREA);
@ -196,7 +196,7 @@ int ObHashGroupByOp::inner_open()
} else if (FALSE_IT(init_size = std::max((int64_t)MIN_GROUP_HT_INIT_SIZE, init_size))) {
} else if (FALSE_IT(init_size = std::min((int64_t)MAX_GROUP_HT_INIT_SIZE, init_size))) {
} else if (FALSE_IT(init_size = MY_SPEC.by_pass_enabled_ ?
std::min(init_size, init_bkt_size) : init_size)) {
std::min(init_size, (int64_t)INIT_BKT_SIZE_FOR_ADAPTIVE_GBY) : init_size)) {
} else if (OB_FAIL(append(dup_groupby_exprs_, MY_SPEC.group_exprs_))) {
LOG_WARN("failed to append groupby exprs", K(ret));
} else if (OB_FAIL(append(all_groupby_exprs_, dup_groupby_exprs_))) {
@ -529,11 +529,6 @@ int ObHashGroupByOp::inner_get_next_row()
} else if (bypass_ctrl_.by_passing()) {
if (OB_FAIL(load_one_row())) {
LOG_WARN("failed to load one row", K(ret));
} else if (OB_ISNULL(by_pass_group_row_) || OB_ISNULL(by_pass_group_row_->group_row_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to get group row", K(ret));
} else if (OB_FAIL(aggr_processor_.collect_group_row(by_pass_group_row_->group_row_))) {
LOG_WARN("failed to collect group by row", K(ret));
}
} else if (OB_FAIL(restore_groupby_datum())) {
LOG_WARN("failed to restore_groupby_datum", K(ret));
@ -2431,32 +2426,22 @@ int ObHashGroupByOp::init_by_pass_group_row_item()
int ObHashGroupByOp::init_by_pass_group_batch_item()
{
int ret = OB_SUCCESS;
char *group_item_buf = nullptr;
const int64_t group_id = 0;
if (by_pass_batch_size_ <= 0) {
if (OB_ISNULL(by_pass_group_batch_ =
static_cast<ObGroupRowItem **> (mem_context_->get_arena_allocator()
.alloc(sizeof(ObGroupRowItem *) * MY_SPEC.max_batch_size_)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("failed to alloc memory", K(ret));
} else if (OB_ISNULL(group_item_buf =
static_cast<char *> (mem_context_->get_arena_allocator()
.alloc(sizeof(ObGroupRowItem) * MY_SPEC.max_batch_size_)))) {
static_cast<ObAggregateProcessor::GroupRow **> (mem_context_->get_arena_allocator()
.alloc(sizeof(ObAggregateProcessor::GroupRow *) * MY_SPEC.max_batch_size_)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("failed to alloc memory", K(ret));
} else {
for (int64_t i = 0; i < MY_SPEC.max_batch_size_; ++i) {
by_pass_group_batch_[i] = new (group_item_buf) ObGroupRowItem();
group_item_buf += sizeof(ObGroupRowItem);
}
by_pass_batch_size_ = MY_SPEC.max_batch_size_;
}
}
CK (by_pass_batch_size_ > 0);
for (int64_t i = 0; OB_SUCC(ret) && i < by_pass_batch_size_; ++i) {
if (OB_FAIL(aggr_processor_.generate_group_row(by_pass_group_batch_[i]->group_row_, group_id))) {
if (OB_FAIL(aggr_processor_.generate_group_row(by_pass_group_batch_[i], group_id))) {
LOG_WARN("generate group row failed", K(ret));
} else if (OB_ISNULL(by_pass_group_batch_[i]->group_row_)) {
} else if (OB_ISNULL(by_pass_group_batch_[i])) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("generate wrong group row", K(ret), K(i));
}
@ -2506,11 +2491,10 @@ int ObHashGroupByOp::load_one_row()
ret = OB_ERR_UNEXPECTED;
LOG_WARN("by pass group row is not init", K(ret));
} else {
by_pass_group_row_->group_row_->reuse();
++agged_row_cnt_;
++agged_group_cnt_;
if (OB_FAIL(aggr_processor_.prepare(*by_pass_group_row_->group_row_))) {
LOG_WARN("failed to prepare group row", K(ret));
if (OB_FAIL(aggr_processor_.single_row_agg(*by_pass_group_row_->group_row_, eval_ctx_))) {
LOG_WARN("failed to do single row agg", K(ret));
}
}
return ret;
@ -2552,30 +2536,16 @@ int ObHashGroupByOp::by_pass_prepare_one_batch(const int64_t batch_size)
|| by_pass_batch_size_ <= 0) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("by pass group row is not init", K(ret), K(by_pass_batch_size_));
} else if (OB_FAIL(aggr_processor_.eval_aggr_param_batch(*by_pass_child_brs_))) {
LOG_WARN("fail to eval aggr param batch", K(ret), K(*by_pass_child_brs_));
} else {
ObEvalCtx::BatchInfoScopeGuard batch_info_guard(eval_ctx_);
batch_info_guard.set_batch_size(by_pass_child_brs_->size_);
for (int64_t i = 0; OB_SUCC(ret) && i < by_pass_child_brs_->size_; ++i) {
if (by_pass_child_brs_->skip_->at(i)) {
continue;
}
if (OB_ISNULL(by_pass_group_batch_[i]->group_row_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("group row is not init", K(ret), K(i));
} else {
by_pass_group_batch_[i]->group_row_->reuse();
++agged_row_cnt_;
++agged_group_cnt_;
batch_info_guard.set_batch_idx(i);
if (OB_FAIL(aggr_processor_.prepare(*by_pass_group_batch_[i]->group_row_))) {
LOG_WARN("failed to prepare group row", K(ret));
} else if (OB_ISNULL(by_pass_group_batch_[i]) || OB_ISNULL(by_pass_group_batch_[i]->group_row_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to get group row", K(ret));
} else if (OB_FAIL(aggr_processor_.collect_group_row(by_pass_group_batch_[i]->group_row_))) {
LOG_WARN("failed to collect group by row", K(ret));
}
}
if (OB_FAIL(aggr_processor_.single_row_agg_batch(by_pass_group_batch_, eval_ctx_,
by_pass_child_brs_->size_, by_pass_child_brs_->skip_))) {
LOG_WARN("failed to single row agg", K(ret));
} else {
int64_t aggr_cnt = by_pass_child_brs_->size_ - by_pass_child_brs_->skip_->accumulate_bit_cnt(by_pass_child_brs_->size_);
agged_row_cnt_ += aggr_cnt;
agged_group_cnt_ += aggr_cnt;
}
}
if (OB_SUCC(ret)) {
@ -2650,6 +2620,7 @@ int ObHashGroupByOp::init_by_pass_op()
int ret = OB_SUCCESS;
int err_sim = 0;
void *store_row_buf = nullptr;
aggr_processor_.set_support_fast_single_row_agg(MY_SPEC.support_fast_single_row_agg_);
bypass_ctrl_.open_by_pass_ctrl();
uint64_t cut_ratio = 0;
uint64_t default_cut_ratio = ObAdaptiveByPassCtrl::INIT_CUT_RATIO;

View File

@ -586,7 +586,7 @@ private:
ObSqlMemMgrProcessor distinct_sql_mem_processor_;
ObAdaptiveByPassCtrl bypass_ctrl_;
ObGroupRowItem *by_pass_group_row_;
ObGroupRowItem **by_pass_group_batch_;
ObAggregateProcessor::GroupRow **by_pass_group_batch_;
int64_t by_pass_batch_size_;
int64_t by_pass_nth_group_;
ObChunkDatumStore::LastStoredRow *last_child_row_;