[FEAT MERGE] impl vectorization 2.0
Co-authored-by: Naynahs <cfzy002@126.com> Co-authored-by: hwx65 <1780011298@qq.com> Co-authored-by: oceanoverflow <oceanoverflow@gmail.com>
This commit is contained in:
@ -137,6 +137,16 @@ int ObCodeGenerator::detect_batch_size(
|
||||
scan_cardinality));
|
||||
// overwrite batch size if hint is specified
|
||||
OZ(opt_params->get_integer_opt_param(ObOptParamHint::ROWSETS_MAX_ROWS, batch_size));
|
||||
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
tmp_ret = OB_E(EventTable::EN_ENABLE_RANDOM_BATCH_SIZE) OB_SUCCESS;
|
||||
if (OB_SUCCESS != tmp_ret) {
|
||||
static const int64_t min = 1;
|
||||
static const int64_t max = rowsets_max_rows * 2;
|
||||
batch_size = common::ObRandom::rand(min, max);
|
||||
}
|
||||
LOG_TRACE("detect_batch_size", K(vectorize), K(scan_cardinality), K(batch_size),
|
||||
K(rowsets_max_rows), K(tmp_ret));
|
||||
}
|
||||
// TODO qubin.qb: remove the tracelog when rowsets/batch_size is displayed
|
||||
// in plan
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@ -59,6 +59,7 @@ class ObLogUnpivot;
|
||||
class ObUnpivotSpec;
|
||||
class ObFakeCTETableSpec;
|
||||
class ObHashJoinSpec;
|
||||
class ObHashJoinVecSpec;
|
||||
class ObNestedLoopJoinSpec;
|
||||
class ObBasicNestedLoopJoinSpec;
|
||||
class ObMergeJoinSpec;
|
||||
@ -75,6 +76,7 @@ class ObPxReceiveSpec;
|
||||
class ObPxTransmitSpec;
|
||||
class ObPxFifoReceiveSpec;
|
||||
class ObPxMSReceiveSpec;
|
||||
class ObPxMSReceiveVecSpec;
|
||||
class ObPxDistTransmitSpec;
|
||||
class ObPxDistTransmitOp;
|
||||
class ObPxRepartTransmitSpec;
|
||||
@ -83,6 +85,7 @@ class ObPxCoordSpec;
|
||||
class ObPxFifoCoordSpec;
|
||||
class ObPxOrderedCoordSpec;
|
||||
class ObPxMSCoordSpec;
|
||||
class ObPxMSCoordVecSpec;
|
||||
class ObLogSubPlanFilter;
|
||||
class ObSubPlanFilterSpec;
|
||||
class ObLogSubPlanScan;
|
||||
@ -108,8 +111,11 @@ class ObPxMultiPartDeleteSpec;
|
||||
class ObPxMultiPartInsertSpec;
|
||||
class ObPxMultiPartUpdateSpec;
|
||||
class ObTempTableAccessOpSpec;
|
||||
class ObTempTableAccessVecOpSpec;
|
||||
class ObTempTableInsertOpSpec;
|
||||
class ObTempTableInsertVecOpSpec;
|
||||
class ObTempTableTransformationOpSpec;
|
||||
class ObTempTableTransformationVecOpSpec;
|
||||
class ObErrLogSpec;
|
||||
class ObSelectIntoSpec;
|
||||
class ObFunctionTableSpec;
|
||||
@ -123,6 +129,8 @@ class ObDuplicatedKeyChecker;
|
||||
struct ObTableScanCtDef;
|
||||
struct ObDASScanCtDef;
|
||||
struct InsertAllTableInfo;
|
||||
class ObHashDistinctVecSpec;
|
||||
class ObSortVecSpec;
|
||||
typedef common::ObList<uint64_t, common::ObIAllocator> DASTableIdList;
|
||||
typedef common::ObSEArray<common::ObSEArray<int64_t, 8, common::ModulePageAllocator, true>,
|
||||
1, common::ModulePageAllocator, true> RowParamMap;
|
||||
@ -188,7 +196,7 @@ public:
|
||||
|
||||
// detect physical operator type from logic operator.
|
||||
static int get_phy_op_type(ObLogicalOperator &op, ObPhyOperatorType &type,
|
||||
const bool in_root_job);
|
||||
const bool in_root_job, const bool use_rich_format = false);
|
||||
//set is json constraint type is strict or relax
|
||||
const static uint8_t IS_JSON_CONSTRAINT_RELAX = 1;
|
||||
const static uint8_t IS_JSON_CONSTRAINT_STRICT = 4;
|
||||
@ -256,9 +264,11 @@ private:
|
||||
//////////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
int generate_spec(ObLogLimit &op, ObLimitSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogLimit &op, ObLimitVecSpec &spec, const bool in_root_job);
|
||||
|
||||
int generate_spec(ObLogDistinct &op, ObMergeDistinctSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogDistinct &op, ObHashDistinctSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogDistinct &op, ObHashDistinctVecSpec &spec, const bool in_root_job);
|
||||
|
||||
int generate_spec(ObLogSet &op, ObHashUnionSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogSet &op, ObHashIntersectSpec &spec, const bool in_root_job);
|
||||
@ -275,7 +285,10 @@ private:
|
||||
|
||||
int generate_spec(ObLogMaterial &op, ObMaterialSpec &spec, const bool in_root_job);
|
||||
|
||||
int generate_spec(ObLogMaterial &op, ObMaterialVecSpec &spec, const bool in_root_job);
|
||||
|
||||
int generate_spec(ObLogSort &op, ObSortSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogSort &op, ObSortVecSpec &spec, const bool in_root_job);
|
||||
|
||||
int generate_spec(ObLogCount &op, ObCountSpec &spec, const bool in_root_job);
|
||||
|
||||
@ -299,8 +312,10 @@ private:
|
||||
int set_3stage_info(ObLogGroupBy &op, ObGroupBySpec &spec);
|
||||
int set_rollup_adaptive_info(ObLogGroupBy &op, ObMergeGroupBySpec &spec);
|
||||
int generate_spec(ObLogGroupBy &op, ObScalarAggregateSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogGroupBy &op, ObScalarAggregateVecSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogGroupBy &op, ObMergeGroupBySpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogGroupBy &op, ObHashGroupBySpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogGroupBy &op, ObHashGroupByVecSpec &spec, const bool in_root_job);
|
||||
int generate_dist_aggr_distinct_columns(ObLogGroupBy &op, ObHashGroupBySpec &spec);
|
||||
int generate_dist_aggr_group(ObLogGroupBy &op, ObGroupBySpec &spec);
|
||||
|
||||
@ -324,6 +339,8 @@ private:
|
||||
int generate_cte_table_spec(ObLogTableScan &op, ObFakeCTETableSpec &spec);
|
||||
|
||||
int generate_spec(ObLogJoin &op, ObHashJoinSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogJoin &op, ObHashJoinVecSpec &spec, const bool in_root_job);
|
||||
|
||||
// generate nested loop join
|
||||
int generate_spec(ObLogJoin &op, ObNestedLoopJoinSpec &spec, const bool in_root_job);
|
||||
// generate merge join
|
||||
@ -372,12 +389,14 @@ private:
|
||||
ObIArray<int64_t> &dml_tsc_op_ids, ObIArray<int64_t> &dml_tsc_ref_ids);
|
||||
int generate_spec(ObLogExchange &op, ObPxFifoReceiveSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogExchange &op, ObPxMSReceiveSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogExchange &op, ObPxMSReceiveVecSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogExchange &op, ObPxDistTransmitSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogExchange &op, ObPxRepartTransmitSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogExchange &op, ObPxReduceTransmitSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogExchange &op, ObPxFifoCoordSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogExchange &op, ObPxOrderedCoordSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogExchange &op, ObPxMSCoordSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogExchange &op, ObPxMSCoordVecSpec &spec, const bool in_root_job);
|
||||
int check_rollup_distributor(ObPxTransmitSpec *spec);
|
||||
|
||||
// for remote execute
|
||||
@ -405,7 +424,12 @@ private:
|
||||
|
||||
// online optimizer stats gathering
|
||||
int generate_spec(ObLogOptimizerStatsGathering &op, ObOptimizerStatsGatheringSpec &spec, const bool in_root_job);
|
||||
|
||||
int generate_spec(ObLogTempTableInsert &op, ObTempTableInsertVecOpSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogTempTableAccess &op, ObTempTableAccessVecOpSpec &spec, const bool in_root_job);
|
||||
int generate_spec(ObLogTempTableTransformation &op, ObTempTableTransformationVecOpSpec &spec, const bool in_root_job);
|
||||
private:
|
||||
int disable_use_rich_format(const ObLogicalOperator &op, ObOpSpec &spec);
|
||||
int add_update_set(ObSubPlanFilterSpec &spec);
|
||||
int generate_basic_transmit_spec(
|
||||
ObLogExchange &op, ObPxTransmitSpec &spec, const bool in_root_job);
|
||||
@ -424,6 +448,9 @@ private:
|
||||
const ObSortCollations &collations,
|
||||
ObSortFuncs &sort_funcs,
|
||||
const ObIArray<ObExpr*> &sort_exprs);
|
||||
int check_not_support_cmp_type(
|
||||
const ObSortCollations &collations,
|
||||
const ObIArray<ObExpr*> &sort_exprs);
|
||||
int recursive_get_column_expr(const ObColumnRefRawExpr *&column, const TableItem &table_item);
|
||||
int fill_aggr_infos(ObLogGroupBy &op,
|
||||
ObGroupBySpec &spec,
|
||||
@ -494,20 +521,17 @@ private:
|
||||
|
||||
int generate_insert_all_table_info(const ObInsertAllTableInfo &insert_tbl_info,
|
||||
InsertAllTableInfo *&tbl_info);
|
||||
static int find_rownum_expr(bool &support, ObLogicalOperator *op);
|
||||
static int find_rownum_expr_recursively(bool &support,
|
||||
const ObRawExpr *raw_expr);
|
||||
inline static int find_rownum_expr(bool &support,
|
||||
const common::ObIArray<ObRawExpr *> &exprs);
|
||||
int map_value_param_index(const ObInsertStmt *insert_stmt, RowParamMap &row_params_map);
|
||||
int add_output_datum_check_flag(ObOpSpec &spec);
|
||||
int generate_calc_part_id_expr(const ObRawExpr &src, const ObDASTableLocMeta *loc_meta, ObExpr *&dst);
|
||||
int check_only_one_unique_key(const ObLogPlan &log_plan, const ObTableSchema* table_schema, bool& only_one_unique_key);
|
||||
|
||||
bool is_simple_aggr_expr(const ObItemType &expr_type) { return T_FUN_COUNT == expr_type
|
||||
|| T_FUN_SUM == expr_type
|
||||
|| T_FUN_MAX == expr_type
|
||||
|| T_FUN_MIN == expr_type; }
|
||||
bool is_simple_aggr_expr(const ObItemType &expr_type,
|
||||
const bool enable_rich_format) { return T_FUN_COUNT == expr_type
|
||||
|| T_FUN_SUM == expr_type
|
||||
|| T_FUN_MAX == expr_type
|
||||
|| T_FUN_MIN == expr_type
|
||||
|| (enable_rich_format && T_FUN_COUNT_SUM == expr_type); }
|
||||
int check_fk_nested_dup_del(const uint64_t table_id,
|
||||
const uint64_t root_table_id,
|
||||
DASTableIdList &parent_tables,
|
||||
@ -520,6 +544,18 @@ private:
|
||||
const ObFixedArray<ObDynamicParamSetter, ObIAllocator>& setters);
|
||||
|
||||
int check_window_functions_order(const ObIArray<ObWinFunRawExpr *> &winfunc_exprs);
|
||||
|
||||
int prepare_runtime_filter_cmp_info(ObLogJoinFilter &join_filter_create, ObJoinFilterSpec &spec);
|
||||
int append_child_output_no_dup(const bool is_store_sortkey_separately,
|
||||
const ObIArray<ObExpr *> &child_output_exprs,
|
||||
ObIArray<ObExpr *> &sk_exprs, ObIArray<ObExpr *> &addon_exprs);
|
||||
int generate_encode_sort_exprs(const bool is_store_sortkey_separately, ObLogSort &op,
|
||||
ObSortVecSpec &spec, ObIArray<OrderItem> &sk_keys,
|
||||
ObIArray<OrderItem> &addon_keys);
|
||||
|
||||
int generate_sort_exprs(const bool is_store_sortkey_separately, ObLogSort &op, ObSortVecSpec &spec,
|
||||
ObIArray<OrderItem> &sk_keys);
|
||||
|
||||
private:
|
||||
struct BatchExecParamCache {
|
||||
BatchExecParamCache(ObExecParamRawExpr* expr, ObOpSpec* spec, bool is_left)
|
||||
|
||||
@ -19,6 +19,7 @@
|
||||
#include "sql/engine/expr/ob_expr_util.h"
|
||||
#include "sql/engine/expr/ob_expr_extra_info_factory.h"
|
||||
#include "sql/engine/expr/ob_expr_lob_utils.h"
|
||||
#include "share/vector/ob_vector_define.h"
|
||||
#include "sql/engine/expr/ob_datum_cast.h"
|
||||
|
||||
namespace oceanbase
|
||||
@ -263,6 +264,8 @@ int ObStaticEngineExprCG::cg_expr_basic(const ObIArray<ObRawExpr *> &raw_exprs)
|
||||
rt_expr->is_static_const_ = raw_expr->is_static_const_expr();
|
||||
rt_expr->is_dynamic_const_ = raw_expr->is_dynamic_const_expr();
|
||||
rt_expr->is_boolean_ = raw_expr->is_bool_expr();
|
||||
rt_expr->nullable_ = !(raw_expr->is_column_ref_expr()
|
||||
&& (static_cast<ObColumnRefRawExpr *> (raw_expr))->get_result_type().has_result_flag(NOT_NULL_FLAG));
|
||||
if (T_OP_ROW != raw_expr->get_expr_type()) {
|
||||
// init datum_meta_
|
||||
rt_expr->datum_meta_ = ObDatumMeta(result_meta.get_type(),
|
||||
@ -290,6 +293,10 @@ int ObStaticEngineExprCG::cg_expr_basic(const ObIArray<ObRawExpr *> &raw_exprs)
|
||||
rt_expr->max_length_ = raw_expr->get_result_type().get_length();
|
||||
// init obj_datum_map_
|
||||
rt_expr->obj_datum_map_ = ObDatum::get_obj_datum_map_type(result_meta.get_type());
|
||||
rt_expr->vec_value_tc_ = get_vec_value_tc(rt_expr->datum_meta_.type_,
|
||||
rt_expr->datum_meta_.scale_,
|
||||
rt_expr->datum_meta_.precision_);
|
||||
rt_expr->is_fixed_length_data_ = common::is_fixed_length_vec(rt_expr->vec_value_tc_);
|
||||
if (ob_is_decimal_int(rt_expr->datum_meta_.type_)) {
|
||||
const int16_t precision = rt_expr->datum_meta_.precision_;
|
||||
const int16_t scale = rt_expr->datum_meta_.scale_;
|
||||
@ -511,7 +518,9 @@ int ObStaticEngineExprCG::cg_expr_by_operator(const ObIArray<ObRawExpr *> &raw_e
|
||||
} else if (OB_INVALID_INDEX == ObFuncSerialization::get_serialize_index(
|
||||
reinterpret_cast<void *>(rt_expr->eval_func_))
|
||||
|| OB_INVALID_INDEX == ObFuncSerialization::get_serialize_index(
|
||||
reinterpret_cast<void *>(rt_expr->eval_batch_func_))) {
|
||||
reinterpret_cast<void *>(rt_expr->eval_batch_func_))
|
||||
/*|| OB_INVALID_INDEX == ObFuncSerialization::get_serialize_index(
|
||||
reinterpret_cast<void *>(rt_expr->eval_vector_func_))*/) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("evaluate function or evaluate batch function not serializable, "
|
||||
"may be you should add the function into ob_expr_eval_functions",
|
||||
@ -527,7 +536,7 @@ int ObStaticEngineExprCG::cg_expr_by_operator(const ObIArray<ObRawExpr *> &raw_e
|
||||
rt_expr->inner_functions_[j]);
|
||||
if (0 == idx || OB_INVALID_INDEX == idx) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("inner function not serializable", K(ret), K(idx),
|
||||
LOG_WARN("inner function not serializable", K(ret), K(idx), K(j),
|
||||
KP(rt_expr->inner_functions_[j]), K(*raw_expr), K(*rt_expr));
|
||||
}
|
||||
}
|
||||
@ -537,9 +546,13 @@ int ObStaticEngineExprCG::cg_expr_by_operator(const ObIArray<ObRawExpr *> &raw_e
|
||||
|
||||
// set default eval batch func
|
||||
if (OB_SUCC(ret) && batch_size_ > 0
|
||||
&& NULL != rt_expr->eval_func_ && NULL == rt_expr->eval_batch_func_
|
||||
&& rt_expr->is_batch_result()) {
|
||||
rt_expr->eval_batch_func_ = &expr_default_eval_batch_func;
|
||||
&& NULL != rt_expr->eval_func_) {
|
||||
if (NULL == rt_expr->eval_batch_func_) {
|
||||
rt_expr->eval_batch_func_ = &expr_default_eval_batch_func;
|
||||
}
|
||||
if (NULL == rt_expr->eval_vector_func_) {
|
||||
rt_expr->eval_vector_func_ = &expr_default_eval_vector_func;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -578,7 +591,7 @@ int ObStaticEngineExprCG::cg_all_frame_layout(const ObIArray<ObRawExpr *> &raw_e
|
||||
} else if (OB_FAIL(cg_datum_frame_layouts(no_const_param_exprs,
|
||||
frame_idx_pos,
|
||||
expr_info.datum_frame_))) {
|
||||
LOG_WARN("fail to init const", K(ret), K(no_const_param_exprs));
|
||||
LOG_WARN("fail to init datum frame", K(ret), K(no_const_param_exprs));
|
||||
} else if (OB_FAIL(alloc_const_frame(const_exprs,
|
||||
expr_info.const_frame_,
|
||||
expr_info.const_frame_ptrs_))) {
|
||||
@ -643,7 +656,14 @@ void ObStaticEngineExprCG::get_param_frame_idx(const int64_t idx,
|
||||
int64_t &frame_idx,
|
||||
int64_t &datum_idx)
|
||||
{
|
||||
const int64_t cnt_per_frame = ObExprFrameInfo::EXPR_CNT_PER_FRAME;
|
||||
int64_t item_size = 0;
|
||||
if (use_rich_format()) {
|
||||
item_size = sizeof(ObDatum) + sizeof(ObEvalInfo) + sizeof(VectorHeader);
|
||||
} else {
|
||||
item_size = sizeof(ObDatum) + sizeof(ObEvalInfo);
|
||||
}
|
||||
int64_t cnt_per_frame = common::MAX_FRAME_SIZE / item_size;
|
||||
|
||||
if (idx < original_param_cnt_) {
|
||||
frame_idx = idx / cnt_per_frame;
|
||||
datum_idx = idx % cnt_per_frame;
|
||||
@ -672,6 +692,12 @@ int ObStaticEngineExprCG::cg_param_frame_layout(const ObIArray<ObRawExpr *> &par
|
||||
}
|
||||
int64_t frame_idx = 0;
|
||||
int64_t datum_idx = 0;
|
||||
int64_t item_size = 0;
|
||||
if (use_rich_format()) {
|
||||
item_size = sizeof(ObDatum) + sizeof(ObEvalInfo) + sizeof(VectorHeader);
|
||||
} else {
|
||||
item_size = sizeof(ObDatum) + sizeof(ObEvalInfo);
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < param_exprs.count(); i++) {
|
||||
ObExpr *rt_expr = get_rt_expr(*param_exprs.at(i));
|
||||
if (T_QUESTIONMARK != rt_expr->type_) {
|
||||
@ -689,8 +715,15 @@ int ObStaticEngineExprCG::cg_param_frame_layout(const ObIArray<ObRawExpr *> &par
|
||||
} else {
|
||||
get_param_frame_idx(param_idx, frame_idx, datum_idx);
|
||||
rt_expr->frame_idx_ = frame_index_pos + frame_idx;
|
||||
rt_expr->datum_off_ = datum_idx * DATUM_EVAL_INFO_SIZE;
|
||||
rt_expr->datum_off_ = datum_idx * item_size;
|
||||
rt_expr->eval_info_off_ = rt_expr->datum_off_ + sizeof(ObDatum);
|
||||
if (use_rich_format()) {
|
||||
rt_expr->vector_header_off_ = rt_expr->eval_info_off_ + sizeof(ObEvalInfo);
|
||||
if (rt_expr->is_fixed_length_data_) {
|
||||
rt_expr->len_ = ObDatum::get_reserved_size(rt_expr->obj_datum_map_,
|
||||
rt_expr->datum_meta_.precision_);
|
||||
}
|
||||
}
|
||||
rt_expr->res_buf_off_ = 0;
|
||||
rt_expr->res_buf_len_ = 0;
|
||||
// 对于T_QUESTIONMARK的param表达式, 使用extra_记录其实际value在
|
||||
@ -723,9 +756,10 @@ int ObStaticEngineExprCG::cg_param_frame_layout(const ObIArray<ObRawExpr *> &par
|
||||
if (OB_SUCC(ret)) {
|
||||
ObFrameInfo &frame_info = frame_info_arr.at(i);
|
||||
frame_info.expr_cnt_ = datum_idx + 1;
|
||||
frame_info.frame_size_ = (datum_idx + 1) * DATUM_EVAL_INFO_SIZE;
|
||||
frame_info.frame_size_ = (datum_idx + 1) * item_size;
|
||||
frame_info.frame_idx_ = frame_index_pos + i;
|
||||
total -= frame_info.expr_cnt_;
|
||||
frame_info.use_rich_format_ = use_rich_format();
|
||||
}
|
||||
}
|
||||
CK(0 == total);
|
||||
@ -840,18 +874,28 @@ int ObStaticEngineExprCG::cg_frame_layout(const ObIArray<ObRawExpr *> &exprs,
|
||||
} else {
|
||||
rt_expr->res_buf_len_ = def_res_len;
|
||||
}
|
||||
if (use_rich_format() && rt_expr->is_fixed_length_data_) {
|
||||
rt_expr->len_ = rt_expr->res_buf_len_;
|
||||
}
|
||||
} else {
|
||||
rt_expr->res_buf_len_ = 0;
|
||||
}
|
||||
} else {
|
||||
rt_expr->res_buf_len_ = def_res_len;
|
||||
}
|
||||
if (use_rich_format() && rt_expr->is_fixed_length_data_) {
|
||||
rt_expr->len_ = rt_expr->res_buf_len_;
|
||||
}
|
||||
}
|
||||
for (int64_t expr_idx = 0;
|
||||
OB_SUCC(ret) && expr_idx < exprs.count();
|
||||
expr_idx++) {
|
||||
ObExpr *rt_expr = get_rt_expr(*exprs.at(expr_idx));
|
||||
const int64_t datum_size = DATUM_EVAL_INFO_SIZE + reserve_data_consume(*rt_expr);
|
||||
int64_t datum_size = DATUM_EVAL_INFO_SIZE
|
||||
+ reserve_data_consume(*rt_expr);
|
||||
if (use_rich_format()) {
|
||||
datum_size += sizeof(VectorHeader);
|
||||
}
|
||||
if (frame_size + datum_size <= MAX_FRAME_SIZE) {
|
||||
frame_size += datum_size;
|
||||
frame_expr_cnt++;
|
||||
@ -861,7 +905,8 @@ int ObStaticEngineExprCG::cg_frame_layout(const ObIArray<ObRawExpr *> &exprs,
|
||||
frame_index_pos + frame_idx,
|
||||
frame_size,
|
||||
0, /*zero_init_pos*/
|
||||
frame_size/*zero_init_size*/)))) {
|
||||
frame_size/*zero_init_size*/,
|
||||
use_rich_format())))) {
|
||||
LOG_WARN("fail to push frame_size", K(ret));
|
||||
} else {
|
||||
++frame_idx;
|
||||
@ -878,7 +923,8 @@ int ObStaticEngineExprCG::cg_frame_layout(const ObIArray<ObRawExpr *> &exprs,
|
||||
frame_index_pos + frame_idx,
|
||||
frame_size,
|
||||
0, /*zero_init_pos*/
|
||||
frame_size/*zero_init_size*/)))) {
|
||||
frame_size/*zero_init_size*/,
|
||||
use_rich_format())))) {
|
||||
LOG_WARN("fail to push frame_size", K(ret), K(frame_size));
|
||||
}
|
||||
}
|
||||
@ -915,12 +961,21 @@ int ObStaticEngineExprCG::arrange_datum_data(ObIArray<ObRawExpr *> &exprs,
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (continuous_datum) {
|
||||
int64_t data_off = frame.expr_cnt_ * DATUM_EVAL_INFO_SIZE;
|
||||
int64_t item_size = 0;
|
||||
if (use_rich_format()) {
|
||||
item_size = sizeof(ObDatum) + sizeof(ObEvalInfo) + sizeof(VectorHeader);
|
||||
} else {
|
||||
item_size = sizeof(ObDatum) + sizeof(ObEvalInfo);
|
||||
}
|
||||
int64_t data_off = frame.expr_cnt_ * item_size;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
e->frame_idx_ = frame.frame_idx_;
|
||||
e->datum_off_ = i * DATUM_EVAL_INFO_SIZE;
|
||||
e->datum_off_ = i * item_size;
|
||||
e->eval_info_off_ = e->datum_off_ + sizeof(ObDatum);
|
||||
if (use_rich_format()) {
|
||||
e->vector_header_off_ = e->eval_info_off_ + sizeof(ObEvalInfo);
|
||||
}
|
||||
const int64_t consume_size = reserve_data_consume(*e);
|
||||
if (consume_size > 0) {
|
||||
data_off += consume_size;
|
||||
@ -932,47 +987,49 @@ int ObStaticEngineExprCG::arrange_datum_data(ObIArray<ObRawExpr *> &exprs,
|
||||
}
|
||||
CK(data_off == frame.frame_size_);
|
||||
} else {
|
||||
// FIXME bin.lb: ALIGN_SIZE may affect the performance, set to 1 if no affect
|
||||
// make sure all ObDatum is aligned with %ALIGN_SIZE
|
||||
const static int64_t ALIGN_SIZE = 8;
|
||||
// offset for data only area
|
||||
int64_t data_off = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
data_off += DATUM_EVAL_INFO_SIZE;
|
||||
if (!ob_is_string_type(e->datum_meta_.type_) && 0 == (e->res_buf_len_ % ALIGN_SIZE)) {
|
||||
// data follow ObDatum
|
||||
data_off += reserve_data_consume(*e);
|
||||
}
|
||||
}
|
||||
// offset for datum + data area
|
||||
int64_t datum_off = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
e->frame_idx_ = frame.frame_idx_;
|
||||
e->datum_off_ = datum_off;
|
||||
e->eval_info_off_ = e->datum_off_ + sizeof(ObDatum);
|
||||
datum_off += DATUM_EVAL_INFO_SIZE;
|
||||
const int64_t consume_size = reserve_data_consume(*e);
|
||||
if (!ob_is_string_type(e->datum_meta_.type_) && 0 == (e->res_buf_len_ % ALIGN_SIZE)) {
|
||||
if (consume_size > 0) {
|
||||
datum_off += consume_size;
|
||||
e->res_buf_off_ = datum_off - e->res_buf_len_;
|
||||
e->dyn_buf_header_offset_ = e->res_buf_off_ - sizeof(ObDynReserveBuf);
|
||||
} else {
|
||||
e->res_buf_off_ = 0;
|
||||
}
|
||||
} else {
|
||||
if (consume_size > 0) {
|
||||
data_off += consume_size;
|
||||
e->res_buf_off_ = data_off - e->res_buf_len_;
|
||||
e->dyn_buf_header_offset_ = e->res_buf_off_ - sizeof(ObDynReserveBuf);
|
||||
} else {
|
||||
e->res_buf_off_ = 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
CK(data_off == frame.frame_size_);
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("not support non-continuous datum", K(ret));
|
||||
// // FIXME bin.lb: ALIGN_SIZE may affect the performance, set to 1 if no affect
|
||||
// // make sure all ObDatum is aligned with %ALIGN_SIZE
|
||||
// const static int64_t ALIGN_SIZE = 8;
|
||||
// // offset for data only area
|
||||
// int64_t data_off = 0;
|
||||
// for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
// ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
// data_off += DATUM_EVAL_INFO_SIZE;
|
||||
// if (!ob_is_string_type(e->datum_meta_.type_) && 0 == (e->res_buf_len_ % ALIGN_SIZE)) {
|
||||
// // data follow ObDatum
|
||||
// data_off += reserve_data_consume(*e);
|
||||
// }
|
||||
// }
|
||||
// // offset for datum + data area
|
||||
// int64_t datum_off = 0;
|
||||
// for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
// ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
// e->frame_idx_ = frame.frame_idx_;
|
||||
// e->datum_off_ = datum_off;
|
||||
// e->eval_info_off_ = e->datum_off_ + sizeof(ObDatum);
|
||||
// datum_off += DATUM_EVAL_INFO_SIZE;
|
||||
// const int64_t consume_size = reserve_data_consume(*e);
|
||||
// if (!ob_is_string_type(e->datum_meta_.type_) && 0 == (e->res_buf_len_ % ALIGN_SIZE)) {
|
||||
// if (consume_size > 0) {
|
||||
// datum_off += consume_size;
|
||||
// e->res_buf_off_ = datum_off - e->res_buf_len_;
|
||||
// e->dyn_buf_header_offset_ = e->res_buf_off_ - sizeof(ObDynReserveBuf);
|
||||
// } else {
|
||||
// e->res_buf_off_ = 0;
|
||||
// }
|
||||
// } else {
|
||||
// if (consume_size > 0) {
|
||||
// data_off += consume_size;
|
||||
// e->res_buf_off_ = data_off - e->res_buf_len_;
|
||||
// e->dyn_buf_header_offset_ = e->res_buf_off_ - sizeof(ObDynReserveBuf);
|
||||
// } else {
|
||||
// e->res_buf_off_ = 0;
|
||||
// }
|
||||
// }
|
||||
// }
|
||||
// CK(data_off == frame.frame_size_);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -997,24 +1054,34 @@ int ObStaticEngineExprCG::arrange_datums_data(ObIArray<ObRawExpr *> &exprs,
|
||||
// | PVT Skip in Expr2 |
|
||||
// +--------------------------------+
|
||||
// | ...... |
|
||||
// |--------------------------------+
|
||||
// | EvalInfo in Expr1 |
|
||||
// +--------------------------------+
|
||||
// | EvalInfo in Expr2 |
|
||||
// | Lens |
|
||||
// +--------------------------------+
|
||||
// | ...... |
|
||||
// | Ptrs/Offsets |
|
||||
// +--------------------------------+
|
||||
// | VectorHeader |-----|
|
||||
// +--------------------------------+ |
|
||||
// | Null Bitmap | |
|
||||
// +--------------------------------+ |
|
||||
// | vector dynamic buf header | |
|
||||
// |--------------------------------+ |
|
||||
// | EvalInfo in Expr1 | |
|
||||
// +--------------------------------+ |
|
||||
// | EvalInfo in Expr2 | |
|
||||
// +--------------------------------+ |
|
||||
// | ...... | |
|
||||
// +--------------------------------+ |
|
||||
// | EvalFlag in Expr1 |
|
||||
// +--------------------------------+
|
||||
// | EvalFlag in Expr2 |
|
||||
// +--------------------------------+
|
||||
// | ...... |
|
||||
// +--------------------------------+
|
||||
// | Dynamic buf header in expr1 |
|
||||
// +--------------------------------+
|
||||
// | Dynamic buf header in expr2 |
|
||||
// |--------------------------------+
|
||||
// | ...... |
|
||||
// +--------------------------------+ need memset
|
||||
// | EvalFlag in Expr2 | |
|
||||
// +--------------------------------+ |
|
||||
// | ...... | |
|
||||
// +--------------------------------+ |
|
||||
// | Dynamic buf header in expr1 | |
|
||||
// +--------------------------------+ |
|
||||
// | Dynamic buf header in expr2 | |
|
||||
// |--------------------------------+ |
|
||||
// | ...... |-----|
|
||||
// +--------------------------------+
|
||||
// | Reserved datum data in expr1 |
|
||||
// +--------------------------------+
|
||||
@ -1022,38 +1089,94 @@ int ObStaticEngineExprCG::arrange_datums_data(ObIArray<ObRawExpr *> &exprs,
|
||||
// |--------------------------------+
|
||||
// | ...... |
|
||||
// +--------------------------------+
|
||||
int64_t total_header_len = 0;
|
||||
int64_t cur_total_size = 0;
|
||||
//datums , private skip bitmap, vector_header
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
e->datum_off_ = total_header_len;
|
||||
total_header_len +=
|
||||
get_expr_skip_vector_size(*e) /* skip bitmap + evalflag bitmap */ +
|
||||
get_datums_header_size(*e);
|
||||
e->datum_off_ = cur_total_size;
|
||||
cur_total_size += get_expr_skip_vector_size(*e) /* pvt skip bitmap */
|
||||
+ get_datums_header_size(*e) /* datums */;
|
||||
}
|
||||
|
||||
LOG_TRACE("arrange datums data", K(cur_total_size), K(use_rich_format()));
|
||||
if (use_rich_format()) {
|
||||
// lens/offsets
|
||||
uint32_t len_arr_total = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
if (!e->is_fixed_length_data_) {
|
||||
e->len_arr_off_ = cur_total_size + len_arr_total;
|
||||
len_arr_total += get_offsets_size(*e);
|
||||
} else {
|
||||
e->len_ = e->res_buf_len_;
|
||||
}
|
||||
}
|
||||
cur_total_size += len_arr_total;
|
||||
|
||||
// ptrs
|
||||
uint32_t ptr_arr_total = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
if (!e->is_fixed_length_data_) {
|
||||
e->offset_off_ = cur_total_size + ptr_arr_total;
|
||||
ptr_arr_total += get_ptrs_size(*e);
|
||||
}
|
||||
}
|
||||
cur_total_size += ptr_arr_total;
|
||||
|
||||
// vector header
|
||||
uint32_t vector_header_total = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
e->vector_header_off_ = cur_total_size + vector_header_total;
|
||||
vector_header_total += get_vector_header_size();
|
||||
}
|
||||
cur_total_size += vector_header_total;
|
||||
|
||||
// nulls
|
||||
uint32_t null_bitmap_total = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
e->null_bitmap_off_ = cur_total_size + null_bitmap_total;
|
||||
null_bitmap_total += get_expr_bitmap_vector_size(*e);
|
||||
}
|
||||
cur_total_size += null_bitmap_total;
|
||||
|
||||
// cont_buf
|
||||
uint32_t cont_buf_total = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
if (!e->is_fixed_length_data_) {
|
||||
e->cont_buf_off_ = cur_total_size + cont_buf_total;
|
||||
cont_buf_total += sizeof(ObDynReserveBuf);
|
||||
}
|
||||
}
|
||||
cur_total_size += cont_buf_total;
|
||||
}
|
||||
|
||||
uint32_t eval_info_total = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
e->eval_info_off_ = total_header_len + eval_info_total;
|
||||
e->eval_info_off_ = cur_total_size + eval_info_total;
|
||||
eval_info_total += sizeof(ObEvalInfo);
|
||||
}
|
||||
total_header_len += eval_info_total;
|
||||
cur_total_size += eval_info_total;
|
||||
|
||||
uint32_t eval_flags_total = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
e->eval_flags_off_ = total_header_len + eval_flags_total;
|
||||
eval_flags_total += get_expr_skip_vector_size(*e);
|
||||
e->eval_flags_off_ = cur_total_size + eval_flags_total;
|
||||
eval_flags_total += get_expr_bitmap_vector_size(*e);
|
||||
}
|
||||
total_header_len += eval_flags_total;
|
||||
cur_total_size += eval_flags_total;
|
||||
|
||||
uint32_t dyn_buf_total = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
e->dyn_buf_header_offset_ = total_header_len + dyn_buf_total;
|
||||
e->dyn_buf_header_offset_ = cur_total_size + dyn_buf_total;
|
||||
dyn_buf_total += dynamic_buf_header_size(*e);
|
||||
}
|
||||
total_header_len += dyn_buf_total;
|
||||
cur_total_size += dyn_buf_total;
|
||||
|
||||
uint32_t expr_data_offset = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
@ -1064,13 +1187,15 @@ int ObStaticEngineExprCG::arrange_datums_data(ObIArray<ObRawExpr *> &exprs,
|
||||
e->pvt_skip_off_ = e->datum_off_ + get_datums_header_size(*e);
|
||||
// datum data part: reserved buf data + dynamic buf header
|
||||
e->res_buf_off_ =
|
||||
total_header_len + expr_data_offset;
|
||||
cur_total_size + expr_data_offset;
|
||||
expr_data_offset += e->res_buf_len_ * get_expr_datums_count(*e);
|
||||
LOG_TRACE("expression details during CG", K(e->is_batch_result()), KPC(e),
|
||||
K(expr_data_offset));
|
||||
}
|
||||
CK((total_header_len + expr_data_offset) == frame.frame_size_);
|
||||
CK((cur_total_size + expr_data_offset) == frame.frame_size_);
|
||||
} else {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("not support non-continuous datum", K(ret));
|
||||
// Layout2: Frame is seperated by exprs
|
||||
// All data(metas + reserved data/buf) within one expr are allocated continuously
|
||||
// Frame layouts:
|
||||
@ -1101,24 +1226,24 @@ int ObStaticEngineExprCG::arrange_datums_data(ObIArray<ObRawExpr *> &exprs,
|
||||
// |--------------------------------+
|
||||
// | ...... |
|
||||
// +--------------------------------+
|
||||
uint64_t expr_offset = 0;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
e->frame_idx_ = frame.frame_idx_;
|
||||
// datum meta/header part:
|
||||
e->datum_off_ = expr_offset;
|
||||
e->eval_info_off_ = e->datum_off_ + get_datums_header_size(*e);
|
||||
e->eval_flags_off_ = e->eval_info_off_ + sizeof(ObEvalInfo);
|
||||
e->pvt_skip_off_ = e->eval_flags_off_ + get_expr_skip_vector_size(*e);
|
||||
// datum data part: reserved buf data + dynamic buf header
|
||||
const int64_t cur_off = e->pvt_skip_off_ + get_expr_skip_vector_size(*e);
|
||||
e->res_buf_off_ = cur_off + dynamic_buf_header_size(*e);
|
||||
expr_offset = cur_off + reserve_datums_buf_len(*e);
|
||||
LOG_TRACE("expression details during CG", K(e->is_batch_result()), KPC(e),
|
||||
K(expr_offset));
|
||||
CK(get_expr_datums_size(*e) == (expr_offset - e->datum_off_));
|
||||
}
|
||||
CK(expr_offset == frame.frame_size_);
|
||||
// uint64_t expr_offset = 0;
|
||||
// for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
|
||||
// ObExpr *e = get_rt_expr(*exprs.at(i));
|
||||
// e->frame_idx_ = frame.frame_idx_;
|
||||
// // datum meta/header part:
|
||||
// e->datum_off_ = expr_offset;
|
||||
// e->eval_info_off_ = e->datum_off_ + get_datums_header_size(*e);
|
||||
// e->eval_flags_off_ = e->eval_info_off_ + sizeof(ObEvalInfo);
|
||||
// e->pvt_skip_off_ = e->eval_flags_off_ + get_expr_skip_vector_size(*e);
|
||||
// // datum data part: reserved buf data + dynamic buf header
|
||||
// const int64_t cur_off = e->pvt_skip_off_ + get_expr_skip_vector_size(*e);
|
||||
// e->res_buf_off_ = cur_off + dynamic_buf_header_size(*e);
|
||||
// expr_offset = cur_off + reserve_datums_buf_len(*e);
|
||||
// LOG_TRACE("expression details during CG", K(e->is_batch_result()), KPC(e),
|
||||
// K(expr_offset));
|
||||
// CK(get_expr_datums_size(*e) == (expr_offset - e->datum_off_));
|
||||
// }
|
||||
// CK(expr_offset == frame.frame_size_);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -1153,8 +1278,11 @@ int ObStaticEngineExprCG::alloc_const_frame(const ObIArray<ObRawExpr *> &exprs,
|
||||
tmp_obj))) {
|
||||
LOG_WARN("fail to deep copy obj", K(ret));
|
||||
} else {
|
||||
ObDatum *datum = reinterpret_cast<ObDatum *>(
|
||||
frame_mem + j * DATUM_EVAL_INFO_SIZE);
|
||||
int64_t item_size = DATUM_EVAL_INFO_SIZE;
|
||||
if (use_rich_format()) {
|
||||
item_size += sizeof(VectorHeader);
|
||||
}
|
||||
ObDatum *datum = reinterpret_cast<ObDatum *>(frame_mem + j * item_size);
|
||||
datum->ptr_ = frame_mem + rt_expr->res_buf_off_;
|
||||
datum->from_obj(tmp_obj);
|
||||
if (0 == datum->len_) {
|
||||
@ -1166,6 +1294,13 @@ int ObStaticEngineExprCG::alloc_const_frame(const ObIArray<ObRawExpr *> &exprs,
|
||||
}
|
||||
}
|
||||
}
|
||||
if (use_rich_format()) {
|
||||
ObEvalInfo *eval_info = reinterpret_cast<ObEvalInfo *>(
|
||||
frame_mem + rt_expr->eval_info_off_);
|
||||
VectorHeader *vec_header = reinterpret_cast<VectorHeader *>(
|
||||
frame_mem + rt_expr->vector_header_off_);
|
||||
vec_header->init_uniform_const_vector(rt_expr->get_vec_value_tc(), datum, eval_info);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && OB_FAIL(frame_ptrs.push_back(frame_mem))) {
|
||||
@ -1417,10 +1552,29 @@ int ObStaticEngineExprCG::create_tmp_frameinfo(const common::ObIArray<ObRawExpr
|
||||
for (int64_t expr_idx = 0; OB_SUCC(ret) && expr_idx < raw_exprs.count(); expr_idx++) {
|
||||
ObExpr *rt_expr = get_rt_expr(*raw_exprs.at(expr_idx));
|
||||
const int64_t expr_datums_size = get_expr_datums_size(*rt_expr);
|
||||
const int64_t expr_zero_init_pos = get_datums_header_size(*rt_expr)
|
||||
+ get_expr_skip_vector_size(*rt_expr);
|
||||
const int64_t expr_zero_init_size = dynamic_buf_header_size(*rt_expr) +
|
||||
sizeof(ObEvalInfo) + get_expr_skip_vector_size(*rt_expr);
|
||||
int64_t expr_zero_init_pos = 0;
|
||||
int64_t expr_zero_init_size = 0;
|
||||
if (use_rich_format()) {
|
||||
expr_zero_init_pos = get_datums_header_size(*rt_expr)
|
||||
// skip bitmap
|
||||
+ get_expr_skip_vector_size(*rt_expr)
|
||||
+ get_offsets_size(*rt_expr)
|
||||
+ get_ptrs_size(*rt_expr);
|
||||
|
||||
expr_zero_init_size = sizeof(VectorHeader)
|
||||
+ get_expr_bitmap_vector_size(*rt_expr) /*nulls*/
|
||||
+ cont_dynamic_buf_header_size(*rt_expr)
|
||||
+ dynamic_buf_header_size(*rt_expr)
|
||||
+ sizeof(ObEvalInfo)
|
||||
+ get_expr_bitmap_vector_size(*rt_expr);/*evalflags*/
|
||||
} else {
|
||||
expr_zero_init_pos = get_datums_header_size(*rt_expr)
|
||||
+ get_expr_skip_vector_size(*rt_expr);
|
||||
|
||||
expr_zero_init_size = dynamic_buf_header_size(*rt_expr)
|
||||
+ sizeof(ObEvalInfo)
|
||||
+ get_expr_bitmap_vector_size(*rt_expr); /*evalflags*/
|
||||
}
|
||||
|
||||
if (expr_datums_size > MAX_FRAME_SIZE) {
|
||||
// FIXME: should never hit this block.
|
||||
@ -1446,7 +1600,8 @@ int ObStaticEngineExprCG::create_tmp_frameinfo(const common::ObIArray<ObRawExpr
|
||||
frame_index_pos + frame_idx,
|
||||
frame_size,
|
||||
zero_init_pos,
|
||||
zero_init_size)))) {
|
||||
zero_init_size,
|
||||
use_rich_format())))) {
|
||||
LOG_WARN("fail to push frame_size", K(ret));
|
||||
} else {
|
||||
++frame_idx;
|
||||
@ -1467,7 +1622,8 @@ int ObStaticEngineExprCG::create_tmp_frameinfo(const common::ObIArray<ObRawExpr
|
||||
frame_index_pos + frame_idx,
|
||||
frame_size,
|
||||
zero_init_pos,
|
||||
zero_init_size)))) {
|
||||
zero_init_size,
|
||||
use_rich_format())))) {
|
||||
LOG_WARN("fail to push frame_size", K(ret), K(frame_size));
|
||||
}
|
||||
}
|
||||
@ -1576,6 +1732,10 @@ int ObStaticEngineExprCG::divide_probably_local_exprs(common::ObIArray<ObRawExpr
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObStaticEngineExprCG::use_rich_format() const {
|
||||
return op_cg_ctx_.session_->use_rich_format();
|
||||
}
|
||||
|
||||
int ObStaticEngineExprCG::gen_expr_with_row_desc(const ObRawExpr *expr,
|
||||
const RowDesc &row_desc,
|
||||
ObIAllocator &allocator,
|
||||
@ -1656,7 +1816,8 @@ static inline bool expr_is_added(ObExpr &e)
|
||||
int ObStaticEngineExprCG::generate_partial_expr_frame(
|
||||
const ObPhysicalPlan &plan,
|
||||
ObExprFrameInfo &partial_expr_frame_info,
|
||||
ObIArray<ObRawExpr *> &raw_exprs)
|
||||
ObIArray<ObRawExpr *> &raw_exprs,
|
||||
const bool use_rich_format)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ObExprFrameInfo &global = plan.get_expr_frame_info();
|
||||
@ -1771,14 +1932,20 @@ int ObStaticEngineExprCG::generate_partial_expr_frame(
|
||||
OZ(partial.param_frame_.assign(global.param_frame_));
|
||||
OZ(partial.dynamic_frame_.assign(global.dynamic_frame_));
|
||||
OZ(partial.datum_frame_.assign(global.datum_frame_));
|
||||
|
||||
int64_t item_size = 0;
|
||||
if (use_rich_format) {
|
||||
item_size = sizeof(ObDatum) + sizeof(ObEvalInfo) + sizeof(VectorHeader);
|
||||
} else {
|
||||
item_size = sizeof(ObDatum) + sizeof(ObEvalInfo);
|
||||
}
|
||||
auto set_param_frame = [&](ObFrameInfo &f) {
|
||||
ObExpr *e = farthest_exprs.at(f.frame_idx_);
|
||||
int cnt = NULL == e
|
||||
? std::min<int64_t>(1, f.expr_cnt_)
|
||||
: (e->datum_off_ / DATUM_EVAL_INFO_SIZE) + 1;
|
||||
: (e->datum_off_ / item_size) + 1;
|
||||
f.expr_cnt_ = std::min<int64_t>(f.expr_cnt_, cnt);
|
||||
f.frame_size_ = std::min<int64_t>(f.frame_size_, cnt * DATUM_EVAL_INFO_SIZE);
|
||||
f.frame_size_ = std::min<int64_t>(f.frame_size_, cnt * item_size);
|
||||
f.use_rich_format_ = plan.get_use_rich_format();
|
||||
};
|
||||
FOREACH_CNT_X(f, partial.param_frame_, OB_SUCC(ret)) {
|
||||
set_param_frame(*f);
|
||||
@ -1788,7 +1955,7 @@ int ObStaticEngineExprCG::generate_partial_expr_frame(
|
||||
}
|
||||
FOREACH_CNT_X(f, partial.datum_frame_, OB_SUCC(ret)) {
|
||||
ObExpr *e = farthest_exprs.at(f->frame_idx_);
|
||||
int64_t size = DATUM_EVAL_INFO_SIZE; // avoid zero frame size
|
||||
int64_t size = item_size; // avoid zero frame size
|
||||
if (NULL != e) {
|
||||
size = std::max(size, frame_max_offset(*e, batch_size));
|
||||
}
|
||||
|
||||
@ -80,9 +80,10 @@ public:
|
||||
uint32_t frame_idx,
|
||||
uint32_t frame_size,
|
||||
uint32_t zero_init_pos,
|
||||
uint32_t zero_init_size)
|
||||
uint32_t zero_init_size,
|
||||
bool use_rich_format)
|
||||
: expr_start_pos_(start_pos),
|
||||
frame_info_(expr_cnt, frame_idx, frame_size, zero_init_pos, zero_init_size)
|
||||
frame_info_(expr_cnt, frame_idx, frame_size, zero_init_pos, zero_init_size, use_rich_format)
|
||||
{}
|
||||
TO_STRING_KV(K_(expr_start_pos), K_(frame_info));
|
||||
public:
|
||||
@ -157,7 +158,8 @@ public:
|
||||
|
||||
static int generate_partial_expr_frame(const ObPhysicalPlan &plan,
|
||||
ObExprFrameInfo &partial_expr_frame_info,
|
||||
ObIArray<ObRawExpr *> &raw_exprs);
|
||||
ObIArray<ObRawExpr *> &raw_exprs,
|
||||
const bool use_rich_format);
|
||||
|
||||
void set_need_flatten_gen_col(const bool v) { need_flatten_gen_col_ = v; }
|
||||
|
||||
@ -318,7 +320,13 @@ private:
|
||||
|
||||
inline int64_t get_expr_skip_vector_size(const ObExpr &expr)
|
||||
{
|
||||
return expr.is_batch_result() ? ObBitVector::memory_size(batch_size_) : 0;
|
||||
return expr.is_batch_result() ? ObBitVector::memory_size(batch_size_) : 1;
|
||||
}
|
||||
|
||||
inline int64_t get_expr_bitmap_vector_size(const ObExpr &expr)
|
||||
{
|
||||
int64_t batch_size = expr.is_batch_result() ? batch_size_: 1;
|
||||
return ObBitVector::memory_size(batch_size);
|
||||
}
|
||||
int64_t dynamic_buf_header_size(const ObExpr &expr)
|
||||
{
|
||||
@ -357,7 +365,12 @@ private:
|
||||
|
||||
// total datums size: header + reserved data
|
||||
int64_t get_expr_datums_size(const ObExpr &expr) {
|
||||
return get_expr_datums_header_size(expr) + reserve_datums_buf_len(expr);
|
||||
int64_t size = get_expr_datums_header_size(expr) + reserve_datums_buf_len(expr);
|
||||
if (use_rich_format()) {
|
||||
size += get_rich_format_size(expr);
|
||||
}
|
||||
|
||||
return size;
|
||||
}
|
||||
|
||||
// datums meta/header size vector version.
|
||||
@ -367,9 +380,46 @@ private:
|
||||
// - EvalFlag(BitVector) instance + BitVector data
|
||||
// - SkipBitmap(BitVector) + BitVector data
|
||||
int64_t get_expr_datums_header_size(const ObExpr &expr) {
|
||||
return get_datums_header_size(expr) + sizeof(ObEvalInfo) +
|
||||
2 * get_expr_skip_vector_size(expr);
|
||||
int64_t size = get_datums_header_size(expr)
|
||||
+ sizeof(ObEvalInfo)
|
||||
+ get_expr_skip_vector_size(expr) /*skip*/
|
||||
+ get_expr_bitmap_vector_size(expr); /*eval flags*/
|
||||
|
||||
return size;
|
||||
}
|
||||
|
||||
int64_t get_vector_header_size() {
|
||||
return sizeof(VectorHeader);
|
||||
}
|
||||
|
||||
// ptrs
|
||||
int64_t get_ptrs_size(const ObExpr &expr) {
|
||||
return expr.is_fixed_length_data_ ? 0 : sizeof(char *) * get_expr_datums_count(expr);
|
||||
}
|
||||
|
||||
// cont dynamic buf header size
|
||||
int64_t cont_dynamic_buf_header_size(const ObExpr &expr) {
|
||||
return expr.is_fixed_length_data_
|
||||
? 0
|
||||
: sizeof(ObDynReserveBuf);
|
||||
}
|
||||
|
||||
// lens / offset
|
||||
int64_t get_offsets_size(const ObExpr &expr) {
|
||||
return expr.is_fixed_length_data_ ? 0 : sizeof(uint32_t) * (get_expr_datums_count(expr) + 1);
|
||||
}
|
||||
|
||||
int64_t get_rich_format_size(const ObExpr &expr) {
|
||||
int64_t size = 0;
|
||||
size += get_offsets_size(expr);
|
||||
size += get_ptrs_size(expr);
|
||||
size += get_vector_header_size();
|
||||
size += get_expr_bitmap_vector_size(expr); /* null bitmaps*/
|
||||
size += cont_dynamic_buf_header_size(expr);
|
||||
|
||||
return size;
|
||||
}
|
||||
|
||||
// datum meta/header size non-vector version.
|
||||
// two parts:
|
||||
// - datum instance
|
||||
@ -408,6 +458,8 @@ private:
|
||||
|
||||
int divide_probably_local_exprs(common::ObIArray<ObRawExpr *> &exprs);
|
||||
|
||||
bool use_rich_format() const;
|
||||
|
||||
private:
|
||||
int generate_extra_questionmarks(ObRawExprUniqueSet &flattened_raw_exprs);
|
||||
bool is_dynamic_eval_qm(const ObRawExpr &raw_expr) const;
|
||||
|
||||
@ -439,7 +439,9 @@ int ObTscCgService::generate_tsc_filter(const ObLogTableScan &op, ObTableScanSpe
|
||||
if (OB_FAIL(cg_.generate_rt_exprs(scan_pushdown_filters, scan_ctdef.pd_expr_spec_.pushdown_filters_))) {
|
||||
LOG_WARN("generate scan ctdef pushdown filter");
|
||||
} else if (pd_filter) {
|
||||
ObPushdownFilterConstructor filter_constructor(&cg_.phy_plan_->get_allocator(), cg_);
|
||||
ObPushdownFilterConstructor filter_constructor(
|
||||
&cg_.phy_plan_->get_allocator(), cg_,
|
||||
scan_ctdef.pd_expr_spec_.pd_storage_flag_.is_use_column_store());
|
||||
if (OB_FAIL(filter_constructor.apply(
|
||||
scan_pushdown_filters, scan_ctdef.pd_expr_spec_.pd_storage_filters_.get_pushdown_filter()))) {
|
||||
LOG_WARN("failed to apply filter constructor", K(ret));
|
||||
@ -451,7 +453,9 @@ int ObTscCgService::generate_tsc_filter(const ObLogTableScan &op, ObTableScanSpe
|
||||
if (OB_FAIL(cg_.generate_rt_exprs(lookup_pushdown_filters, lookup_ctdef->pd_expr_spec_.pushdown_filters_))) {
|
||||
LOG_WARN("generate lookup ctdef pushdown filter failed", K(ret));
|
||||
} else if (pd_filter) {
|
||||
ObPushdownFilterConstructor filter_constructor(&cg_.phy_plan_->get_allocator(), cg_);
|
||||
ObPushdownFilterConstructor filter_constructor(
|
||||
&cg_.phy_plan_->get_allocator(), cg_,
|
||||
lookup_ctdef->pd_expr_spec_.pd_storage_flag_.is_use_column_store());
|
||||
if (OB_FAIL(filter_constructor.apply(
|
||||
lookup_pushdown_filters, lookup_ctdef->pd_expr_spec_.pd_storage_filters_.get_pushdown_filter()))) {
|
||||
LOG_WARN("failed to apply filter constructor", K(ret));
|
||||
|
||||
Reference in New Issue
Block a user