[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

@ -30,31 +30,27 @@
#include "sql/ob_sql_context.h"
#include "sql/optimizer/ob_fd_item.h"
#include "sql/dblink/ob_spell_link_stmt.h"
#include "sql/monitor/ob_sql_plan.h"
#include "sql/monitor/ob_sql_plan_manager.h"
namespace oceanbase
{
namespace sql
{
class planText;
struct JoinFilterInfo;
struct EstimateCostInfo;
struct ObSqlPlanItem;
struct PlanText;
struct partition_location
{
int64_t partition_id;
// server id
TO_STRING_KV(K(partition_id));
};
static const char ID[] = "ID";
static const char OPERATOR[] = "OPERATOR";
static const char NAME[] = "NAME";
static const char ROWS[] = "EST.ROWS";
static const char COST[] = "COST";
static const char OPCOST[] = "OP_COST";
/**
* Print log info with expressions
*/
#define EXPLAIN_PRINT_EXPRS(exprs, type) \
#define EXPLAIN_PRINT_EXPRS(exprs, type) \
{ \
int64_t N = -1; \
if (OB_FAIL(ret)) { /* Do nothing */ \
@ -68,7 +64,7 @@ static const char OPCOST[] = "OP_COST";
} else if (OB_ISNULL(exprs.at(i))) { \
ret = OB_ERR_UNEXPECTED; \
} else if (OB_FAIL(exprs.at(i) \
->get_name(buf, buf_len, pos, type))) { /*Do nothing */ \
->get_name(buf, buf_len, pos, type))) { /*Do nothing */ \
} else if (OB_FAIL(BUF_PRINTF("]"))) { /* Do nothing */ \
} else if (i < N - 1) { \
ret = BUF_PRINTF(", "); \
@ -80,7 +76,7 @@ static const char OPCOST[] = "OP_COST";
} else { /* Do nothing */ } \
}
#define EXPLAIN_PRINT_EXPR(expr, type) \
#define EXPLAIN_PRINT_EXPR(expr, type) \
{ \
if (OB_FAIL(ret)) { /* Do nothing */ \
} else if (OB_FAIL(BUF_PRINTF(#expr"("))) { /* Do nothing */ \
@ -94,39 +90,6 @@ static const char OPCOST[] = "OP_COST";
} else { /* Do nothing */ } \
}
/**
* Print log info with expressions
*/
#define EXPLAIN_PRINT_SORT_KEYS(exprs, type) \
{ \
int64_t N = -1; \
if (OB_FAIL(ret)) { /* Do nothing */ \
} else if (OB_FAIL(BUF_PRINTF(#exprs"("))) { /* Do nothing */ \
} else if (FALSE_IT(N = exprs.count())) { \
} else if (N == 0) { \
BUF_PRINTF("nil"); \
} else { \
for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { \
if (OB_FAIL(BUF_PRINTF("["))) { /* Do nothing */ \
} else if (OB_ISNULL(exprs.at(i).expr_)) { \
ret = common::OB_ERR_UNEXPECTED; \
} else if (OB_FAIL(exprs.at(i).expr_ \
->get_name(buf, buf_len, pos, type))) { /* Do nothing */ \
} else if (exprs.at(i).is_descending()) { \
ret = BUF_PRINTF(", DESC"); \
} else { /* Do nothing */ } \
if (OB_FAIL(ret)) { /* Do nothing */ \
} else if (OB_FAIL(BUF_PRINTF("]"))) { /* Do nothing */ \
} else if (i < N - 1) { \
ret = BUF_PRINTF(", "); \
} else { /* Do nothing */ } \
} \
} \
if (OB_SUCC(ret)) { /* Do nothing */ \
ret = BUF_PRINTF(")"); \
} else { /* Do nothing */ } \
}
/**
* Print log info with exec params
*/
@ -158,41 +121,12 @@ static const char OPCOST[] = "OP_COST";
} else { /* Do nothing */ } \
}
/**
* Print log info with exec params
*/
#define EXPLAIN_PRINT_EXEC_PARAMS(exec_params, type) \
{ \
int64_t N = -1; \
if (OB_FAIL(ret)) { /* Do nothing */ \
} else if (OB_FAIL(BUF_PRINTF(#exec_params"("))) { /* Do nothing */ \
} else if (FALSE_IT(N = exec_params.count())) { /* Do nothing */ \
} else if (N == 0) { \
ret = BUF_PRINTF("nil"); \
} else { \
for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { \
if (OB_FAIL(BUF_PRINTF("["))) { /* Do nothing */ \
} else if (OB_ISNULL(exec_params.at(i).second)) { \
ret = OB_ERR_UNEXPECTED; \
} else if (OB_FAIL(exec_params.at(i).second \
->get_name(buf, buf_len, pos, type))) { \
} else if (OB_FAIL(BUF_PRINTF("]"))) { /* Do nothing */ \
} else if (i < N - 1) { \
ret = BUF_PRINTF(", "); \
} else { /* Do nothing */ } \
} \
} \
if (OB_SUCC(ret)) { \
ret = BUF_PRINTF(")"); \
} else { /* Do nothing */ } \
}
/**
* Print log info with idxs
*/
#define EXPLAIN_PRINT_IDXS(idxs) \
#define EXPLAIN_PRINT_IDXS(idxs) \
{ \
ObSEArray<int64_t, 4, common::ModulePageAllocator, true> arr; \
ObSEArray<int64_t, 4, common::ModulePageAllocator, true> arr; \
int64_t N = -1; \
if (OB_FAIL(ret)) { /* Do nothing */ \
} else if (OB_FAIL(BUF_PRINTF(#idxs"("))) { /* Do nothing */ \
@ -201,14 +135,14 @@ static const char OPCOST[] = "OP_COST";
} else if (N == 0) { \
ret = BUF_PRINTF("nil"); \
} else { \
for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { \
if (OB_FAIL(BUF_PRINTF("["))) { /* Do nothing */ \
} else if (OB_FAIL(BUF_PRINTF("%lu", arr.at(i)))) { /* Do nothing */ \
} else if (OB_FAIL(BUF_PRINTF("]"))) { /* Do nothing */ \
} else if (i < N - 1) { \
ret = BUF_PRINTF(", "); \
} else { /* Do nothing */ } \
} \
for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { \
if (OB_FAIL(BUF_PRINTF("["))) { /* Do nothing */ \
} else if (OB_FAIL(BUF_PRINTF("%lu", arr.at(i)))) { /* Do nothing */ \
} else if (OB_FAIL(BUF_PRINTF("]"))) { /* Do nothing */ \
} else if (i < N - 1) { \
ret = BUF_PRINTF(", "); \
} else { /* Do nothing */ } \
} \
} \
if (OB_SUCC(ret)) { /* Do nothing */ \
ret = BUF_PRINTF(")"); \
@ -218,7 +152,7 @@ static const char OPCOST[] = "OP_COST";
/**
* Print log info with expressions
*/
#define EXPLAIN_PRINT_MERGE_DIRECTIONS(directions) \
#define EXPLAIN_PRINT_MERGE_DIRECTIONS(directions) \
{ \
int64_t N = -1; \
if (OB_FAIL(ret)) { /* Do nothing */ \
@ -248,11 +182,11 @@ static const char OPCOST[] = "OP_COST";
/**
* Print log info with expressions
*/
#define EXPLAIN_PRINT_SORT_ITEMS(sort_keys, type) \
#define EXPLAIN_PRINT_SORT_ITEMS(sort_keys, type) \
{ \
int64_t N = -1; \
if (OB_FAIL(ret)) { /* Do nothing */ \
} else if (OB_FAIL(BUF_PRINTF(#sort_keys"("))) { /* Do nothing */ \
} else if (OB_FAIL(BUF_PRINTF(#sort_keys"("))) { /* Do nothing */ \
} else if (FALSE_IT(N = sort_keys.count())) { /* Do nothing */ \
} else if (0 == N) { \
ret = BUF_PRINTF("nil"); \
@ -262,7 +196,7 @@ static const char OPCOST[] = "OP_COST";
} else if(OB_ISNULL(sort_keys.at(i).expr_)) { \
ret = OB_ERR_UNEXPECTED; \
} else if (OB_FAIL(sort_keys.at(i).expr_ \
->get_name(buf, buf_len, pos, type))) { \
->get_name(buf, buf_len, pos, type))) { \
LOG_WARN("fail to get name", K(i), K(ret)); \
} else if (OB_FAIL(BUF_PRINTF(", "))) { /* Do nothing */ \
} else if (is_ascending_direction(sort_keys.at(i).order_type_)) { \
@ -282,6 +216,39 @@ static const char OPCOST[] = "OP_COST";
} else { /* Do nothing */ } \
}
/**
* Print log info with expressions
*/
#define EXPLAIN_PRINT_POPULAR_VALUES(popular_values) \
{ \
int64_t N = -1; \
if (OB_FAIL(ret)) { /* Do nothing */ \
} else if (OB_FAIL(BUF_PRINTF("popular_values("))) { /* Do nothing */ \
} else if (FALSE_IT(N = popular_values.count())) { /* Do nothing */ \
} else if (0 == N) { \
ret = BUF_PRINTF("nil"); \
} else { \
if (OB_FAIL(BUF_PRINTF("["))) { /* Do nothing */ \
} \
for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { \
if (OB_FAIL(popular_values.at(i).print_sql_literal( \
buf, buf_len, pos))) { \
LOG_WARN("fail to get name", K(i), K(ret)); \
} \
if (OB_FAIL(ret)) { /* Do nothing */ \
} else if(i < N - 1) { \
ret = BUF_PRINTF(","); \
} else { /* Do nothing */ } \
} \
if (OB_FAIL(ret)) { /* Do nothing */ \
} else if (OB_FAIL(BUF_PRINTF("]"))) { /* Do nothing */ \
} \
} \
if (OB_SUCC(ret)) { \
ret = BUF_PRINTF(")"); \
} else { /* Do nothing */ } \
}
/**
* these operator never generate expr
*/
@ -326,6 +293,7 @@ public:
GIS_IN_PARTITION_WISE,
GIS_PARTITION_WITH_AFFINITY,
GIS_PARTITION,
GIS_AFFINITY,
};
public:
explicit AllocGIContext() :
@ -338,8 +306,7 @@ public:
partition_count_(0),
hash_part_(false),
slave_mapping_type_(SM_NONE),
is_valid_for_gi_(false),
enable_gi_partition_pruning_(false)
is_valid_for_gi_(false)
{
}
~AllocGIContext()
@ -350,10 +317,11 @@ public:
bool is_in_pw_affinity_state();
bool is_partition_gi() { return GIS_PARTITION == state_; };
void set_in_partition_wise_state(ObLogicalOperator *op_ptr);
bool is_in_affinity_state();
void set_in_affinity_state(ObLogicalOperator *op_ptr);
bool try_set_out_partition_wise_state(ObLogicalOperator *op_ptr);
bool is_op_set_pw(ObLogicalOperator *op_ptr);
int set_pw_affinity_state();
void enable_gi_partition_pruning() { enable_gi_partition_pruning_ = true; }
void reset_info();
GIState get_state();
void add_exchange_op_count() { exchange_op_above_count_++; }
@ -387,7 +355,6 @@ public:
bool hash_part_;
SlaveMappingType slave_mapping_type_;
bool is_valid_for_gi_;
bool enable_gi_partition_pruning_;
};
class ObAllocGIInfo
@ -396,8 +363,7 @@ public:
ObAllocGIInfo () :
state_(AllocGIContext::GIS_NORMAL),
pw_op_ptr_(nullptr),
multi_child_op_above_count_in_dfo_(0),
enable_gi_partition_pruning_(false)
multi_child_op_above_count_in_dfo_(0)
{
}
TO_STRING_KV(K(state_),
@ -408,19 +374,16 @@ public:
state_ = ctx.state_;
pw_op_ptr_ = ctx.pw_op_ptr_;
multi_child_op_above_count_in_dfo_ = ctx.multi_child_op_above_count_in_dfo_;
enable_gi_partition_pruning_ = ctx.enable_gi_partition_pruning_;
}
void get_info(AllocGIContext &ctx)
{
ctx.state_ = state_;
ctx.pw_op_ptr_ = pw_op_ptr_;
ctx.multi_child_op_above_count_in_dfo_ = multi_child_op_above_count_in_dfo_;
ctx.enable_gi_partition_pruning_ = enable_gi_partition_pruning_;
}
AllocGIContext::GIState state_;
ObLogicalOperator *pw_op_ptr_;
int64_t multi_child_op_above_count_in_dfo_;
bool enable_gi_partition_pruning_;
};
class AllocMDContext
@ -473,6 +436,7 @@ struct ObExchangeInfo
repartition_func_exprs_(),
calc_part_id_expr_(NULL),
hash_dist_exprs_(),
popular_values_(),
dist_method_(ObPQDistributeMethod::LOCAL), // pull to local
unmatch_row_dist_method_(ObPQDistributeMethod::LOCAL),
null_row_dist_method_(ObNullDistributeMethod::NONE),
@ -481,6 +445,9 @@ struct ObExchangeInfo
weak_sharding_(),
need_null_aware_shuffle_(false),
is_rollup_hybrid_(false),
is_wf_hybrid_(false),
wf_hybrid_aggr_status_expr_(NULL),
wf_hybrid_pby_exprs_cnt_array_(),
may_add_interval_part_(MayAddIntervalPart::NO),
sample_type_(NOT_INIT_SAMPLE_TYPE)
{
@ -521,6 +488,8 @@ struct ObExchangeInfo
common::ObSEArray<ObRawExpr*, 4> repartition_func_exprs_;
ObRawExpr *calc_part_id_expr_;
common::ObSEArray<HashExpr, 4> hash_dist_exprs_;
// for hybrid hash distr
common::ObSEArray<ObObj, 20> popular_values_;
ObPQDistributeMethod::Type dist_method_;
ObPQDistributeMethod::Type unmatch_row_dist_method_;
ObNullDistributeMethod::Type null_row_dist_method_;
@ -533,6 +502,10 @@ struct ObExchangeInfo
// for null aware anti join
bool need_null_aware_shuffle_;
bool is_rollup_hybrid_;
bool is_wf_hybrid_;
ObRawExpr *wf_hybrid_aggr_status_expr_;
// pby exprs cnt of every wf for wf hybrid dist
common::ObSEArray<int64_t, 4> wf_hybrid_pby_exprs_cnt_array_;
MayAddIntervalPart may_add_interval_part_;
// sample type for range distribution or partition range distribution
ObPxSampleType sample_type_;
@ -556,6 +529,8 @@ struct ObExchangeInfo
K_(slave_mapping_type),
K_(need_null_aware_shuffle),
K_(is_rollup_hybrid),
K_(is_wf_hybrid),
K_(wf_hybrid_pby_exprs_cnt_array),
K_(may_add_interval_part),
K_(sample_type));
private:
@ -614,10 +589,6 @@ public:
is_shared_ = other.is_shared_;
return *this;
}
bool not_produced() const
{
return common::OB_INVALID_ID == producer_branch_;
}
TO_STRING_KV(K_(consumer_id), K_(producer_id), K_(producer_branch), K(is_shared_), K(expr_), KPC_(expr));
ObRawExpr *expr_;
@ -660,11 +631,11 @@ struct ObAllocExprContext
int add(const ExprProducer &producer);
int add_flattern_expr(const ObRawExpr* expr);
int get_expr_ref_cnt(const ObRawExpr* expr, int64_t &ref_cnt);
DISALLOW_COPY_AND_ASSIGN(ObAllocExprContext);
// record producers expr index to search producer by expr quickly
// {key => ExprProducer.expr_ , value => index of expr_producers_}
hash::ObHashMap<uint64_t, int64_t> expr_map_;
@ -1093,6 +1064,15 @@ public:
return op_ordering_;
}
inline common::ObIArray<common::ObAddr> &get_server_list()
{
return server_list_;
}
inline const common::ObIArray<common::ObAddr> &get_server_list() const
{
return server_list_;
}
inline const ObFdItemSet& get_fd_item_set() const
{
return NULL == fd_item_set_ ? empty_fd_item_set_ : *fd_item_set_;
@ -1164,37 +1144,6 @@ public:
inline int64_t get_interesting_order_info() const { return interesting_order_info_; }
inline void set_interesting_order_info(int64_t info) { interesting_order_info_ = info; }
inline bool has_any_interesting_order_info_flag() const { return interesting_order_info_ > 0; }
/**
* Do explain collect width operation(pre)
*/
int explain_collect_width_pre(void *ctx);
/**
* Do explain collect width operation(post)
*/
int explain_collect_width_post(void *ctx);
/**
* Do explain write buffer operation(pre)
*/
int explain_write_buffer_pre(void *ctx);
int explain_index_selection_info_pre(void *ctx);
/**
* Do explain write buffer operation(pre)
*/
int explain_write_buffer_post(void *ctx);
/**
* Do explain write buffer with output & filter exprs
*/
int explain_write_buffer_output_pre(void *ctx);
/**
* Do explain write buffer with outline
*/
int explain_write_buffer_outline_pre(void *ctx);
/**
* Do pre-child-traverse operation
@ -1206,13 +1155,6 @@ public:
*/
int do_post_traverse_operation(const TraverseOp &op, void *ctx);
/**
* Rerturn a JSON object of the operator
*
* 'buf' is used when calling 'to_string()' internally.
*/
virtual int to_json(char *buf, const int64_t buf_len, int64_t &pos, json::Value *&ret_val);
/**
* Get predefined operator name
*/
@ -1221,14 +1163,6 @@ public:
return log_op_def::get_op_name(type_);
}
/**
* Get the length of the operator name
*/
inline virtual int32_t get_explain_name_length() const
{
return ((int32_t) strlen(get_name()));
}
virtual int get_explain_name_internal(char *buf, const int64_t buf_len, int64_t &pos)
{
int ret = common::OB_SUCCESS;
@ -1243,16 +1177,6 @@ public:
return common::OB_SUCCESS;
}
virtual int explain_index_selection_info(char *buf,
int64_t &buf_len,
int64_t &pos)
{
int ret = common::OB_SUCCESS;
UNUSED(buf);
UNUSED(buf_len);
UNUSED(pos);
return ret;
}
virtual int allocate_expr_pre(ObAllocExprContext &ctx);
virtual int get_op_exprs(ObIArray<ObRawExpr*> &all_exprs);
@ -1441,9 +1365,6 @@ public:
virtual int allocate_startup_expr_post();
int allocate_startup_expr_post(int64_t child_idx);
int allocate_link_node_above(int64_t child_idx);
virtual int copy_part_expr_pre(CopyPartExprCtx &ctx);
int copy_part_expr(CopyPartExprCtx &ctx, ObRawExpr* &calc_part_id_expr);
int check_is_on_null_side(ObLogicalOperator *op, bool &is_on_null_side);
virtual int collect_link_sql_context_pre(GenLinkStmtPostContext &link_ctx);
virtual int generate_link_sql_post(GenLinkStmtPostContext &link_ctx);
@ -1564,9 +1485,9 @@ public:
return log_op_def::LOG_OPTIMIZER_STATS_GATHERING == type_;
}
/*
* replace agg expr generated by group by push down during allocating exchange
* replace exprs which will be returned by get_op_exprs during allocating expr
*/
int replace_generated_agg_expr(
int replace_op_exprs(
const common::ObIArray<std::pair<ObRawExpr *, ObRawExpr*> >&to_replace_exprs);
// check if this operator is some kind of table scan.
@ -1587,8 +1508,6 @@ public:
int pw_allocate_granule_pre(AllocGIContext &ctx);
int pw_allocate_granule_post(AllocGIContext &ctx);
int print_operator_for_use_join_filter(planText &plan_text);
// dblink
void set_dblink_id(uint64_t dblink_id) { dblink_id_ = dblink_id; }
uint64_t get_dblink_id() const { return dblink_id_; }
@ -1652,6 +1571,46 @@ public:
int has_window_function_below(bool &has_win_func) const;
int get_pushdown_op(log_op_def::ObLogOpType op_type, const ObLogicalOperator *&op) const;
virtual int get_plan_item_info(PlanText &plan_text,
ObSqlPlanItem &plan_item);
virtual int print_outline_data(PlanText &plan_text);
virtual int print_used_hint(PlanText &plan_text);
int print_outline_table(PlanText &plan_text, const TableItem *table_item) const;
/**
* pure partition wise:
* match multiple children and all children meet strict partition wise
* e.g, original pw join and pw set(not union all)
*
* affinite partition wise:
* match multiple children and all children meet partition wise with affinity
* e.g, union all with the same data distribution node for all children
*
* extended partition wise:
* match multiple children and all children have already with hash-hash distribution
* and no need additional exchange allocated
* e.g, union all hash-hash and none-none hash join
*/
inline bool is_pure_partition_wise()
{
return is_partition_wise_ && !exchange_allocated_ &&
NULL != get_sharding() &&
get_sharding()->is_distributed_with_table_location_and_partitioning();
}
inline bool is_affinite_partition_wise()
{
return is_partition_wise_ && exchange_allocated_ &&
NULL != get_sharding() &&
get_sharding()->is_distributed_with_table_location_and_partitioning();
}
inline bool is_extended_partition_wise()
{
return is_partition_wise_ && exchange_allocated_ &&
NULL != get_sharding() &&
get_sharding()->is_distributed_without_table_location_with_partitioning();
}
int collect_batch_exec_param_pre(void* ctx);
int collect_batch_exec_param_post(void* ctx);
int collect_batch_exec_param(void* ctx,
@ -1661,6 +1620,7 @@ public:
public:
ObSEArray<ObLogicalOperator *, 16, common::ModulePageAllocator, true> child_;
ObSEArray<ObPCParamEqualInfo, 4, common::ModulePageAllocator, true> equal_param_constraints_;
ObSEArray<ObPCConstParamInfo, 4, common::ModulePageAllocator, true> const_param_constraints_;
ObSEArray<ObExprConstraint, 4, common::ModulePageAllocator, true> expr_constraints_;
protected:
enum TraverseType
@ -1679,65 +1639,12 @@ protected:
BUSHY
};
/**
* Check if all expresionss are produced by some operator.
*/
bool is_all_expr_produced(const common::ObIArray<ExprProducer> &ctx);
/**
* Print the footer of the explain plan table
*/
int print_plan_annotation(char *buf,
int64_t &buf_len,
int64_t &pos,
ExplainType type);
/**
* Interface for operators to print their specific info in the plan table footer
*/
virtual int print_my_plan_annotation(char *buf,
int64_t &buf_len,
int64_t &pos,
ExplainType type)
{
int ret = common::OB_SUCCESS;
UNUSED(buf);
UNUSED(buf_len);
UNUSED(pos);
UNUSED(type);
return ret;
}
/**
* Interface for operators to print their specific info in the plan table footer
*/
virtual int print_plan_head_annotation(char *buf,
int64_t &buf_len,
int64_t &pos,
ExplainType type)
{
int ret = common::OB_SUCCESS;
UNUSED(buf);
UNUSED(buf_len);
UNUSED(pos);
UNUSED(type);
return ret;
}
virtual int print_outline(planText &plan)
{
int ret = common::OB_SUCCESS;
UNUSED(plan);
return ret;
}
int project_pruning_pre();
virtual int check_output_dependance(common::ObIArray<ObRawExpr *> &child_output, PPDeps &deps);
void do_project_pruning(common::ObIArray<ObRawExpr *> &exprs,
PPDeps &deps);
int try_add_remove_const_exprs();
virtual int inner_replace_generated_agg_expr(
virtual int inner_replace_op_exprs(
const common::ObIArray<std::pair<ObRawExpr *, ObRawExpr*> >&to_replace_exprs);
int replace_exprs_action(const common::ObIArray<std::pair<ObRawExpr *, ObRawExpr*> >&to_replace_exprs,
common::ObIArray<ObRawExpr*> &dest_exprs);
@ -1839,7 +1746,10 @@ private:
int find_px_for_batch_rescan(const log_op_def::ObLogOpType, int64_t op_id, bool &find);
int find_nested_dis_rescan(bool &find, bool nested);
int add_op_exprs(ObRawExpr* expr);
// alloc mat for sync in output
int need_alloc_material_for_shared_hj(ObLogicalOperator &curr_op, bool &need_alloc);
// alloc mat for sync in intput
int need_alloc_material_for_push_down_wf(ObLogicalOperator &curr_op, bool &need_alloc);
private:
ObLogicalOperator *parent_; // parent operator
bool is_plan_root_; // plan root operator
@ -1888,6 +1798,7 @@ protected:
int64_t interesting_order_info_; // 记录算子的序在stmt中的哪些地方用到 e.g. join, group by, order by
int64_t parallel_;
int64_t server_cnt_;
ObSEArray<common::ObAddr, 8, common::ModulePageAllocator, true> server_list_;
bool need_late_materialization_;
// all non_const exprs for this op, generated by allocate_expr_pre and used by project pruning
ObSEArray<ObRawExpr*, 8, common::ModulePageAllocator, true> op_exprs_;