[FEAT MERGE] support auto dop
This commit is contained in:
@ -282,18 +282,37 @@ struct EstimateCostInfo {
|
||||
EstimateCostInfo()
|
||||
:join_filter_infos_(),
|
||||
need_row_count_(-1), //no need to refine row count
|
||||
need_parallel_(ObGlobalHint::UNSET_PARALLEL), //no need to refine parallel
|
||||
override_(false) {}
|
||||
|
||||
int assign(const EstimateCostInfo& other);
|
||||
void reset() {
|
||||
join_filter_infos_.reuse();
|
||||
need_row_count_ = -1;
|
||||
need_parallel_ = ObGlobalHint::UNSET_PARALLEL;
|
||||
override_ = false;
|
||||
}
|
||||
int assign(const EstimateCostInfo& other) {
|
||||
need_row_count_ = other.need_row_count_;
|
||||
need_parallel_ = other.need_parallel_;
|
||||
override_ = other.override_;
|
||||
return join_filter_infos_.assign(other.join_filter_infos_);
|
||||
}
|
||||
bool need_re_est(int64_t cur_parallel, double cur_row_count) const {
|
||||
return override_ || !join_filter_infos_.empty()
|
||||
|| (ObGlobalHint::UNSET_PARALLEL != need_parallel_ && need_parallel_ != cur_parallel)
|
||||
|| (need_row_count_ >= 0 && need_row_count_ < cur_row_count);
|
||||
}
|
||||
|
||||
TO_STRING_KV(
|
||||
K_(join_filter_infos),
|
||||
K_(need_row_count),
|
||||
K_(need_parallel),
|
||||
K_(override)
|
||||
);
|
||||
|
||||
ObSEArray<JoinFilterInfo, 4> join_filter_infos_;
|
||||
double need_row_count_;
|
||||
int64_t need_parallel_;
|
||||
bool override_;
|
||||
};
|
||||
|
||||
@ -323,7 +342,9 @@ struct EstimateCostInfo {
|
||||
contain_pw_merge_op_(false),
|
||||
contain_match_all_fake_cte_(false),
|
||||
contain_das_op_(false),
|
||||
parallel_(1),
|
||||
parallel_(ObGlobalHint::UNSET_PARALLEL),
|
||||
op_parallel_rule_(OpParallelRule::OP_DOP_RULE_MAX),
|
||||
available_parallel_(ObGlobalHint::DEFAULT_PARALLEL),
|
||||
server_cnt_(1)
|
||||
{ }
|
||||
Path(ObJoinOrder* parent)
|
||||
@ -349,7 +370,9 @@ struct EstimateCostInfo {
|
||||
contain_pw_merge_op_(false),
|
||||
contain_match_all_fake_cte_(false),
|
||||
contain_das_op_(false),
|
||||
parallel_(1),
|
||||
parallel_(ObGlobalHint::UNSET_PARALLEL),
|
||||
op_parallel_rule_(OpParallelRule::OP_DOP_RULE_MAX),
|
||||
available_parallel_(ObGlobalHint::DEFAULT_PARALLEL),
|
||||
server_cnt_(1),
|
||||
server_list_(),
|
||||
is_pipelined_path_(false),
|
||||
@ -449,9 +472,10 @@ struct EstimateCostInfo {
|
||||
}
|
||||
inline bool parallel_more_than_part_cnt() const
|
||||
{
|
||||
return NULL != strong_sharding_ && parallel_ > strong_sharding_->get_part_cnt();
|
||||
return NULL != get_sharding() && parallel_ > get_sharding()->get_part_cnt();
|
||||
}
|
||||
int compute_path_property_from_log_op();
|
||||
int set_parallel_and_server_info_for_match_all();
|
||||
TO_STRING_KV(K_(is_local_order),
|
||||
K_(ordering),
|
||||
K_(interesting_order_info),
|
||||
@ -501,6 +525,8 @@ struct EstimateCostInfo {
|
||||
bool contain_das_op_;
|
||||
// remember the parallel info to get this sharding
|
||||
int64_t parallel_;
|
||||
OpParallelRule op_parallel_rule_;
|
||||
int64_t available_parallel_; // parallel degree used by serial path to enable parallel again
|
||||
int64_t server_cnt_;
|
||||
common::ObSEArray<common::ObAddr, 8, common::ModulePageAllocator, true> server_list_;
|
||||
bool is_pipelined_path_;
|
||||
@ -576,9 +602,39 @@ struct EstimateCostInfo {
|
||||
const ObCostTableScanInfo &get_cost_table_scan_info() const
|
||||
{ return est_cost_info_; }
|
||||
ObCostTableScanInfo &get_cost_table_scan_info() { return est_cost_info_; }
|
||||
int compute_parallel_degree(const int64_t parallel_degree_limit,
|
||||
const int64_t micro_block_threshold,
|
||||
int64_t ¶llel) const;
|
||||
int check_and_prepare_estimate_parallel_params(const int64_t input_parallel_degree_limit,
|
||||
const double cost_threshold_ms,
|
||||
int64_t &px_part_gi_min_part_per_dop,
|
||||
double &cost_threshold_us,
|
||||
int64_t &server_cnt,
|
||||
int64_t &cur_parallel_degree_limit) const;
|
||||
int prepare_estimate_parallel(const int64_t pre_parallel,
|
||||
const int64_t parallel_degree_limit,
|
||||
const double cost_threshold_us,
|
||||
const int64_t server_cnt,
|
||||
const int64_t px_part_gi_min_part_per_dop,
|
||||
const double px_cost,
|
||||
const double cost,
|
||||
int64_t &cur_parallel,
|
||||
double &part_cnt_per_dop) const;
|
||||
int estimate_cost_for_parallel(const int64_t cur_parallel,
|
||||
const double part_cnt_per_dop,
|
||||
double &px_cost,
|
||||
double &cost) const;
|
||||
virtual int estimate_cost() override;
|
||||
virtual int re_estimate_cost(EstimateCostInfo &info, double &card, double &cost) override;
|
||||
int re_estimate_cost(EstimateCostInfo &info, double &card, double &index_back_cost, double &cost);
|
||||
static int re_estimate_cost(const EstimateCostInfo ¶m,
|
||||
ObCostTableScanInfo &est_cost_info,
|
||||
const SampleInfo &sample_info,
|
||||
const ObOptEstCost::MODEL_TYPE model_type,
|
||||
const double orign_phy_query_range_row_count,
|
||||
const double orign_query_range_row_count,
|
||||
double &card,
|
||||
double &index_back_cost,
|
||||
double &cost);
|
||||
inline bool can_use_remote_estimate()
|
||||
{
|
||||
return NULL == table_opt_info_ ? false :
|
||||
@ -722,24 +778,35 @@ struct EstimateCostInfo {
|
||||
virtual int estimate_cost() override;
|
||||
void reuse();
|
||||
virtual int re_estimate_cost(EstimateCostInfo &info, double &card, double &cost) override;
|
||||
int get_re_estimate_param(EstimateCostInfo ¶m, EstimateCostInfo &left_param, EstimateCostInfo &right_param);
|
||||
int do_re_estimate_cost(EstimateCostInfo &info, double &card, double &op_cost, double &cost);
|
||||
int get_re_estimate_param(EstimateCostInfo ¶m,
|
||||
EstimateCostInfo &left_param,
|
||||
EstimateCostInfo &right_param,
|
||||
bool re_est_for_op);
|
||||
int try_set_batch_nlj_for_right_access_path(bool enable);
|
||||
int re_estimate_rows(double left_output_rows, double right_output_rows, double &row_count);
|
||||
int cost_nest_loop_join(double left_output_rows,
|
||||
int cost_nest_loop_join(int64_t join_parallel,
|
||||
double left_output_rows,
|
||||
double left_cost,
|
||||
double right_output_rows,
|
||||
double right_cost,
|
||||
bool re_est_for_op,
|
||||
double &op_cost,
|
||||
double &cost);
|
||||
int cost_merge_join(double left_output_rows,
|
||||
int cost_merge_join(int64_t join_parallel,
|
||||
double left_output_rows,
|
||||
double left_cost,
|
||||
double right_output_rows,
|
||||
double right_cost,
|
||||
bool re_est_for_op,
|
||||
double &op_cost,
|
||||
double &cost);
|
||||
int cost_hash_join(double left_output_rows,
|
||||
int cost_hash_join(int64_t join_parallel,
|
||||
double left_output_rows,
|
||||
double left_cost,
|
||||
double right_output_rows,
|
||||
double right_cost,
|
||||
bool re_est_for_op,
|
||||
double &op_cost,
|
||||
double &cost);
|
||||
int compute_join_path_property();
|
||||
@ -866,6 +933,15 @@ struct EstimateCostInfo {
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
static int compute_join_path_parallel_and_server_info(const common::ObAddr &local_server_addr,
|
||||
const Path *left_path,
|
||||
const Path *right_path,
|
||||
const DistAlgo join_dist_algo,
|
||||
bool const is_slave_mapping,
|
||||
int64_t ¶llel,
|
||||
int64_t &available_parallel,
|
||||
int64_t &server_cnt,
|
||||
ObIArray<common::ObAddr> &server_list);
|
||||
private:
|
||||
int compute_hash_hash_sharding_info();
|
||||
int compute_join_path_ordering();
|
||||
@ -1196,7 +1272,6 @@ struct NullAwareAntiJoinInfo {
|
||||
output_row_size_(-1.0),
|
||||
anti_or_semi_match_sel_(1.0),
|
||||
table_partition_info_(NULL),
|
||||
table_sharding_info_(NULL),
|
||||
table_meta_info_(common::OB_INVALID_ID),
|
||||
join_info_(NULL),
|
||||
used_conflict_detectors_(),
|
||||
@ -1272,8 +1347,6 @@ struct NullAwareAntiJoinInfo {
|
||||
|
||||
inline ObTablePartitionInfo *get_table_partition_info() { return table_partition_info_; }
|
||||
|
||||
inline ObShardingInfo *get_table_sharding_info() { return table_sharding_info_; }
|
||||
|
||||
int param_funct_table_expr(ObRawExpr* &function_table_expr,
|
||||
ObIArray<ObExecParamRawExpr *> &nl_params,
|
||||
ObIArray<ObRawExpr*> &subquery_exprs);
|
||||
@ -1613,13 +1686,24 @@ struct NullAwareAntiJoinInfo {
|
||||
|
||||
int compute_sharding_info_for_base_paths(ObIArray<AccessPath *> &access_paths);
|
||||
|
||||
int compute_sharding_info_for_base_path(const bool use_das,
|
||||
ObTablePartitionInfo &table_partition_info,
|
||||
ObShardingInfo *&sharding_info);
|
||||
|
||||
int compute_sharding_info_for_base_path(ObIArray<AccessPath *> &access_paths,
|
||||
const int64_t cur_idx);
|
||||
int get_sharding_info_from_available_access_paths(ObIArray<AccessPath *> &access_paths,
|
||||
const int64_t cur_idx,
|
||||
ObShardingInfo *&sharding_info) const;
|
||||
int compute_base_table_path_plan_type(AccessPath *access_path);
|
||||
int compute_base_table_path_ordering(AccessPath *access_path);
|
||||
int compute_base_table_parallel_and_server_info(AccessPath *path);
|
||||
int compute_parallel_and_server_info_for_base_paths(ObIArray<AccessPath *> &access_paths);
|
||||
int get_base_path_table_dop(uint64_t index_id, int64_t ¶llel);
|
||||
int compute_access_path_parallel(ObIArray<AccessPath *> &access_paths,
|
||||
int64_t ¶llel);
|
||||
int get_random_parallel(const ObIArray<AccessPath *> &access_paths, int64_t ¶llel);
|
||||
int get_parallel_from_available_access_paths(int64_t ¶llel) const;
|
||||
int compute_base_table_parallel_and_server_info(const OpParallelRule op_parallel_rule,
|
||||
const int64_t parallel,
|
||||
AccessPath *path);
|
||||
int get_explicit_dop_for_path(const uint64_t index_id, int64_t ¶llel);
|
||||
int prune_paths_due_to_parallel(ObIArray<AccessPath *> &access_paths);
|
||||
/**
|
||||
* 根据输入的左右树,生成连接之后的树。
|
||||
* 此过程会生成一个ObJoinOrder输出,ObJoinOrder中包含若干个JoinOrder,每个JoinOrder是一个确定的连接方法。
|
||||
@ -1742,12 +1826,12 @@ struct NullAwareAntiJoinInfo {
|
||||
const double other_cond_sel,
|
||||
const ValidPathInfo &path_info);
|
||||
|
||||
int find_minimal_cost_merge_path(const int64_t in_parallel,
|
||||
const Path &left_path,
|
||||
int find_minimal_cost_merge_path(const Path &left_path,
|
||||
const MergeKeyInfo &left_merge_key,
|
||||
const ObIArray<ObRawExpr*> &right_join_exprs,
|
||||
const ObIArray<Path*> &right_path_list,
|
||||
const DistAlgo join_dist_algo,
|
||||
const bool is_slave_mapping,
|
||||
ObIArray<OrderItem> &best_order_items,
|
||||
Path *&best_path,
|
||||
bool &best_need_sort,
|
||||
@ -1791,8 +1875,8 @@ struct NullAwareAntiJoinInfo {
|
||||
const bool is_naaj,
|
||||
const bool is_sna);
|
||||
|
||||
int generate_join_filter_infos(const Path *left_path,
|
||||
const Path *right_path,
|
||||
int generate_join_filter_infos(const Path &left_path,
|
||||
const Path &right_path,
|
||||
const ObJoinType join_type,
|
||||
const DistAlgo join_dist_algo,
|
||||
const ObIArray<ObRawExpr*> &equal_join_conditions,
|
||||
@ -2168,9 +2252,6 @@ struct NullAwareAntiJoinInfo {
|
||||
int64_t &common_prefix_idx);
|
||||
|
||||
private:
|
||||
int check_px_spatial_index(ObSqlSchemaGuard *schema_guard,
|
||||
uint64_t index_id,
|
||||
bool &res_tmp);
|
||||
int compute_cost_and_prune_access_path(PathHelper &helper,
|
||||
ObIArray<AccessPath *> &access_paths);
|
||||
int revise_output_rows_after_creating_path(PathHelper &helper,
|
||||
@ -2366,7 +2447,6 @@ struct NullAwareAntiJoinInfo {
|
||||
double output_row_size_;
|
||||
double anti_or_semi_match_sel_; //for anti/semi join
|
||||
ObTablePartitionInfo *table_partition_info_; // only for base table
|
||||
ObShardingInfo *table_sharding_info_; // only for base table
|
||||
ObTableMetaInfo table_meta_info_; // only for base table
|
||||
JoinInfo* join_info_; //记录连接信息
|
||||
common::ObSEArray<ConflictDetector*, 8, common::ModulePageAllocator, true> used_conflict_detectors_; //记录当前join order用掉了哪些冲突检测器
|
||||
|
||||
Reference in New Issue
Block a user