adjust allocate window function operator, enhance pq_distribute_window hint
This commit is contained in:
parent
9a244755cd
commit
32636e05c4
@ -1278,6 +1278,7 @@ typedef enum ObItemType
|
||||
T_MONITOR,
|
||||
T_PQ_DISTRIBUTE,
|
||||
T_PQ_DISTRIBUTE_WINDOW,
|
||||
T_WIN_FUNC_IDX_LIST,
|
||||
T_PQ_SET,
|
||||
T_DISTRIBUTE_BROADCAST,
|
||||
T_DISTRIBUTE_HASH,
|
||||
|
@ -143,15 +143,10 @@ int ObExplainLogPlan::check_explain_generate_plan_with_outline(ObLogPlan *real_p
|
||||
} else if (0 == sql_ctx->first_plan_hash_) { /* generate plan first time */
|
||||
void *tmp_ptr = NULL;
|
||||
sql_ctx->first_outline_data_.reset();
|
||||
bool has_win_func = false;
|
||||
if (OB_UNLIKELY(0 == real_plan->get_signature()) || 0 < sql_ctx->retry_times_) {
|
||||
/* do nothing */
|
||||
} else if (OB_SUCC(OB_E(EventTable::EN_EXPLAIN_GENERATE_PLAN_WITH_OUTLINE) OB_SUCCESS)) {
|
||||
/* do nothing */
|
||||
} else if (OB_FAIL(check_has_win_func(explain_stmt->get_explain_query_stmt(), has_win_func))) {
|
||||
LOG_WARN("failed to check has window function", K(ret));
|
||||
} else if (has_win_func) {
|
||||
/* outline is usually invalid for plan contain window functions, remove this after some hint is added. */
|
||||
} else if (OB_UNLIKELY(NULL == (tmp_ptr = get_optimizer_context().get_allocator().alloc(OB_MAX_SQL_LENGTH)))) {
|
||||
/* allocator in optimizer context is from ObResultSet */
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
@ -198,26 +193,3 @@ int ObExplainLogPlan::check_explain_generate_plan_with_outline(ObLogPlan *real_p
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
|
||||
int ObExplainLogPlan::check_has_win_func(const ObDMLStmt *stmt, bool &has_win_func)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(stmt)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("stmt is null", K(ret), K(stmt));
|
||||
} else if (stmt->is_select_stmt() && static_cast<const ObSelectStmt*>(stmt)->has_window_function()) {
|
||||
has_win_func = true;
|
||||
} else {
|
||||
ObSEArray<ObSelectStmt*, 8> child_stmts;
|
||||
if (OB_FAIL(stmt->get_child_stmts(child_stmts))) {
|
||||
LOG_WARN("get child stmt failed", K(ret));
|
||||
}
|
||||
for (int64_t i = 0; !has_win_func && OB_SUCC(ret) && i < child_stmts.count(); ++i) {
|
||||
if (OB_FAIL(SMART_CALL(check_has_win_func(child_stmts.at(i), has_win_func)))) {
|
||||
LOG_WARN("failed to check has window function", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
@ -29,7 +29,6 @@ namespace sql
|
||||
virtual int generate_normal_raw_plan() override;
|
||||
private:
|
||||
int check_explain_generate_plan_with_outline(ObLogPlan *real_plan);
|
||||
int check_has_win_func(const ObDMLStmt *stmt, bool &has_win_func);
|
||||
DISALLOW_COPY_AND_ASSIGN(ObExplainLogPlan);
|
||||
};
|
||||
}
|
||||
|
@ -257,14 +257,30 @@ inline DistAlgo get_opposite_distributed_type(DistAlgo dist_type)
|
||||
}
|
||||
|
||||
// Window function distribution
|
||||
enum class WinDistAlgo
|
||||
enum WinDistAlgo
|
||||
{
|
||||
NONE = 0,
|
||||
HASH = 1, // hash distribute
|
||||
RANGE = 2, // range distribute
|
||||
LIST = 3 // range + random distribute
|
||||
WIN_DIST_INVALID = 0,
|
||||
WIN_DIST_NONE = (1UL),
|
||||
WIN_DIST_HASH = (1UL << 1), // hash distribute
|
||||
WIN_DIST_RANGE = (1UL << 2), // range distribute
|
||||
WIN_DIST_LIST = (1UL << 3) // range + random distribute
|
||||
};
|
||||
|
||||
inline WinDistAlgo get_win_dist_algo(uint64_t method)
|
||||
{
|
||||
if (method & WinDistAlgo::WIN_DIST_LIST) {
|
||||
return WinDistAlgo::WIN_DIST_LIST;
|
||||
} else if (method & WinDistAlgo::WIN_DIST_RANGE) {
|
||||
return WinDistAlgo::WIN_DIST_RANGE;
|
||||
} else if (method & WinDistAlgo::WIN_DIST_HASH) {
|
||||
return WinDistAlgo::WIN_DIST_HASH;
|
||||
} else if (method & WinDistAlgo::WIN_DIST_NONE) {
|
||||
return WinDistAlgo::WIN_DIST_NONE;
|
||||
} else {
|
||||
return WinDistAlgo::WIN_DIST_INVALID;
|
||||
}
|
||||
}
|
||||
|
||||
class ObLogPlan;
|
||||
class ObLogOperatorFactory
|
||||
{
|
||||
|
@ -8043,7 +8043,7 @@ int ObLogPlan::allocate_sort_and_exchange_as_top(ObLogicalOperator *&top,
|
||||
const bool is_local_order,
|
||||
ObRawExpr *topn_expr,
|
||||
bool is_fetch_with_ties,
|
||||
OrderItem *hash_sortkey)
|
||||
const OrderItem *hash_sortkey)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(top)) {
|
||||
@ -8240,7 +8240,7 @@ int ObLogPlan::allocate_sort_as_top(ObLogicalOperator *&top,
|
||||
const bool is_local_merge_sort,
|
||||
ObRawExpr *topn_expr,
|
||||
bool is_fetch_with_ties,
|
||||
OrderItem *hash_sortkey)
|
||||
const OrderItem *hash_sortkey)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObLogSort *sort = NULL;
|
||||
|
@ -834,10 +834,10 @@ public:
|
||||
const ObIArray<OrderItem> &sort_keys,
|
||||
const bool need_sort,
|
||||
const int64_t prefix_pos,
|
||||
const bool is_partition_wise,
|
||||
const bool is_local_order,
|
||||
ObRawExpr *topn_expr = NULL,
|
||||
bool is_fetch_with_ties = false,
|
||||
OrderItem *hash_sortkey = NULL);
|
||||
const OrderItem *hash_sortkey = NULL);
|
||||
|
||||
int allocate_dist_range_sort_as_top(ObLogicalOperator *&top,
|
||||
const ObIArray<OrderItem> &sort_keys,
|
||||
@ -856,7 +856,7 @@ public:
|
||||
const bool is_local_merge_sort = false,
|
||||
ObRawExpr *topn_expr = NULL,
|
||||
bool is_fetch_with_ties = false,
|
||||
OrderItem *hash_sortkey = NULL);
|
||||
const OrderItem *hash_sortkey = NULL);
|
||||
|
||||
int allocate_exchange_as_top(ObLogicalOperator *&top,
|
||||
const ObExchangeInfo &exch_info);
|
||||
@ -1730,9 +1730,11 @@ public:
|
||||
const ObLogPlanHint &get_log_plan_hint() const { return log_plan_hint_; }
|
||||
bool has_join_order_hint() { return !log_plan_hint_.join_order_.leading_tables_.is_empty(); }
|
||||
const ObRelIds& get_leading_tables() { return log_plan_hint_.join_order_.leading_tables_; }
|
||||
void set_added_leading() { outline_print_flags_ |= ADDED_LEADING_HINT; }
|
||||
void reset_outline_print_flags() { outline_print_flags_ = 0; }
|
||||
bool has_added_leading() const { return outline_print_flags_ & ADDED_LEADING_HINT; }
|
||||
void set_added_leading() { outline_print_flags_ |= ADDED_LEADING_HINT; }
|
||||
bool has_added_win_dist() const { return outline_print_flags_ & ADDED_WIN_DIST_HINT; }
|
||||
void set_added_win_dist() { outline_print_flags_ |= ADDED_WIN_DIST_HINT; }
|
||||
const common::ObIArray<ObRawExpr*> &get_onetime_query_refs() const { return onetime_query_refs_; }
|
||||
private:
|
||||
static const int64_t IDP_PATHNUM_THRESHOLD = 5000;
|
||||
@ -1817,7 +1819,8 @@ private:
|
||||
|
||||
ObLogPlanHint log_plan_hint_;
|
||||
enum OUTLINE_PRINT_FLAG {
|
||||
ADDED_LEADING_HINT = 1 << 0
|
||||
ADDED_LEADING_HINT = 1 << 0,
|
||||
ADDED_WIN_DIST_HINT = 1 << 1
|
||||
};
|
||||
uint64_t outline_print_flags_; // used print outline
|
||||
common::ObSEArray<ObRelIds, 8, common::ModulePageAllocator, true> bushy_tree_infos_;
|
||||
|
@ -387,24 +387,57 @@ bool ObLogWindowFunction::is_block_op() const
|
||||
int ObLogWindowFunction::print_outline_data(PlanText &plan_text)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ObDMLStmt *stmt = NULL;
|
||||
const ObSelectStmt *stmt = NULL;
|
||||
ObString qb_name;
|
||||
if (OB_ISNULL(get_plan()) || OB_ISNULL(stmt = get_plan()->get_stmt())) {
|
||||
if (OB_ISNULL(get_plan()) || OB_ISNULL(stmt = dynamic_cast<const ObSelectStmt*>(get_plan()->get_stmt()))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected NULL", K(ret), K(get_plan()), K(stmt));
|
||||
} else if (get_plan()->has_added_win_dist()) {
|
||||
/* do nothing */
|
||||
} else if (OB_FAIL(stmt->get_qb_name(qb_name))) {
|
||||
LOG_WARN("get qb name failed", K(ret));
|
||||
} else {
|
||||
if (!dist_hint_.empty()) {
|
||||
ObWindowDistHint win_dist;
|
||||
if (OB_FAIL(stmt->get_qb_name(qb_name))) {
|
||||
LOG_WARN("get qb name failed", K(ret));
|
||||
} else if (OB_FAIL(win_dist.get_algos().assign(dist_hint_))) {
|
||||
LOG_WARN("array assign failed", K(ret));
|
||||
} else {
|
||||
win_dist.set_qb_name(qb_name);
|
||||
if (OB_FAIL(win_dist.print_hint(plan_text))) {
|
||||
LOG_WARN("print hint failed", K(ret));
|
||||
}
|
||||
}
|
||||
get_plan()->set_added_win_dist();
|
||||
ObWindowDistHint win_dist_hint;
|
||||
win_dist_hint.set_qb_name(qb_name);
|
||||
if (OB_FAIL(add_win_dist_options(this, stmt->get_window_func_exprs(), win_dist_hint))) {
|
||||
LOG_WARN("failed to add win dist options", K(ret));
|
||||
} else if (OB_FAIL(win_dist_hint.print_hint(plan_text))) {
|
||||
LOG_WARN("print hint failed", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObLogWindowFunction::add_win_dist_options(const ObLogicalOperator *op,
|
||||
const ObIArray<ObWinFunRawExpr*> &all_win_funcs,
|
||||
ObWindowDistHint &win_dist_hint)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(op)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected NULL", K(ret), K(op));
|
||||
} else if (LOG_WINDOW_FUNCTION != op->get_type()
|
||||
&& LOG_EXCHANGE != op->get_type()
|
||||
&& LOG_SORT != op->get_type()
|
||||
&& LOG_GRANULE_ITERATOR != op->get_type()
|
||||
&& LOG_TOPK != op->get_type()
|
||||
&& LOG_MATERIAL != op->get_type()) {
|
||||
/* do nothing */
|
||||
} else if (OB_FAIL(SMART_CALL(add_win_dist_options(op->get_child(ObLogicalOperator::first_child),
|
||||
all_win_funcs,
|
||||
win_dist_hint)))) {
|
||||
LOG_WARN("failed to add win dist options", K(ret));
|
||||
} else if (LOG_WINDOW_FUNCTION == op->get_type()) {
|
||||
const ObLogWindowFunction *win_func = static_cast<const ObLogWindowFunction*>(op);
|
||||
if (win_func->is_consolidator()) {
|
||||
/* CONSOLIDATOR replaced win_expr, generate outline hint in PARTICIPATOR only */
|
||||
} else if (OB_FAIL(win_dist_hint.add_win_dist_option(all_win_funcs,
|
||||
win_func->get_window_exprs(),
|
||||
win_func->get_win_dist_algo(),
|
||||
win_func->is_push_down(),
|
||||
win_func->get_use_hash_sort()))) {
|
||||
LOG_WARN("failed to add win dist option", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -413,16 +446,35 @@ int ObLogWindowFunction::print_outline_data(PlanText &plan_text)
|
||||
int ObLogWindowFunction::print_used_hint(PlanText &plan_text)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(get_plan())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected NULL", K(ret), K(get_plan()));
|
||||
const ObWindowDistHint *win_dist_hint = NULL;
|
||||
const ObSelectStmt *stmt = NULL;
|
||||
if (OB_ISNULL(get_plan()) || OB_ISNULL(stmt = dynamic_cast<const ObSelectStmt*>(get_plan()->get_stmt()))) {
|
||||
LOG_WARN("unexpected NULL", K(ret), K(get_plan()), K(stmt));
|
||||
} else if (OB_FALSE_IT(win_dist_hint = get_plan()->get_log_plan_hint().get_window_dist())) {
|
||||
} else if (NULL == win_dist_hint || get_plan()->has_added_win_dist()) {
|
||||
/* do nothing */
|
||||
} else {
|
||||
auto win_dist = get_plan()->get_log_plan_hint().get_window_dist();
|
||||
if (NULL != win_dist && !win_dist->get_algos().empty()) {
|
||||
if (is_array_equal(dist_hint_, win_dist->get_algos())) {
|
||||
if (OB_FAIL(win_dist->print_hint(plan_text))) {
|
||||
LOG_WARN("print hint failed", K(ret));
|
||||
}
|
||||
get_plan()->set_added_win_dist();
|
||||
ObWindowDistHint outline_hint;
|
||||
if (OB_FAIL(add_win_dist_options(this, stmt->get_window_func_exprs(), outline_hint))) {
|
||||
LOG_WARN("failed to add win dist options", K(ret));
|
||||
} else if (win_dist_hint->get_win_dist_options().count() > outline_hint.get_win_dist_options().count()) {
|
||||
/* do nothing */
|
||||
} else {
|
||||
bool hint_match = true;
|
||||
const ObIArray<ObWindowDistHint::WinDistOption> &hint_opts = win_dist_hint->get_win_dist_options();
|
||||
const ObIArray<ObWindowDistHint::WinDistOption> &outline_opts = outline_hint.get_win_dist_options();
|
||||
for (int64_t i = 0; hint_match && OB_SUCC(ret) && i < hint_opts.count(); ++i) {
|
||||
const ObWindowDistHint::WinDistOption &hint_opt = hint_opts.at(i);
|
||||
const ObWindowDistHint::WinDistOption &outline_opt = outline_opts.at(i);
|
||||
hint_match = hint_opt.algo_ == outline_opt.algo_
|
||||
&& hint_opt.use_hash_sort_ == outline_opt.use_hash_sort_
|
||||
&& hint_opt.is_push_down_ == outline_opt.is_push_down_
|
||||
&& (hint_opt.win_func_idxs_.empty()
|
||||
|| is_array_equal(hint_opt.win_func_idxs_, outline_opt.win_func_idxs_));
|
||||
}
|
||||
if (OB_SUCC(ret) && hint_match && OB_FAIL(win_dist_hint->print_hint(plan_text))) {
|
||||
LOG_WARN("print hint failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -29,6 +29,8 @@ namespace sql
|
||||
};
|
||||
ObLogWindowFunction(ObLogPlan &plan)
|
||||
: ObLogicalOperator(plan),
|
||||
algo_(WinDistAlgo::WIN_DIST_INVALID),
|
||||
use_hash_sort_(false),
|
||||
single_part_parallel_(false),
|
||||
range_dist_parallel_(false),
|
||||
role_type_(WindowFunctionRoleType::NORMAL),
|
||||
@ -46,6 +48,7 @@ namespace sql
|
||||
inline int add_window_expr(ObWinFunRawExpr *win_expr)
|
||||
{ return win_exprs_.push_back(win_expr); }
|
||||
inline ObIArray<ObWinFunRawExpr *> &get_window_exprs() { return win_exprs_; }
|
||||
inline const ObIArray<ObWinFunRawExpr *> &get_window_exprs() const { return win_exprs_; }
|
||||
virtual uint64_t hash(uint64_t seed) const override;
|
||||
virtual int est_cost() override;
|
||||
virtual int est_width() override;
|
||||
@ -67,9 +70,9 @@ namespace sql
|
||||
int get_winfunc_output_exprs(ObIArray<ObRawExpr *> &output_exprs);
|
||||
void set_role_type(WindowFunctionRoleType v) { role_type_ = v; }
|
||||
WindowFunctionRoleType get_role_type() const { return role_type_; }
|
||||
bool is_push_down() { return PARTICIPATOR == role_type_|| CONSOLIDATOR == role_type_; }
|
||||
bool is_participator() { return PARTICIPATOR == role_type_; }
|
||||
bool is_consolidator() { return CONSOLIDATOR == role_type_; }
|
||||
bool is_push_down() const { return PARTICIPATOR == role_type_|| CONSOLIDATOR == role_type_; }
|
||||
bool is_participator() const { return PARTICIPATOR == role_type_; }
|
||||
bool is_consolidator() const { return CONSOLIDATOR == role_type_; }
|
||||
int get_rd_sort_keys(common::ObIArray<OrderItem> &rd_sort_keys);
|
||||
int set_sort_keys(const common::ObIArray<OrderItem> &sort_keys)
|
||||
{
|
||||
@ -91,18 +94,24 @@ namespace sql
|
||||
void set_rd_pby_sort_cnt(const int64_t cnt) { rd_pby_sort_cnt_ = cnt; }
|
||||
int64_t get_rd_pby_sort_cnt() const { return rd_pby_sort_cnt_; }
|
||||
|
||||
int set_dist_hint(const common::ObIArray<WinDistAlgo> &dist_hint)
|
||||
{
|
||||
return dist_hint_.assign(dist_hint);
|
||||
}
|
||||
void set_win_dist_algo(const WinDistAlgo algo) { algo_ = algo; }
|
||||
WinDistAlgo get_win_dist_algo() const { return algo_; }
|
||||
void set_use_hash_sort(const bool use_hash_sort) { use_hash_sort_ = use_hash_sort; }
|
||||
bool get_use_hash_sort() const { return use_hash_sort_; }
|
||||
virtual int get_plan_item_info(PlanText &plan_text,
|
||||
ObSqlPlanItem &plan_item) override;
|
||||
virtual int print_outline_data(PlanText &plan_text) override;
|
||||
virtual int print_used_hint(PlanText &plan_text) override;
|
||||
|
||||
int add_win_dist_options(const ObLogicalOperator *op,
|
||||
const ObIArray<ObWinFunRawExpr*> &all_win_funcs,
|
||||
ObWindowDistHint &win_dist_hint);
|
||||
private:
|
||||
ObSEArray<ObWinFunRawExpr *, 4, common::ModulePageAllocator, true> win_exprs_;
|
||||
|
||||
// for print PQ_DISTRIBUTE_WINDOW hint outline
|
||||
WinDistAlgo algo_;
|
||||
bool use_hash_sort_;
|
||||
|
||||
// Single partition (no partition by) window function parallel process, need the PX COORD
|
||||
// to collect the partial result and broadcast the final result to each worker.
|
||||
// Enable condition:
|
||||
@ -131,9 +140,6 @@ namespace sql
|
||||
// the first %rd_pby_sort_cnt_ of %rd_sort_keys_ is the partition by of window function.
|
||||
int64_t rd_pby_sort_cnt_;
|
||||
|
||||
// for PQ_DISTRIBUTE_WINDOW hint outline
|
||||
common::ObSEArray<WinDistAlgo, 8, common::ModulePageAllocator, true> dist_hint_;
|
||||
|
||||
// for reporting window function adaptive pushdown
|
||||
ObOpPseudoColumnRawExpr *wf_aggr_status_expr_;
|
||||
common::ObSEArray<bool, 8, common::ModulePageAllocator, true> pushdown_info_;
|
||||
|
@ -900,6 +900,26 @@ bool ObOptimizerUtil::same_exprs(const common::ObIArray<ObRawExpr*> &src_exprs,
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerUtil::intersect_exprs(const ObIArray<ObRawExpr *> &first,
|
||||
const ObIArray<ObRawExpr *> &right,
|
||||
const EqualSets &equal_sets,
|
||||
ObIArray<ObRawExpr *> &result)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSEArray<ObRawExpr *, 4> tmp;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < first.count(); ++i) {
|
||||
if (!find_equal_expr(right, first.at(i), equal_sets)) {
|
||||
// do nothing
|
||||
} else if (OB_FAIL(tmp.push_back(first.at(i)))) {
|
||||
LOG_WARN("failed to push back first expr", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && OB_FAIL(result.assign(tmp))) {
|
||||
LOG_WARN("failed to assign expr array", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerUtil::intersect_exprs(const ObIArray<ObRawExpr *> &first,
|
||||
const ObIArray<ObRawExpr *> &right,
|
||||
ObIArray<ObRawExpr *> &result)
|
||||
@ -3603,7 +3623,8 @@ int ObOptimizerUtil::check_need_sort(const ObIArray<OrderItem> &expected_order_i
|
||||
const bool is_at_most_one_row,
|
||||
bool &need_sort,
|
||||
int64_t &prefix_pos,
|
||||
const int64_t part_cnt)
|
||||
const int64_t part_cnt,
|
||||
const bool check_part_only/* default false */)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSEArray<ObRawExpr*, 6> expected_order_exprs;
|
||||
@ -3622,7 +3643,8 @@ int ObOptimizerUtil::check_need_sort(const ObIArray<OrderItem> &expected_order_i
|
||||
is_at_most_one_row,
|
||||
need_sort,
|
||||
prefix_pos,
|
||||
part_cnt))) {
|
||||
part_cnt,
|
||||
check_part_only))) {
|
||||
LOG_WARN("failed to check need sort", K(ret));
|
||||
} else { /*do nothing*/ }
|
||||
return ret;
|
||||
@ -3638,11 +3660,13 @@ int ObOptimizerUtil::check_need_sort(const ObIArray<ObRawExpr*> &expected_order_
|
||||
const bool is_at_most_one_row,
|
||||
bool &need_sort,
|
||||
int64_t &prefix_pos,
|
||||
const int64_t part_cnt)
|
||||
const int64_t part_cnt,
|
||||
const bool check_part_only/* default false */)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
need_sort = true;
|
||||
if (OB_FAIL(ObOptimizerUtil::check_need_sort(expected_order_exprs,
|
||||
if (!check_part_only &&
|
||||
OB_FAIL(ObOptimizerUtil::check_need_sort(expected_order_exprs,
|
||||
expected_order_directions,
|
||||
input_ordering,
|
||||
fd_item_set,
|
||||
|
@ -184,6 +184,11 @@ public:
|
||||
const uint64_t expr_prefix_count,
|
||||
bool &is_subset);
|
||||
|
||||
static int intersect_exprs(const ObIArray<ObRawExpr *> &first,
|
||||
const ObIArray<ObRawExpr *> &right,
|
||||
const EqualSets &equal_sets,
|
||||
ObIArray<ObRawExpr *> &result);
|
||||
|
||||
static int intersect_exprs(const ObIArray<ObRawExpr *> &first,
|
||||
const ObIArray<ObRawExpr *> &right,
|
||||
ObIArray<ObRawExpr *> &result);
|
||||
@ -706,7 +711,8 @@ public:
|
||||
const bool is_at_most_one_row,
|
||||
bool &need_sort,
|
||||
int64_t &prefix_pos,
|
||||
const int64_t part_cnt);
|
||||
const int64_t part_cnt,
|
||||
const bool check_part_only = false);
|
||||
|
||||
static int check_need_sort(const ObIArray<OrderItem> &expected_order_items,
|
||||
const ObIArray<OrderItem> &input_ordering,
|
||||
@ -739,7 +745,8 @@ public:
|
||||
const bool is_at_most_one_row,
|
||||
bool &need_sort,
|
||||
int64_t &prefix_pos,
|
||||
const int64_t part_cnt);
|
||||
const int64_t part_cnt,
|
||||
const bool check_part_only = false);
|
||||
|
||||
static int decide_sort_keys_for_merge_style_op(const ObDMLStmt *stmt,
|
||||
const EqualSets &stmt_equal_sets,
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -473,6 +473,75 @@ private:
|
||||
// int process_subplan();
|
||||
int candi_allocate_subplan_filter_for_select_item();
|
||||
|
||||
struct WinFuncOpHelper
|
||||
{
|
||||
WinFuncOpHelper(const ObIArray<ObWinFunRawExpr*> &all_win_func_exprs,
|
||||
const ObWindowDistHint *win_dist_hint,
|
||||
const bool explicit_hint,
|
||||
const ObFdItemSet &fd_item_set,
|
||||
const EqualSets &equal_sets,
|
||||
const ObIArray<ObRawExpr*> &const_exprs,
|
||||
const double card,
|
||||
const bool is_at_most_one_row)
|
||||
: all_win_func_exprs_(all_win_func_exprs),
|
||||
win_dist_hint_(win_dist_hint),
|
||||
explicit_hint_(explicit_hint),
|
||||
fd_item_set_(fd_item_set),
|
||||
equal_sets_(equal_sets),
|
||||
const_exprs_(const_exprs),
|
||||
card_(card),
|
||||
is_at_most_one_row_(is_at_most_one_row),
|
||||
win_dist_method_(WinDistAlgo::WIN_DIST_INVALID),
|
||||
force_normal_sort_(false),
|
||||
force_hash_sort_(false),
|
||||
force_no_pushdown_(false),
|
||||
force_pushdown_(false),
|
||||
part_cnt_(false),
|
||||
win_op_idx_(false),
|
||||
wf_aggr_status_expr_(NULL)
|
||||
{
|
||||
}
|
||||
virtual ~WinFuncOpHelper() {}
|
||||
|
||||
// is const during allocate multi window function op
|
||||
const ObIArray<ObWinFunRawExpr*> &all_win_func_exprs_; // original ordered window functions from stmt
|
||||
const ObWindowDistHint *win_dist_hint_;
|
||||
const bool explicit_hint_;
|
||||
const ObFdItemSet &fd_item_set_;
|
||||
const EqualSets &equal_sets_;
|
||||
const ObIArray<ObRawExpr*> &const_exprs_;
|
||||
const double card_;
|
||||
const bool is_at_most_one_row_;
|
||||
|
||||
// attribute for single window function op
|
||||
WinDistAlgo win_dist_method_;
|
||||
bool force_normal_sort_;
|
||||
bool force_hash_sort_;
|
||||
bool force_no_pushdown_; // hint force pushdown, at least pushdown one window function
|
||||
bool force_pushdown_; // hint force pushdown, at least pushdown one window function
|
||||
|
||||
int64_t part_cnt_;
|
||||
int64_t win_op_idx_;
|
||||
ObOpPseudoColumnRawExpr *wf_aggr_status_expr_;
|
||||
ObArray<ObRawExpr*> partition_exprs_;
|
||||
ObArray<ObWinFunRawExpr*> ordered_win_func_exprs_;
|
||||
ObArray<double> sort_key_ndvs_;
|
||||
ObArray<std::pair<int64_t, int64_t>> pby_oby_prefixes_;
|
||||
ObArray<OrderItem> sort_keys_;
|
||||
|
||||
TO_STRING_KV(K_(win_dist_method),
|
||||
K_(win_op_idx),
|
||||
K_(force_normal_sort),
|
||||
K_(force_hash_sort),
|
||||
K_(force_no_pushdown),
|
||||
K_(force_pushdown),
|
||||
K_(partition_exprs),
|
||||
K_(part_cnt),
|
||||
K_(ordered_win_func_exprs),
|
||||
K_(win_dist_hint),
|
||||
K_(explicit_hint));
|
||||
};
|
||||
|
||||
private:
|
||||
int decide_sort_keys_for_runion(const common::ObIArray<OrderItem> &order_items,
|
||||
common::ObIArray<OrderItem> &new_order_items);
|
||||
@ -483,81 +552,171 @@ private:
|
||||
const bool can_ignore_merge_plan);
|
||||
|
||||
int candi_allocate_window_function();
|
||||
int candi_allocate_window_function_with_hint(const ObIArray<ObWinFunRawExpr*> &win_func_exprs,
|
||||
common::ObIArray<CandidatePlan> &total_plans);
|
||||
int candi_allocate_window_function(const ObIArray<ObWinFunRawExpr*> &win_func_exprs,
|
||||
ObIArray<CandidatePlan> &total_plans);
|
||||
int create_one_window_function(CandidatePlan &candidate_plan,
|
||||
const WinFuncOpHelper &win_func_helper,
|
||||
ObIArray<CandidatePlan> &all_plans);
|
||||
int create_none_dist_win_func(ObLogicalOperator *top,
|
||||
const WinFuncOpHelper &win_func_helper,
|
||||
const int64_t need_sort,
|
||||
const int64_t prefix_pos,
|
||||
const int64_t part_cnt,
|
||||
ObIArray<CandidatePlan> &all_plans);
|
||||
int create_range_list_dist_win_func(ObLogicalOperator *top,
|
||||
const WinFuncOpHelper &win_func_helper,
|
||||
const int64_t part_cnt,
|
||||
ObIArray<CandidatePlan> &all_plans);
|
||||
int get_range_dist_keys(const WinFuncOpHelper &win_func_helper,
|
||||
const ObWinFunRawExpr *win_func,
|
||||
ObIArray<OrderItem> &range_dist_keys,
|
||||
int64_t &pby_prefix);
|
||||
int get_range_list_win_func_exchange_info(const WinDistAlgo dist_method,
|
||||
const ObIArray<OrderItem> &range_dist_keys,
|
||||
ObExchangeInfo &exch_info,
|
||||
ObRawExpr *&random_expr);
|
||||
int set_exchange_random_expr(ObLogicalOperator *top, ObRawExpr *random_expr);
|
||||
|
||||
int generate_window_functions_plan(const ObIArray<ObWinFunRawExpr*> &winfunc_exprs,
|
||||
ObOpPseudoColumnRawExpr *wf_aggr_status_expr,
|
||||
common::ObIArray<CandidatePlan> &total_plans,
|
||||
int create_hash_dist_win_func(ObLogicalOperator *top,
|
||||
const WinFuncOpHelper &win_func_helper,
|
||||
const int64_t need_sort,
|
||||
const int64_t prefix_pos,
|
||||
const int64_t part_cnt,
|
||||
ObIArray<CandidatePlan> &all_plans);
|
||||
int create_normal_hash_dist_win_func(ObLogicalOperator *&top,
|
||||
const ObIArray<ObWinFunRawExpr*> &win_func_exprs,
|
||||
const ObIArray<ObRawExpr*> &partition_exprs,
|
||||
const ObIArray<OrderItem> &sort_keys,
|
||||
const int64_t need_sort,
|
||||
const int64_t prefix_pos,
|
||||
OrderItem *hash_sortkey);
|
||||
int create_pushdown_hash_dist_win_func(ObLogicalOperator *&top,
|
||||
const ObIArray<ObWinFunRawExpr*> &win_func_exprs,
|
||||
const ObIArray<OrderItem> &sort_keys,
|
||||
const ObIArray<bool> &pushdown_info,
|
||||
ObOpPseudoColumnRawExpr *wf_aggr_status_expr,
|
||||
const int64_t need_sort,
|
||||
const int64_t prefix_pos,
|
||||
OrderItem *hash_sortkey);
|
||||
int check_is_win_func_hint_valid(const ObIArray<ObWinFunRawExpr*> &all_win_exprs,
|
||||
const ObWindowDistHint *hint,
|
||||
bool &is_valid);
|
||||
int init_win_func_helper_with_hint(const ObIArray<CandidatePlan> &candi_plans,
|
||||
ObIArray<ObWinFunRawExpr*> &remaining_exprs,
|
||||
WinFuncOpHelper &win_func_helper,
|
||||
bool &is_valid);
|
||||
int calc_win_func_helper_with_hint(const ObLogicalOperator *op,
|
||||
WinFuncOpHelper &win_func_helper,
|
||||
bool &is_valid);
|
||||
int check_win_dist_method_valid(const WinFuncOpHelper &win_func_helper,
|
||||
bool &is_valid);
|
||||
int generate_window_functions_plan(WinFuncOpHelper &win_func_helper,
|
||||
ObIArray<ObOpPseudoColumnRawExpr*> &status_exprs,
|
||||
ObIArray<CandidatePlan> &total_plans,
|
||||
CandidatePlan &orig_candidate_plan);
|
||||
|
||||
int get_next_group_window_exprs(const ObLogicalOperator *top,
|
||||
int check_win_func_need_sort(const ObLogicalOperator &top,
|
||||
const WinFuncOpHelper &win_func_helper,
|
||||
bool &need_sort,
|
||||
int64_t &prefix_pos,
|
||||
int64_t &part_cnt);
|
||||
int prepare_next_group_win_funcs(const bool distributed,
|
||||
const ObIArray<OrderItem> &op_ordering,
|
||||
const int64_t dop,
|
||||
WinFuncOpHelper &win_func_helper,
|
||||
ObIArray<ObWinFunRawExpr*> &remaining_exprs,
|
||||
ObIArray<ObWinFunRawExpr*> &ordered_win_func_exprs,
|
||||
ObIArray<std::pair<int64_t, int64_t>> &pby_oby_prefixes,
|
||||
ObIArray<int64_t> &split,
|
||||
ObIArray<WinDistAlgo> &methods);
|
||||
int init_win_func_helper(const ObIArray<ObWinFunRawExpr*> &ordered_win_func_exprs,
|
||||
const ObIArray<std::pair<int64_t, int64_t>> &pby_oby_prefixes,
|
||||
const ObIArray<int64_t> &split,
|
||||
const ObIArray<WinDistAlgo> &methods,
|
||||
const int64_t splict_idx,
|
||||
ObIArray<ObOpPseudoColumnRawExpr*> &status_exprs,
|
||||
WinFuncOpHelper &win_func_helper);
|
||||
int get_next_group_window_exprs(const ObIArray<OrderItem> &op_ordering,
|
||||
WinFuncOpHelper &win_func_helper,
|
||||
ObIArray<ObWinFunRawExpr*> &remaining_exprs,
|
||||
ObIArray<OrderItem> ¤t_sort_keys,
|
||||
ObIArray<ObWinFunRawExpr*> ¤t_exprs,
|
||||
ObIArray<OrderItem> &next_sort_keys);
|
||||
ObIArray<ObWinFunRawExpr*> ¤t_exprs);
|
||||
int gen_win_func_sort_keys(const ObIArray<OrderItem> &input_ordering,
|
||||
WinFuncOpHelper &win_func_helper,
|
||||
bool &is_valid);
|
||||
int classify_window_exprs(const WinFuncOpHelper &win_func_helper,
|
||||
const ObIArray<OrderItem> &input_ordering,
|
||||
const ObIArray<ObWinFunRawExpr*> &remaining_exprs,
|
||||
ObIArray<ObWinFunRawExpr*> &no_need_sort_exprs,
|
||||
ObIArray<ObWinFunRawExpr*> &no_need_order_exprs,
|
||||
ObIArray<ObWinFunRawExpr*> &rest_win_func_exprs,
|
||||
ObIArray<OrderItem> &best_sort_keys,
|
||||
ObIArray<OrderItem> &possible_sort_keys);
|
||||
|
||||
int prepare_for_split_winfuncs(
|
||||
const ObLogicalOperator *top,
|
||||
const common::ObIArray<ObWinFunRawExpr *> &winfunc_exprs,
|
||||
const ObIArray<OrderItem> &sort_keys,
|
||||
ObIArray<double> &sort_key_ndvs,
|
||||
ObIArray<std::pair<int64_t, int64_t>> &pby_oby_prefixes);
|
||||
int calc_ndvs_and_pby_oby_prefix(const ObIArray<ObWinFunRawExpr*> &win_func_exprs,
|
||||
WinFuncOpHelper &win_func_helper,
|
||||
ObIArray<std::pair<int64_t, int64_t>> &pby_oby_prefixes);
|
||||
|
||||
int check_winfunc_pushdown(
|
||||
const ObLogicalOperator *top, const common::ObIArray<ObWinFunRawExpr *> &winfunc_exprs,
|
||||
const WinDistAlgo method, const ObIArray<double> &sort_key_ndvs,
|
||||
const ObIArray<std::pair<int64_t, int64_t>> &pby_oby_prefixes,
|
||||
bool &is_pushdown, ObIArray<bool> &pushdown_info);
|
||||
int check_win_func_pushdown(const int64_t dop,
|
||||
const WinFuncOpHelper &win_func_helper,
|
||||
ObIArray<bool> &pushdown_info);
|
||||
|
||||
// Split adjusted window function (`adjusted_winfunc_exprs()` called) into groups such that each
|
||||
// Split adjusted window function (`sort_win_func_exprs()` called) into groups such that each
|
||||
// group hash same distribute method.
|
||||
//
|
||||
// @param top
|
||||
// @param winfunc_exprs window functions to split, must adjusted by `adjusted_winfunc_exprs()`
|
||||
// @param remaining_exprs remaining window functions
|
||||
// @param sort_keys
|
||||
// @param stmt_func_idx window function operator of current stmt
|
||||
// @param distributed
|
||||
// @param win_func_exprs window functions to split, must adjusted by `sort_win_func_exprs()`
|
||||
// @param sort_key_ndvs
|
||||
// @param pby_oby_prefixes
|
||||
// @param[out] split split result which stores the %window_exprs array end positions
|
||||
// @param[out] methods window distribute method for each split array
|
||||
// @param[out] pushdown_supported_array to record if each split array can support pushdown
|
||||
int split_winfuncs_by_dist_method(const ObLogicalOperator *top,
|
||||
const common::ObIArray<ObWinFunRawExpr *> &winfunc_exprs,
|
||||
const common::ObIArray<ObWinFunRawExpr *> &remaining_exprs,
|
||||
const int64_t stmt_func_idx,
|
||||
int split_win_funcs_by_dist_method(const bool distributed,
|
||||
const common::ObIArray<ObWinFunRawExpr *> &win_func_exprs,
|
||||
const ObIArray<double> &sort_key_ndvs,
|
||||
const ObIArray<std::pair<int64_t, int64_t>> &pby_oby_prefixes,
|
||||
const int64_t dop,
|
||||
const double card,
|
||||
const ObWindowDistHint *hint,
|
||||
const int64_t win_op_idx,
|
||||
common::ObIArray<int64_t> &split,
|
||||
common::ObIArray<WinDistAlgo> &methods,
|
||||
bool &has_non_parallel_wf);
|
||||
common::ObIArray<WinDistAlgo> &methods);
|
||||
|
||||
// Generate sort keys for window function
|
||||
// @param[out] order_item generated sort keys
|
||||
// @param[out] pby_prefix set the prefix count of %order_item for partition by exprs if not NULL
|
||||
// @return
|
||||
int get_sort_keys_for_window_function(const ObLogicalOperator *top,
|
||||
int get_sort_keys_for_window_function(const ObFdItemSet &fd_item_set,
|
||||
const EqualSets &equal_sets,
|
||||
const ObIArray<ObRawExpr*> &const_exprs,
|
||||
const ObWinFunRawExpr *win_expr,
|
||||
const ObIArray<OrderItem> &ordering,
|
||||
const ObIArray<ObWinFunRawExpr*> &winfunc_exprs,
|
||||
ObIArray<OrderItem> &order_items,
|
||||
int64_t *pby_prefix = NULL);
|
||||
|
||||
int get_win_func_pby_oby_sort_prefix(const ObFdItemSet &fd_item_set,
|
||||
const EqualSets &equal_sets,
|
||||
const ObIArray<ObRawExpr*> &const_exprs,
|
||||
const ObWinFunRawExpr *win_expr,
|
||||
const ObIArray<OrderItem> &ordering,
|
||||
int64_t &pby_prefix,
|
||||
int64_t &pby_oby_prefix);
|
||||
|
||||
// Get the PBY, PBY + OBY prefix count of %ordering
|
||||
// e.g.:
|
||||
// sum(v) over (partition by a, b order by b, c)
|
||||
// %ordering should start with [a, b, c] or [b, a, c]
|
||||
// %pby_prefix will be 2
|
||||
// %pby_oby_prefix will b 3
|
||||
int get_winfunc_pby_oby_sort_prefix(const ObLogicalOperator *top,
|
||||
int get_winfunc_pby_oby_sort_prefix(const ObFdItemSet &fd_item_set,
|
||||
const EqualSets &equal_sets,
|
||||
const ObIArray<ObRawExpr*> &const_exprs,
|
||||
const ObWinFunRawExpr *win_expr,
|
||||
const ObIArray<OrderItem> &ordering,
|
||||
int64_t &pby_prefix,
|
||||
int64_t &pby_oby_prefix);
|
||||
|
||||
int get_partition_count(const ObSEArray<std::pair<int64_t, int64_t>, 8> pby_oby_prefixes,
|
||||
const int64_t start,
|
||||
const int64_t end,
|
||||
const ObIArray<ObRawExpr*> &partition_exprs,
|
||||
const int64_t prefix_pos,
|
||||
int64_t &part_cnt);
|
||||
int calc_partition_count(WinFuncOpHelper &win_func_helper);
|
||||
|
||||
/**
|
||||
* @brief set_default_sort_directions
|
||||
@ -588,14 +747,17 @@ private:
|
||||
const ObIArray<ObWinFunRawExpr*> &adjusted_winfunc_exprs,
|
||||
const ObIArray<OrderItem> &sort_keys,
|
||||
const ObIArray<ObRawExpr*> &partition_exprs,
|
||||
const int64_t part_cnt,
|
||||
const OrderItem &hash_sortkey,
|
||||
const bool is_pushdown,
|
||||
ObOpPseudoColumnRawExpr *wf_aggr_status_expr,
|
||||
const ObIArray<bool> &pushdown_info);
|
||||
|
||||
int adjust_window_functions(const ObLogicalOperator *top,
|
||||
const ObIArray<ObWinFunRawExpr *> &winfunc_exprs,
|
||||
ObIArray<ObWinFunRawExpr *> &adjusted_winfunc_exprs);
|
||||
int sort_window_functions(const ObFdItemSet &fd_item_set,
|
||||
const EqualSets &equal_sets,
|
||||
const ObIArray<ObRawExpr *> &const_exprs,
|
||||
const ObIArray<ObWinFunRawExpr *> &winfunc_exprs,
|
||||
ObIArray<ObWinFunRawExpr *> &adjusted_winfunc_exprs,
|
||||
bool &ordering_changed);
|
||||
|
||||
int match_window_function_parallel(const ObIArray<ObWinFunRawExpr *> &win_exprs,
|
||||
bool &can_parallel);
|
||||
@ -606,23 +768,30 @@ private:
|
||||
int check_wf_pushdown_supported(ObWinFunRawExpr *win_expr, bool &supported);
|
||||
|
||||
int get_pushdown_window_function_exchange_info(const ObIArray<ObWinFunRawExpr *> &win_exprs,
|
||||
const EqualSets & equal_sets,
|
||||
ObOpPseudoColumnRawExpr *wf_aggr_status_expr,
|
||||
ObLogicalOperator *op,
|
||||
ObExchangeInfo &exch_info);
|
||||
|
||||
int get_window_function_partition_exprs(const ObIArray<ObWinFunRawExpr *> &win_exprs,
|
||||
ObIArray<ObRawExpr*> &partition_exprs);
|
||||
int extract_window_function_partition_exprs(WinFuncOpHelper &winfunc_helper);
|
||||
|
||||
int allocate_window_function_as_top(const ObIArray<ObWinFunRawExpr *> &win_exprs,
|
||||
int allocate_window_function_as_top(const WinDistAlgo dist_algo,
|
||||
const ObIArray<ObWinFunRawExpr *> &win_exprs,
|
||||
const bool match_parallel,
|
||||
const bool is_partition_wise,
|
||||
const bool use_hash_sort,
|
||||
const ObIArray<OrderItem> &sort_keys,
|
||||
ObLogicalOperator *&top);
|
||||
int allocate_window_function_as_top(const WinDistAlgo dist_algo,
|
||||
const ObIArray<ObWinFunRawExpr *> &win_exprs,
|
||||
const bool match_parallel,
|
||||
const bool is_partition_wise,
|
||||
const bool use_hash_sort,
|
||||
const int32_t role_type,
|
||||
const ObIArray<OrderItem> &sort_keys,
|
||||
const int64_t range_dist_keys_cnt,
|
||||
const int64_t range_dist_pby_prefix,
|
||||
ObLogicalOperator *&top,
|
||||
ObOpPseudoColumnRawExpr *wf_aggr_status_expr,
|
||||
const ObIArray<bool> &pushdown_info);
|
||||
ObOpPseudoColumnRawExpr *wf_aggr_status_expr = NULL,
|
||||
const ObIArray<bool> *pushdown_info = NULL);
|
||||
|
||||
int candi_allocate_late_materialization();
|
||||
|
||||
|
@ -1103,6 +1103,7 @@ Timestamp{whitespace}?\"[^\"]*\" {
|
||||
<hint>OPT_PARAM { return OPT_PARAM; }
|
||||
<hint>OB_DDL_SCHEMA_VERSION { return OB_DDL_SCHEMA_VERSION; }
|
||||
<hint>DYNAMIC_SAMPLING { return DYNAMIC_SAMPLING; }
|
||||
<hint>PUSHDOWN { return PUSHDOWN; }
|
||||
<hint>{identifier} {
|
||||
if (!(IS_FAST_PARAMETERIZE)) {
|
||||
check_value(yylval);
|
||||
|
@ -182,6 +182,7 @@ USE_HASH_DISTINCT NO_USE_HASH_DISTINCT
|
||||
DISTINCT_PUSHDOWN NO_DISTINCT_PUSHDOWN
|
||||
USE_HASH_SET NO_USE_HASH_SET
|
||||
USE_DISTRIBUTED_DML NO_USE_DISTRIBUTED_DML
|
||||
PUSHDOWN
|
||||
// direct load data hint
|
||||
DIRECT
|
||||
// hint related to optimizer statistics
|
||||
@ -384,6 +385,8 @@ END_P SET_VAR DELIMITER
|
||||
%type <node> case_expr func_expr in_expr sub_query_flag
|
||||
%type <node> case_arg when_clause_list when_clause case_default
|
||||
%type <node> window_function opt_partition_by generalized_window_clause win_rows_or_range win_preceding_or_following win_interval win_bounding win_window opt_win_window win_fun_lead_lag_params respect_or_ignore opt_respect_or_ignore_nulls win_fun_first_last_params first_or_last opt_from_first_or_last new_generalized_window_clause new_generalized_window_clause_with_blanket opt_named_windows named_windows named_window
|
||||
%type <node> win_dist_list win_dist_desc
|
||||
%type <ival> opt_hash_sort_and_pushdown
|
||||
%type <node> update_asgn_list update_asgn_factor
|
||||
%type <node> update_basic_stmt delete_basic_stmt
|
||||
%type <node> table_element_list table_element column_definition column_definition_ref column_definition_list column_name_list
|
||||
@ -9427,11 +9430,11 @@ INDEX_HINT '(' qb_name_option relation_factor_in_hint NAME_OB ')'
|
||||
{
|
||||
malloc_non_terminal_node($$, result->malloc_pool_, T_PQ_MAP, 2, $3, $4);
|
||||
}
|
||||
| PQ_DISTRIBUTE_WINDOW '('qb_name_option opt_comma distribute_method_list')'
|
||||
| PQ_DISTRIBUTE_WINDOW '('qb_name_option opt_comma win_dist_list')'
|
||||
{
|
||||
(void) $4;
|
||||
ParseNode *method_list = NULL;
|
||||
merge_nodes(method_list, result, T_DISTRIBUTE_METHOD_LIST, $5);
|
||||
merge_nodes(method_list, result, T_METHOD_OPT_LIST, $5);
|
||||
malloc_non_terminal_node($$, result->malloc_pool_, T_PQ_DISTRIBUTE_WINDOW, 2, $3, method_list);
|
||||
}
|
||||
| PQ_SET '(' pq_set_hint_desc ')'
|
||||
@ -9484,6 +9487,56 @@ INDEX_HINT '(' qb_name_option relation_factor_in_hint NAME_OB ')'
|
||||
}
|
||||
;
|
||||
|
||||
win_dist_list:
|
||||
win_dist_desc
|
||||
{
|
||||
$$ = $1;
|
||||
}
|
||||
| win_dist_list opt_comma win_dist_desc
|
||||
{
|
||||
(void)($2); /* unused */
|
||||
malloc_non_terminal_node($$, result->malloc_pool_, T_LINK_NODE, 2, $1, $3);
|
||||
}
|
||||
;
|
||||
|
||||
win_dist_desc:
|
||||
'(' intnum_list ')' distribute_method opt_hash_sort_and_pushdown
|
||||
{
|
||||
ParseNode *win_func_idxs = NULL;
|
||||
merge_nodes(win_func_idxs, result, T_WIN_FUNC_IDX_LIST, $2);
|
||||
malloc_non_terminal_node($$, result->malloc_pool_, T_METHOD_OPT, 2, win_func_idxs, $4);
|
||||
$4->value_ = $5[0];
|
||||
}
|
||||
| distribute_method opt_hash_sort_and_pushdown
|
||||
{
|
||||
malloc_non_terminal_node($$, result->malloc_pool_, T_METHOD_OPT, 2, NULL, $1);
|
||||
$1->value_ = $2[0];
|
||||
}
|
||||
;
|
||||
|
||||
opt_hash_sort_and_pushdown:
|
||||
/*empty*/
|
||||
{
|
||||
$$[0] = 0;
|
||||
}
|
||||
| PARTITION_SORT
|
||||
{
|
||||
$$[0] = 1;
|
||||
}
|
||||
| PUSHDOWN
|
||||
{
|
||||
$$[0] = 2;
|
||||
}
|
||||
| PARTITION_SORT PUSHDOWN
|
||||
{
|
||||
$$[0] = 3;
|
||||
}
|
||||
| PUSHDOWN PARTITION_SORT
|
||||
{
|
||||
$$[0] = 3;
|
||||
}
|
||||
;
|
||||
|
||||
pq_set_hint_desc:
|
||||
'@' qb_name_string qb_name_string distribute_method_list
|
||||
{
|
||||
|
@ -13696,44 +13696,101 @@ int ObDMLResolver::resolve_pq_distribute_window_hint(const ParseNode &node,
|
||||
int ret = OB_SUCCESS;
|
||||
hint = NULL;
|
||||
ObWindowDistHint *win_dist = NULL;
|
||||
ObSEArray<ObWindowDistHint::WinDistOption, 2> win_dist_options;
|
||||
ObString qb_name;
|
||||
const ParseNode *dist_methods_node = NULL;
|
||||
CK(T_PQ_DISTRIBUTE_WINDOW == node.type_ && 2 == node.num_child_);
|
||||
if (OB_ISNULL(dist_methods_node = node.children_[1])
|
||||
|| OB_UNLIKELY(T_DISTRIBUTE_METHOD_LIST != dist_methods_node->type_)) {
|
||||
if (OB_UNLIKELY(T_PQ_DISTRIBUTE_WINDOW != node.type_ || 2 != node.num_child_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected pq_distribute_window hint node", K(ret), K(dist_methods_node));
|
||||
LOG_WARN("unexpected pq_distribute_window hint node", K(ret), K(node.type_), K(node.num_child_));
|
||||
} else if (OB_FAIL(resolve_win_dist_options(node.children_[1], win_dist_options))) {
|
||||
LOG_WARN("failed to resolve win dist options", K(ret));
|
||||
} else if (win_dist_options.empty()) {
|
||||
/* do nothing */
|
||||
} else if (OB_FAIL(resolve_qb_name_node(node.children_[0], qb_name))) {
|
||||
LOG_WARN("failed to resolve query block name", K(ret));
|
||||
} else if (OB_FAIL(ObQueryHint::create_hint(allocator_, T_PQ_DISTRIBUTE_WINDOW, win_dist))) {
|
||||
LOG_WARN("failed to create hint", K(ret));
|
||||
} else if (OB_FAIL(win_dist->set_win_dist_options(win_dist_options))) {
|
||||
LOG_WARN("failed to set win dist options", K(ret));
|
||||
} else {
|
||||
win_dist->set_qb_name(qb_name);
|
||||
hint = win_dist;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDMLResolver::resolve_win_dist_options(const ParseNode *option_list,
|
||||
ObIArray<ObWindowDistHint::WinDistOption> &win_dist_options)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
win_dist_options.reuse();
|
||||
if (OB_ISNULL(option_list) || OB_UNLIKELY(T_METHOD_OPT_LIST != option_list->type_)) {
|
||||
//ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected win func option list in hint node", K(ret), K(option_list));
|
||||
} else {
|
||||
ObWindowDistHint::WinDistOption dist_option;
|
||||
bool is_valid = true;
|
||||
ObSEArray<WinDistAlgo, 2> dist_methods;
|
||||
WinDistAlgo method = WinDistAlgo::NONE;
|
||||
for (int64_t i = 0; is_valid && OB_SUCC(ret) && i < dist_methods_node->num_child_; ++i) {
|
||||
if (OB_ISNULL(dist_methods_node->children_[i])) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null", K(ret), K(i));
|
||||
} else {
|
||||
switch (dist_methods_node->children_[i]->type_) {
|
||||
case T_DISTRIBUTE_NONE: method = WinDistAlgo::NONE; break;
|
||||
case T_DISTRIBUTE_HASH: method = WinDistAlgo::HASH; break;
|
||||
case T_DISTRIBUTE_RANGE: method = WinDistAlgo::RANGE; break;
|
||||
case T_DISTRIBUTE_LIST: method = WinDistAlgo::LIST; break;
|
||||
default: is_valid = false; break;
|
||||
}
|
||||
if (is_valid && OB_FAIL(dist_methods.push_back(method))) {
|
||||
for (int64_t i = 0; is_valid && OB_SUCC(ret) && i < option_list->num_child_; ++i) {
|
||||
if (OB_FAIL(resolve_win_dist_option(option_list->children_[i], dist_option, is_valid))) {
|
||||
LOG_WARN("failed to resolve win dist option", K(ret), K(i));
|
||||
} else if (!is_valid) {
|
||||
win_dist_options.reuse();
|
||||
} else if (OB_FAIL(win_dist_options.push_back(dist_option))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObDMLResolver::resolve_win_dist_option(const ParseNode *option,
|
||||
ObWindowDistHint::WinDistOption &dist_option,
|
||||
bool &is_valid)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
is_valid = false;
|
||||
dist_option.reset();
|
||||
const ParseNode *win_idxs = NULL;
|
||||
const ParseNode *dist_method = NULL;
|
||||
if (OB_ISNULL(option) || OB_UNLIKELY(T_METHOD_OPT != option->type_ || 2 != option->num_child_)
|
||||
|| OB_ISNULL(dist_method = option->children_[1])
|
||||
|| OB_UNLIKELY(NULL != (win_idxs = option->children_[0])
|
||||
&& T_WIN_FUNC_IDX_LIST != win_idxs->type_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected win func option in hint node", K(ret), K(option), K(win_idxs), K(dist_method));
|
||||
} else {
|
||||
is_valid = true;
|
||||
const ParseNode *idx_node = NULL;
|
||||
const int64_t hash_sort_flag = 1;
|
||||
const int64_t push_down_flag = 1 << 1;
|
||||
switch (dist_method->type_) {
|
||||
case T_DISTRIBUTE_NONE: {
|
||||
dist_option.algo_ = WinDistAlgo::WIN_DIST_NONE;
|
||||
dist_option.use_hash_sort_ = dist_method->value_ & hash_sort_flag;
|
||||
break;
|
||||
}
|
||||
case T_DISTRIBUTE_HASH: {
|
||||
dist_option.algo_ = WinDistAlgo::WIN_DIST_HASH;
|
||||
dist_option.use_hash_sort_ = dist_method->value_ & hash_sort_flag;
|
||||
dist_option.is_push_down_ = dist_method->value_ & push_down_flag;
|
||||
break;
|
||||
}
|
||||
case T_DISTRIBUTE_RANGE: dist_option.algo_ = WinDistAlgo::WIN_DIST_RANGE; break;
|
||||
case T_DISTRIBUTE_LIST: dist_option.algo_ = WinDistAlgo::WIN_DIST_LIST; break;
|
||||
default: is_valid = false; break;
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret) && NULL != win_idxs) {
|
||||
for (int64_t i = 0; is_valid && OB_SUCC(ret) && i < win_idxs->num_child_; ++i) {
|
||||
if (OB_ISNULL(idx_node = win_idxs->children_[i])) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null", K(ret), K(i));
|
||||
} else if (OB_FAIL(dist_option.win_func_idxs_.push_back(idx_node->value_))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret) || !is_valid) {
|
||||
} else if (OB_FAIL(ObQueryHint::create_hint(allocator_, node.type_, win_dist))) {
|
||||
LOG_WARN("failed to create hint", K(ret));
|
||||
} else if (OB_FAIL(win_dist->get_algos().assign(dist_methods))) {
|
||||
LOG_WARN("failed to assign dist methods", K(ret));
|
||||
} else {
|
||||
win_dist->set_qb_name(qb_name);
|
||||
hint = win_dist;
|
||||
if (OB_SUCC(ret)) {
|
||||
is_valid &= dist_option.is_valid();
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
|
@ -877,6 +877,11 @@ private:
|
||||
ObIArray<ObItemType> &dist_methods,
|
||||
bool &is_valid);
|
||||
int resolve_pq_distribute_window_hint(const ParseNode &hint_node, ObOptHint *&opt_hint);
|
||||
int resolve_win_dist_options(const ParseNode *option_list,
|
||||
ObIArray<ObWindowDistHint::WinDistOption> &win_dist_options);
|
||||
int resolve_win_dist_option(const ParseNode *option,
|
||||
ObWindowDistHint::WinDistOption &dist_option,
|
||||
bool &is_valid);
|
||||
int resolve_table_dynamic_sampling_hint(const ParseNode &hint_node, ObOptHint *&opt_hint);
|
||||
//////////end of functions for sql hint/////////////
|
||||
|
||||
|
@ -952,7 +952,7 @@ const char* ObHint::get_hint_name(ObItemType type, bool is_enable_hint /* defaul
|
||||
case T_USE_HASH_AGGREGATE: return is_enable_hint ? "USE_HASH_AGGREGATION"
|
||||
: "NO_USE_HASH_AGGREGATION";
|
||||
case T_TABLE_PARALLEL: return "PARALLEL";
|
||||
case T_PQ_DISTRIBUTE_WINDOW: return "PQ_DISIRIBUTE_WINDOW";
|
||||
case T_PQ_DISTRIBUTE_WINDOW: return "PQ_DISTRIBUTE_WINDOW";
|
||||
case T_GBY_PUSHDOWN: return is_enable_hint ? "GBY_PUSHDOWN" : "NO_GBY_PUSHDOWN";
|
||||
case T_USE_HASH_DISTINCT: return is_enable_hint ? "USE_HASH_DISTINCT" : "NO_USE_HASH_DISTINCT";
|
||||
case T_DISTINCT_PUSHDOWN: return is_enable_hint ? "DISTINCT_PUSHDOWN" : "NO_DISTINCT_PUSHDOWN";
|
||||
@ -2561,10 +2561,10 @@ void ObTableInHint::set_table(const TableItem& table)
|
||||
const char *ObWindowDistHint::get_dist_algo_str(WinDistAlgo dist_algo)
|
||||
{
|
||||
switch (dist_algo) {
|
||||
case WinDistAlgo::NONE: return "NONE";
|
||||
case WinDistAlgo::HASH: return "HASH";
|
||||
case WinDistAlgo::RANGE: return "RANGE";
|
||||
case WinDistAlgo::LIST: return "LIST";
|
||||
case WinDistAlgo::WIN_DIST_NONE: return "NONE";
|
||||
case WinDistAlgo::WIN_DIST_HASH: return "HASH";
|
||||
case WinDistAlgo::WIN_DIST_RANGE: return "RANGE";
|
||||
case WinDistAlgo::WIN_DIST_LIST: return "LIST";
|
||||
default: return NULL;
|
||||
}
|
||||
return NULL;
|
||||
@ -2573,17 +2573,130 @@ const char *ObWindowDistHint::get_dist_algo_str(WinDistAlgo dist_algo)
|
||||
int ObWindowDistHint::print_hint_desc(PlanText &plan_text) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
char *buf = plan_text.buf_;
|
||||
int64_t &buf_len = plan_text.buf_len_;
|
||||
int64_t &pos = plan_text.pos_;
|
||||
FOREACH_CNT_X(v, algos_, OB_SUCC(ret)) {
|
||||
if (OB_FAIL(BUF_PRINTF(" %s", get_dist_algo_str(*v)))) {
|
||||
LOG_WARN("print failed", K(ret));
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < win_dist_options_.count(); ++i) {
|
||||
if (OB_FAIL(win_dist_options_.at(i).print_win_dist_option(plan_text))) {
|
||||
LOG_WARN("failed to print win dist option", K(ret), K(i));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObWindowDistHint::add_win_dist_option(const ObIArray<ObWinFunRawExpr*> &all_win_funcs,
|
||||
const ObIArray<ObWinFunRawExpr*> &cur_win_funcs,
|
||||
const WinDistAlgo algo,
|
||||
const bool is_push_down,
|
||||
const bool use_hash_sort)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSEArray<int64_t, 4> win_func_idxs;
|
||||
int64_t idx = OB_INVALID_INDEX;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < cur_win_funcs.count(); ++i) {
|
||||
if (OB_UNLIKELY(!ObOptimizerUtil::find_item(all_win_funcs, cur_win_funcs.at(i), &idx))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("failed to find item", K(ret), K(all_win_funcs), K(cur_win_funcs));
|
||||
} else if (OB_FAIL(win_func_idxs.push_back(idx))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && add_win_dist_option(win_func_idxs, algo, is_push_down, use_hash_sort)) {
|
||||
LOG_WARN("failed to add win dist option", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObWindowDistHint::add_win_dist_option(const ObIArray<int64_t> &win_func_idxs,
|
||||
const WinDistAlgo algo,
|
||||
const bool is_push_down,
|
||||
const bool use_hash_sort)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
int64_t idx = win_dist_options_.count();
|
||||
if (OB_FAIL(win_dist_options_.prepare_allocate(win_dist_options_.count() + 1))) {
|
||||
LOG_WARN("array prepare allocate failed", K(ret));
|
||||
} else {
|
||||
WinDistOption &win_dist_option = win_dist_options_.at(idx);
|
||||
win_dist_option.algo_ = algo;
|
||||
win_dist_option.is_push_down_ = is_push_down;
|
||||
win_dist_option.use_hash_sort_ = use_hash_sort;
|
||||
if (win_dist_option.win_func_idxs_.assign(win_func_idxs)) {
|
||||
LOG_WARN("failed to add win dist option", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObWindowDistHint::WinDistOption::print_win_dist_option(PlanText &plan_text) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
char *buf = plan_text.buf_;
|
||||
int64_t &buf_len = plan_text.buf_len_;
|
||||
int64_t &pos = plan_text.pos_;
|
||||
if (OB_UNLIKELY(!is_valid())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid WinDistOption", K(ret), K(*this));
|
||||
} else if (win_func_idxs_.empty()) {
|
||||
/* do nothing */
|
||||
} else if (OB_FAIL(BUF_PRINTF(" (%ld", win_func_idxs_.at(0)))) {
|
||||
LOG_WARN("fail to print win func idx", K(ret), K(win_func_idxs_));
|
||||
} else {
|
||||
for (int64_t i = 1; OB_SUCC(ret) && i < win_func_idxs_.count(); ++i) {
|
||||
if (OB_FAIL(BUF_PRINTF(",%ld", win_func_idxs_.at(i)))) {
|
||||
LOG_WARN("fail to print win func idx", K(ret), K(win_func_idxs_));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && OB_FAIL(BUF_PRINTF(")"))) {
|
||||
LOG_WARN("failed to print win func idx", K(ret));
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(BUF_PRINTF(" %s", ObWindowDistHint::get_dist_algo_str(algo_)))) {
|
||||
LOG_WARN("failed to print win func dist algo", K(ret));
|
||||
} else if (use_hash_sort_ && OB_FAIL(BUF_PRINTF(" PARTITION_SORT"))) {
|
||||
LOG_WARN("failed to print win func sort", K(ret));
|
||||
} else if (is_push_down_ && OB_FAIL(BUF_PRINTF(" PUSHDOWN"))) {
|
||||
LOG_WARN("failed to print win func push down", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObWindowDistHint::WinDistOption::is_valid() const
|
||||
{
|
||||
bool bret = true;
|
||||
if (WinDistAlgo::WIN_DIST_INVALID == algo_) {
|
||||
bret = false;
|
||||
} else if (WinDistAlgo::WIN_DIST_HASH != algo_ && is_push_down_) {
|
||||
bret = false;
|
||||
} else if (WinDistAlgo::WIN_DIST_HASH != algo_ && WinDistAlgo::WIN_DIST_NONE != algo_ && use_hash_sort_) {
|
||||
bret = false;
|
||||
} else {
|
||||
for (int64_t i = 0; bret && i < win_func_idxs_.count(); ++i) {
|
||||
bret = win_func_idxs_.at(i) >= 0;
|
||||
}
|
||||
}
|
||||
return bret;
|
||||
}
|
||||
|
||||
int ObWindowDistHint::WinDistOption::assign(const WinDistOption& other)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
algo_ = other.algo_;
|
||||
use_hash_sort_ = other.use_hash_sort_;
|
||||
is_push_down_ = other.is_push_down_;
|
||||
if (OB_FAIL(win_func_idxs_.assign(other.win_func_idxs_))) {
|
||||
LOG_WARN("failed to assign", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObWindowDistHint::WinDistOption::reset()
|
||||
{
|
||||
algo_ = WinDistAlgo::WIN_DIST_INVALID;
|
||||
use_hash_sort_ = false;
|
||||
is_push_down_ = false;
|
||||
win_func_idxs_.reuse();
|
||||
}
|
||||
|
||||
int ObAggHint::assign(const ObAggHint &other)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
@ -944,16 +944,38 @@ public:
|
||||
: ObOptHint(hint_type)
|
||||
{
|
||||
}
|
||||
ObIArray<WinDistAlgo> &get_algos() { return algos_; }
|
||||
const ObIArray<WinDistAlgo> &get_algos() const { return algos_; }
|
||||
struct WinDistOption {
|
||||
WinDistOption() { reset(); }
|
||||
int assign(const WinDistOption& other);
|
||||
void reset();
|
||||
bool is_valid() const;
|
||||
int print_win_dist_option(PlanText &plan_text) const;
|
||||
|
||||
WinDistAlgo algo_;
|
||||
common::ObSEArray<int64_t, 2, common::ModulePageAllocator, true> win_func_idxs_;
|
||||
bool use_hash_sort_; // use hash sort for none/hash dist method
|
||||
bool is_push_down_; // push down window function for hash dist method
|
||||
TO_STRING_KV(K_(algo), K_(win_func_idxs), K_(use_hash_sort), K_(is_push_down));
|
||||
};
|
||||
|
||||
const ObIArray<WinDistOption> &get_win_dist_options() const { return win_dist_options_; }
|
||||
int set_win_dist_options(const ObIArray<WinDistOption> &win_dist_options) { return win_dist_options_.assign(win_dist_options); }
|
||||
int add_win_dist_option(const ObIArray<ObWinFunRawExpr*> &all_win_funcs,
|
||||
const ObIArray<ObWinFunRawExpr*> &cur_win_funcs,
|
||||
const WinDistAlgo algo,
|
||||
const bool is_push_down,
|
||||
const bool use_hash_sort);
|
||||
int add_win_dist_option(const ObIArray<int64_t> &win_func_idxs,
|
||||
const WinDistAlgo algo,
|
||||
const bool is_push_down,
|
||||
const bool use_hash_sort);
|
||||
static const char* get_dist_algo_str(WinDistAlgo dist_algo);
|
||||
|
||||
virtual int print_hint_desc(PlanText &plan_text) const override;
|
||||
|
||||
INHERIT_TO_STRING_KV("hint", ObHint, K_(algos));
|
||||
INHERIT_TO_STRING_KV("hint", ObHint, K_(win_dist_options));
|
||||
private:
|
||||
typedef common::ObSEArray<WinDistAlgo, 3, common::ModulePageAllocator, true> Algos;
|
||||
Algos algos_;
|
||||
common::ObSEArray<WinDistOption, 2, common::ModulePageAllocator, true> win_dist_options_;
|
||||
};
|
||||
|
||||
class ObAggHint : public ObOptHint
|
||||
|
@ -6961,7 +6961,7 @@ int ObRawExprUtils::build_inner_aggr_code_expr(ObRawExprFactory &factory,
|
||||
|
||||
int ObRawExprUtils::build_inner_wf_aggr_status_expr(ObRawExprFactory &factory,
|
||||
const ObSQLSessionInfo &session_info,
|
||||
ObRawExpr *&out)
|
||||
ObOpPseudoColumnRawExpr *&out)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObOpPseudoColumnRawExpr *expr = NULL;
|
||||
|
@ -901,7 +901,7 @@ public:
|
||||
ObRawExpr *&out);
|
||||
static int build_inner_wf_aggr_status_expr(ObRawExprFactory &factory,
|
||||
const ObSQLSessionInfo &session_info,
|
||||
ObRawExpr *&out);
|
||||
ObOpPseudoColumnRawExpr *&out);
|
||||
static int build_pseudo_rollup_id(ObRawExprFactory &factory,
|
||||
const ObSQLSessionInfo &session_info,
|
||||
ObRawExpr *&out);
|
||||
|
Loading…
x
Reference in New Issue
Block a user