patch 4.0

This commit is contained in:
wangzelin.wzl
2022-10-24 10:34:53 +08:00
parent 4ad6e00ec3
commit 93a1074b0c
10533 changed files with 2588271 additions and 2299373 deletions

View File

@ -14,48 +14,93 @@
#define OCEANBASE_SQL_OB_LOG_WINDOW_FUNCTION_H
#include "sql/optimizer/ob_logical_operator.h"
#include "sql/optimizer/ob_log_set.h"
namespace oceanbase {
namespace sql {
class ObLogWindowFunction : public ObLogicalOperator {
public:
ObLogWindowFunction(ObLogPlan& plan) : ObLogicalOperator(plan), is_parallel_(false)
{}
virtual ~ObLogWindowFunction()
{}
virtual int allocate_exchange_post(AllocExchContext* ctx) override;
virtual int copy_without_child(ObLogicalOperator*& out) override;
virtual int print_my_plan_annotation(char* buf, int64_t& buf_len, int64_t& pos, ExplainType type) override;
namespace oceanbase
{
namespace sql
{
class ObLogWindowFunction : public ObLogicalOperator
{
public:
ObLogWindowFunction(ObLogPlan &plan)
: ObLogicalOperator(plan),
single_part_parallel_(false),
range_dist_parallel_(false),
rd_pby_sort_cnt_(0)
{}
virtual ~ObLogWindowFunction() {}
virtual int print_my_plan_annotation(char *buf,
int64_t &buf_len,
int64_t &pos,
ExplainType type);
inline int add_window_expr(ObWinFunRawExpr* win_expr)
{
return win_exprs_.push_back(win_expr);
}
inline ObIArray<ObWinFunRawExpr*>& get_window_exprs()
{
return win_exprs_;
}
virtual int est_cost() override;
virtual int re_est_cost(const ObLogicalOperator* parent, double need_row_count, bool& re_est) override;
virtual int transmit_op_ordering() override;
virtual int allocate_expr_pre(ObAllocExprContext& ctx) override;
virtual int check_output_dep_specific(ObRawExprCheckDep& checker) override;
int is_my_window_expr(const ObRawExpr* expr, bool& is_mine);
virtual bool is_block_op() const override;
virtual uint64_t hash(uint64_t seed) const override;
virtual int allocate_granule_post(AllocGIContext& ctx) override;
virtual int allocate_granule_pre(AllocGIContext& ctx) override;
int get_win_partition_intersect_exprs(ObIArray<ObWinFunRawExpr*>& win_exprs, ObIArray<ObRawExpr*>& win_part_exprs);
virtual int inner_append_not_produced_exprs(ObRawExprUniqueSet& raw_exprs) const override;
bool is_parallel() const
{
return is_parallel_;
}
int match_parallel_condition(bool& can_parallel);
inline int add_window_expr(ObWinFunRawExpr *win_expr)
{ return win_exprs_.push_back(win_expr); }
inline ObIArray<ObWinFunRawExpr *> &get_window_exprs() { return win_exprs_; }
virtual int est_cost() override;
virtual int est_width() override;
virtual int re_est_cost(EstimateCostInfo &param, double &card, double &cost) override;
virtual int get_op_exprs(ObIArray<ObRawExpr*> &all_exprs) override;
virtual int compute_op_ordering() override;
virtual int compute_sharding_info() override;
virtual bool is_block_op() const override;
virtual int allocate_granule_post(AllocGIContext &ctx) override;
virtual int allocate_granule_pre(AllocGIContext &ctx) override;
int get_win_partition_intersect_exprs(ObIArray<ObWinFunRawExpr *> &win_exprs,
ObIArray<ObRawExpr *> &win_part_exprs);
virtual int inner_replace_generated_agg_expr(
const common::ObIArray<std::pair<ObRawExpr *, ObRawExpr*>> &to_replace_exprs) override;
void set_single_part_parallel(bool v) { single_part_parallel_ = v; }
bool is_single_part_parallel() const { return single_part_parallel_; }
void set_ragne_dist_parallel(bool v) { range_dist_parallel_ = v; }
bool is_range_dist_parallel() const { return range_dist_parallel_; }
int get_winfunc_output_exprs(ObIArray<ObRawExpr *> &output_exprs);
int set_rd_sort_keys(const common::ObIArray<OrderItem> &sort_keys)
{
return rd_sort_keys_.assign(sort_keys);
}
const common::ObIArray<OrderItem> &get_rd_sort_keys() const
{
return rd_sort_keys_;
}
private:
ObSEArray<ObWinFunRawExpr*, 4> win_exprs_;
bool is_parallel_;
};
} // namespace sql
} // namespace oceanbase
#endif // OCEANBASE_SQL_OB_LOG_WINDOW_FUNCTION_H
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);
}
virtual int print_outline(planText &plan_text) override;
private:
ObSEArray<ObWinFunRawExpr *, 4> win_exprs_;
// 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:
// 1. Only one partition (no partition by)
// 2. Window is the whole partition
// 3. Only the following functions supported: sum,count,max,min
bool single_part_parallel_;
// Range distribution window function parallel process: data is range distributed,
// the first and last partition of each worker may be partial result. Then the PX COORD
// calculate the `patch` of the first row of first partition and last row of last
// partition, finally the other rows of those partition will be updated (apply the patch).
// Enable condition:
// 1. NDV of partition by is too small for parallelism
// 2. Is cumulative window, window is from first row to current row
// 3. Only the following functions supported: rank,dense_rank,sum,count,min,max
bool range_dist_parallel_;
// sort keys for range distributed parallel.
common::ObSEArray<OrderItem, 8, common::ModulePageAllocator, true> rd_sort_keys_;
// 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_;
};
}
}
#endif // OCEANBASE_SQL_OB_LOG_WINDOW_FUNCTION_H