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

@ -17,134 +17,88 @@
#include "ob_optimizer_util.h"
#include "sql/optimizer/ob_log_plan.h"
#include "ob_log_exchange.h"
#include "sql/rewrite/ob_transform_utils.h"
#include "sql/optimizer/ob_join_order.h"
#include "share/ob_order_perserving_encoder.h"
#include "common/ob_smart_call.h"
using namespace oceanbase::sql;
using namespace oceanbase::common;
int ObLogSort::set_topk_params(
ObRawExpr* limit_count, ObRawExpr* limit_offset, int64_t minimum_row_count, int64_t topk_precision)
int ObLogSort::set_sort_keys(const common::ObIArray<OrderItem> &order_keys)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(limit_count)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("limit_count is NULL", K(ret));
if (OB_FAIL(sort_keys_.assign(order_keys))) {
LOG_WARN("failed to set sort keys", K(ret));
} else { /* do nothing */ }
return ret;
}
int ObLogSort::create_encode_sortkey_expr(const common::ObIArray<OrderItem> &order_keys) {
int ret = OB_SUCCESS;
ObOpRawExpr* encode_expr = NULL;
if (OB_ISNULL(get_plan())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(get_plan()), K(ret));
} else if (OB_ISNULL(get_plan()->get_optimizer_context().get_exec_ctx())){
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(get_plan()), K(ret));
} else {
topk_limit_count_ = limit_count;
topk_offset_count_ = limit_offset;
minimum_row_count_ = minimum_row_count;
topk_precision_ = topk_precision;
int64_t ecd_pos = 0;
// Prefix sort and hash-based sort both can combine with encode sort.
// And prefix sort is prior to hash-based sort(part sort).
if (is_prefix_sort() || is_part_sort()) {
int64_t orig_pos = is_prefix_sort() ? get_prefix_pos() : get_part_cnt();
for (int64_t i = 0; OB_SUCC(ret) && i < orig_pos; ++i) {
if (OB_FAIL(encode_sortkeys_.push_back(order_keys.at(i)))) {
LOG_WARN("failed to add encodekey", K(ret));
} else {
ecd_pos++;
}
}
} else {
ecd_pos = 0;
}
ObRawExprFactory &expr_factory = get_plan()->get_optimizer_context().get_expr_factory();
ObExecContext* exec_ctx = get_plan()->get_optimizer_context().get_exec_ctx();
OrderItem encode_sortkey;
if (OB_FAIL(ObSQLUtils::create_encode_sortkey_expr(
expr_factory, exec_ctx, order_keys, ecd_pos, encode_sortkey))) {
LOG_WARN("failed to create encode sortkey expr", K(ret));
} else if (OB_FAIL(encode_sortkeys_.push_back(encode_sortkey))) {
LOG_WARN("failed to push back encode sortkey", K(ret));
} else { /* do nothing*/ }
}
return ret;
}
int ObLogSort::transmit_op_ordering()
int ObLogSort::create_hash_sortkey(const common::ObIArray<OrderItem> &order_keys)
{
int ret = OB_SUCCESS;
// do nothing ,keep self ordering
return ret;
}
int ObLogSort::copy_without_child(ObLogicalOperator*& out)
{
int ret = OB_SUCCESS;
out = NULL;
ObLogicalOperator* op = NULL;
ObLogSort* sort = NULL;
if (OB_FAIL(clone(op))) {
LOG_WARN("failed to clone sort operator", K(ret));
} else if (OB_ISNULL(sort = static_cast<ObLogSort*>(op))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to cast ObLogicalOperator * to ObLogSort *", K(ret));
} else if ((NULL != topk_limit_count_) &&
OB_FAIL(
sort->set_topk_params(topk_limit_count_, topk_offset_count_, minimum_row_count_, topk_precision_))) {
LOG_WARN("failed to set_topk_params", K(ret));
ObOpRawExpr *hash_expr = NULL;
ObRawExprFactory &expr_factory = get_plan()->get_optimizer_context().get_expr_factory();
ObExecContext *exec_ctx = get_plan()->get_optimizer_context().get_exec_ctx();
if (OB_FAIL(expr_factory.create_raw_expr(T_FUN_SYS_HASH, hash_expr))) {
LOG_WARN("failed to create raw expr", K(ret));
} else {
sort->sort_keys_.reset();
for (int64_t i = 0; OB_SUCC(ret) && i < sort_keys_.count(); ++i) {
if (OB_FAIL(sort->sort_keys_.push_back(sort_keys_.at(i)))) {
LOG_WARN("failed to push back item", K(ret));
for (int64_t i = 0; OB_SUCC(ret) && i < part_cnt_; ++i) {
if (OB_FAIL(hash_expr->add_param_expr(order_keys.at(i).expr_))) {
LOG_WARN("failed to add param expr", K(ret));
}
}
sort->prefix_pos_ = prefix_pos_;
out = sort;
}
return ret;
}
int ObLogSort::add_sort_key(const OrderItem& key)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(key.expr_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("key expr is null");
} else if (key.expr_->has_const_or_const_expr_flag()) {
// do nothing
} else if (OB_FAIL(sort_keys_.push_back(key))) {
LOG_WARN("push back sort key failed", K(ret));
} else if (OB_FAIL(get_op_ordering().push_back(key))) {
LOG_WARN("push back sort item failed", K(ret));
}
return ret;
}
int ObLogSort::set_sort_keys(const common::ObIArray<OrderItem>& order_keys)
{
int ret = OB_SUCCESS;
reset_op_ordering();
reset_local_ordering();
for (int64_t i = 0; OB_SUCC(ret) && i < order_keys.count(); ++i) {
if (OB_ISNULL(order_keys.at(i).expr_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("order key expr is null");
} else if (order_keys.at(i).expr_->has_const_or_const_expr_flag()) {
// do nothing
} else if (OB_FAIL(sort_keys_.push_back(order_keys.at(i)))) {
LOG_WARN("push back order key expr failed", K(ret));
} else if (OB_FAIL(get_op_ordering().push_back(order_keys.at(i)))) {
LOG_WARN("failed to push back order item", K(ret));
} else {
// do nothing
}
}
return ret;
}
int ObLogSort::check_prefix_sort()
{
int ret = OB_SUCCESS;
int64_t left_match_count = 0;
ObLogicalOperator* child = NULL;
if (OB_ISNULL(child = get_child(0))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(child), K(ret));
} else if (OB_FAIL(ObOptimizerUtil::find_common_prefix_ordering(child->get_op_ordering(),
sort_keys_,
child->get_ordering_output_equal_sets(),
child->get_output_const_exprs(),
left_match_count,
prefix_pos_))) {
LOG_WARN("failed to find common prefix ordering", K(ret));
if (OB_FAIL(ret)) {
//do nothing
} else if (OB_FAIL(hash_expr->formalize(exec_ctx->get_my_session()))) {
LOG_WARN("failed to formalize expr", K(ret));
} else {
prefix_pos_ = std::min(prefix_pos_, child->get_op_ordering().count());
LOG_TRACE("succeed to check prefix sort", K(left_match_count), K(prefix_pos_), K(child->get_output_const_exprs()));
hash_sortkey_.expr_ = hash_expr;
hash_sortkey_.order_type_ = default_asc_direction();
}
return ret;
}
int ObLogSort::check_local_merge_sort()
{
int ret = OB_SUCCESS;
bool need_sort = false;
if (OB_FAIL(check_need_sort_for_local_order(0, &get_op_ordering(), need_sort))) {
LOG_WARN("failed to check need sort for local order", K(ret));
} else if (!need_sort) {
// If child can retain local ordering for sort, then convert sort to sort with local order for optimization
set_local_merge_sort(true);
}
return ret;
}
int ObLogSort::get_sort_exprs(common::ObIArray<ObRawExpr*>& sort_exprs)
int ObLogSort::get_sort_exprs(common::ObIArray<ObRawExpr*> &sort_exprs)
{
int ret = OB_SUCCESS;
for (int64_t i = 0; OB_SUCC(ret) && i < sort_keys_.count(); ++i) {
@ -155,128 +109,53 @@ int ObLogSort::get_sort_exprs(common::ObIArray<ObRawExpr*>& sort_exprs)
return ret;
}
int ObLogSort::clone_sort_keys_for_topk(
ObIArray<OrderItem>& topk_sort_keys, const ObIArray<std::pair<ObRawExpr*, ObRawExpr*>>& push_down_avg_arr)
int ObLogSort::get_op_exprs(ObIArray<ObRawExpr*> &all_exprs)
{
int ret = OB_SUCCESS;
ObOptimizerContext* opt_ctx = NULL;
ObLogPlan* plan = get_plan();
if (OB_ISNULL(plan) || OB_ISNULL(opt_ctx = &(plan->get_optimizer_context()))) {
if (NULL != topn_expr_ && OB_FAIL(all_exprs.push_back(topn_expr_))) {
LOG_WARN("failed to push back expr", K(ret));
} else if (NULL != topk_limit_expr_ && OB_FAIL(all_exprs.push_back(topk_limit_expr_))) {
LOG_WARN("failed to push back expr", K(ret));
} else if (NULL != topk_offset_expr_ && OB_FAIL(all_exprs.push_back(topk_offset_expr_))) {
LOG_WARN("failed to push back expr", K(ret));
} else if (OB_ISNULL(get_plan())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("plan or opt_ctx is NULL", KP(plan), KP(opt_ctx), K(ret));
LOG_WARN("get unexpected null", K(get_plan()), K(ret));
} else if (GCONF._enable_newsort
&& ObSQLUtils::check_can_encode_sortkey(sort_keys_)
&& OB_FAIL(create_encode_sortkey_expr(sort_keys_))) {
LOG_WARN("failed to create encode sortkey expr", K(ret));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < sort_keys_.count(); ++i) {
OrderItem& cur_order_item = sort_keys_.at(i);
ObRawExpr* sort_expr = cur_order_item.expr_;
ObRawExpr* new_sort_expr = NULL;
if (OB_FAIL(ObOptimizerUtil::clone_expr_for_topk(opt_ctx->get_expr_factory(), sort_expr, new_sort_expr))) {
LOG_WARN("failed to copy expr", K(cur_order_item), K(ret));
} else if (OB_ISNULL(new_sort_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("new_sort_expr is NULL", K(ret));
} else if (OB_FAIL(replace_expr_action(push_down_avg_arr, new_sort_expr))) {
LOG_WARN("failed to replace_expr_action", K(ret));
} else {
OrderItem new_order_item = sort_keys_.at(i);
new_order_item.expr_ = new_sort_expr;
if (OB_FAIL(topk_sort_keys.push_back(new_order_item))) {
LOG_WARN("failed to push back order_item", K(ret));
}
}
}
}
return ret;
}
int ObLogSort::push_down_sort(ObLogicalOperator* consumer_exch)
{
int ret = OB_SUCCESS;
// sort sort
// | |
// exchange => push down exchange
// | |
// exchange exchange
// |
// sort
ObLogicalOperator* child = NULL;
ObLogicalOperator* producer_exch = NULL;
bool need_sort = true;
if (OB_ISNULL(consumer_exch) || OB_ISNULL(producer_exch = consumer_exch->get_child(first_child))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret), K(consumer_exch), K(producer_exch));
} else if (OB_FAIL(producer_exch->check_need_sort_below_node(0, get_sort_keys(), need_sort))) {
LOG_WARN("failed to check need sort", K(ret));
} else if (!need_sort) {
// no need alloc pushed_down sort
} else if (OB_FAIL(producer_exch->allocate_sort_below(first_child, get_sort_keys()))) {
LOG_WARN("failed to insert LOG_SORT operator", K(ret));
} else {
LOG_TRACE("succeed to push down sort");
}
return ret;
}
int ObLogSort::allocate_exchange_post(AllocExchContext* ctx)
{
int ret = OB_SUCCESS;
ObLogicalOperator* child = NULL;
if (OB_ISNULL(ctx) || OB_ISNULL(child = get_child(first_child))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ctx), K(child), K(ret));
} else if (OB_FAIL(sharding_info_.copy_with_part_keys(child->get_sharding_info()))) {
LOG_WARN("failed to deep copy sharding info from child", K(ret));
} else { /*do nothing*/
}
return ret;
}
int ObLogSort::allocate_exchange(AllocExchContext* ctx, ObExchangeInfo& exch_info)
{
int ret = OB_SUCCESS;
if (OB_ISNULL(ctx) || OB_ISNULL(get_child(first_child))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret), K(ctx), K(get_child(first_child)));
} else if (OB_FAIL(get_child(first_child)->allocate_exchange(ctx, exch_info))) {
LOG_WARN("failed to allocate exchange", K(ret));
} else if (OB_ISNULL(get_child(first_child))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (!exch_info.is_pq_dist() && OB_FAIL(push_down_sort(get_child(first_child)))) {
LOG_WARN("failed to push down sort", K(ret));
} else {
LOG_TRACE("succeed to allocate exchange for sort operator");
}
return ret;
}
int ObLogSort::allocate_expr_pre(ObAllocExprContext& ctx)
{
int ret = OB_SUCCESS;
uint64_t producer_id = OB_INVALID_ID;
if (OB_FAIL(get_next_producer_id(get_child(first_child), producer_id))) {
LOG_WARN("failed to get next producer id", K(ret));
} else {
ObSEArray<ObRawExpr*, 8> exprs;
for (int64_t i = 0; OB_SUCC(ret) && i < sort_keys_.count(); i++) {
if (OB_ISNULL(sort_keys_.at(i).expr_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (OB_FAIL(all_exprs.push_back(sort_keys_.at(i).expr_))) {
LOG_WARN("failed to push back exprs", K(ret));
} else { /*do nothing*/ }
}
for (int64_t i = 0; OB_SUCC(ret) && i < encode_sortkeys_.count(); i++) {
if (OB_ISNULL(encode_sortkeys_.at(i).expr_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret), K(i));
} else if (OB_FAIL(exprs.push_back(sort_keys_.at(i).expr_))) {
} else if (OB_FAIL(all_exprs.push_back(encode_sortkeys_.at(i).expr_))) {
LOG_WARN("failed to push back expr", K(ret));
} else { /*do nothing*/
}
} else { /*do nothing*/ }
}
if (OB_SUCC(ret)) {
if (OB_FAIL(add_exprs_to_ctx(ctx, exprs, producer_id))) {
LOG_WARN("failed to add exprs to ctx", K(ret));
} else if (OB_FAIL(ObLogicalOperator::allocate_expr_pre(ctx))) {
LOG_WARN("failed to add exprs to ctx", K(ret));
} else { /*do nothing*/
if (part_cnt_ > 0) {
if (OB_ISNULL(hash_sortkey_.expr_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (OB_FAIL(all_exprs.push_back(hash_sortkey_.expr_))) {
LOG_WARN("failed to push back expr", K(ret));
}
}
if (OB_FAIL(ObLogicalOperator::get_op_exprs(all_exprs))) {
LOG_WARN("failed to get op exprs", K(ret));
} else { /*do nothing*/ }
}
}
return ret;
@ -284,47 +163,51 @@ int ObLogSort::allocate_expr_pre(ObAllocExprContext& ctx)
uint64_t ObLogSort::hash(uint64_t seed) const
{
uint64_t hash_value = seed;
HASH_ARRAY(sort_keys_, hash_value);
hash_value = ObOptimizerUtil::hash_expr(topn_count_, hash_value);
hash_value = do_hash(minimum_row_count_, hash_value);
hash_value = do_hash(topk_precision_, hash_value);
hash_value = do_hash(prefix_pos_, hash_value);
hash_value = ObOptimizerUtil::hash_expr(topk_limit_count_, hash_value);
hash_value = ObOptimizerUtil::hash_expr(topk_offset_count_, hash_value);
hash_value = ObLogicalOperator::hash(hash_value);
bool is_topn = NULL != topn_expr_;
seed = do_hash(is_topn, seed);
seed = ObLogicalOperator::hash(seed);
return hash_value;
return seed;
}
int ObLogSort::print_my_plan_annotation(char* buf, int64_t& buf_len, int64_t& pos, ExplainType type)
int ObLogSort::print_my_plan_annotation(char *buf,
int64_t &buf_len,
int64_t &pos,
ExplainType type)
{
int ret = OB_SUCCESS;
if (OB_FAIL(BUF_PRINTF(", "))) {
LOG_WARN("BUF_PRINTF fails", K(ret));
} else {
const ObIArray<OrderItem>& sort_keys = get_sort_keys();
EXPLAIN_PRINT_SORT_ITEMS(sort_keys, type);
if (NULL != topn_count_) {
ObRawExpr* topn = topn_count_;
ObSEArray<OrderItem, 1> sort_keys;
if (NULL != get_hash_sortkey().expr_
&& OB_FAIL(sort_keys.push_back(get_hash_sortkey()))) {
LOG_WARN("failed to push back sortkeys", K(ret));
} else if (OB_FAIL(append(sort_keys, get_sort_keys()))) {
LOG_WARN("failed to append sortkeys", K(ret));
} else {
EXPLAIN_PRINT_SORT_ITEMS(sort_keys, type);
}
if (OB_SUCC(ret) && NULL != topn_expr_) {
ObRawExpr *topn = topn_expr_;
BUF_PRINTF(", ");
EXPLAIN_PRINT_EXPR(topn, type);
}
ObRawExpr* limit = topk_limit_count_;
if (NULL != limit) {
if (OB_FAIL(BUF_PRINTF(", minimum_row_count:%ld top_precision:%ld ", minimum_row_count_, topk_precision_))) {
ObRawExpr *limit = topk_limit_expr_;
if (OB_SUCC(ret) && NULL != limit) {
if (OB_FAIL(BUF_PRINTF(", minimum_row_count:%ld top_precision:%ld ",
minimum_row_count_, topk_precision_))) {
LOG_WARN("BUF_PRINTF fails", K(ret));
} else {
ObRawExpr* offset = topk_offset_count_;
ObRawExpr *offset = topk_offset_expr_;
BUF_PRINTF(", ");
EXPLAIN_PRINT_EXPR(limit, type);
BUF_PRINTF(", ");
EXPLAIN_PRINT_EXPR(offset, type);
}
} else { /* Do nothing */
}
} else { /* Do nothing */ }
if (OB_SUCC(ret) && prefix_pos_ > 0) {
if (OB_SUCC(ret) && prefix_pos_> 0) {
BUF_PRINTF(", prefix_pos(");
if (OB_FAIL(BUF_PRINTF("%ld)", prefix_pos_))) {
LOG_WARN("BUF_PRINTF fails", K(ret), K(prefix_pos_));
@ -333,110 +216,206 @@ int ObLogSort::print_my_plan_annotation(char* buf, int64_t& buf_len, int64_t& po
if (OB_SUCC(ret) && is_local_merge_sort_) {
BUF_PRINTF(", local merge sort");
}
if (OB_SUCC(ret) &&
(EXPLAIN_EXTENDED == type
|| EXPLAIN_EXTENDED_NOADDR == type
|| EXPLAIN_PLANREGRESS == type)
&& enable_encode_sortkey_opt()) {
BUF_PRINTF(", encoded");
}
if (OB_SUCC(ret) && is_fetch_with_ties_) {
BUF_PRINTF(", with_ties(true)");
}
}
return ret;
}
int ObLogSort::check_output_dep_specific(ObRawExprCheckDep& checker)
{
int ret = OB_SUCCESS;
// sort keys
for (int64_t i = 0; OB_SUCC(ret) && i < sort_keys_.count(); i++) {
if (OB_ISNULL(sort_keys_.at(i).expr_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("sort_keys_.at(i).expr_ is null", K(ret));
} else if (OB_FAIL(checker.check(*sort_keys_.at(i).expr_))) {
LOG_WARN("failed to check sort_keys_.at(i).expr_", K(i), K(ret));
} else {
}
}
if (OB_SUCC(ret) && NULL != topn_count_) {
if (OB_FAIL(checker.check(*topn_count_))) {
LOG_WARN("failed to check limit_count_", K(*topn_count_), K(ret));
}
}
if (OB_SUCC(ret) && NULL != topk_limit_count_) {
if (OB_FAIL(checker.check(*topk_limit_count_))) {
LOG_WARN("failed to check limit_count_", K(*topk_limit_count_), K(ret));
} else if (NULL != topk_offset_count_) {
if (OB_FAIL(checker.check(*topk_offset_count_))) {
LOG_WARN("failed to check limit_offset_", KPC(topk_offset_count_), K(ret));
}
} else { /* Do nothing */
}
}
return ret;
}
int ObLogSort::inner_replace_generated_agg_expr(const ObIArray<std::pair<ObRawExpr*, ObRawExpr*>>& to_replace_exprs)
int ObLogSort::inner_replace_generated_agg_expr(
const ObIArray<std::pair<ObRawExpr *, ObRawExpr *> > &to_replace_exprs)
{
int ret = OB_SUCCESS;
int64_t N = sort_keys_.count();
for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) {
OrderItem& cur_order_item = sort_keys_.at(i);
for(int64_t i = 0; OB_SUCC(ret) && i < N; ++i) {
OrderItem &cur_order_item = sort_keys_.at(i);
if (OB_FAIL(replace_expr_action(to_replace_exprs, cur_order_item.expr_))) {
LOG_WARN("failed to resolve ref params in sort key ", K(cur_order_item), K(ret));
} else { /* Do nothing */
}
} else { /* Do nothing */ }
}
for(int64_t i = 0; OB_SUCC(ret) && i < encode_sortkeys_.count(); ++i) {
OrderItem &cur_order_item = encode_sortkeys_.at(i);
if (OB_FAIL(replace_expr_action(to_replace_exprs, cur_order_item.expr_))) {
LOG_WARN("failed to resolve ref params in sort key ", K(cur_order_item), K(ret));
} else { /* Do nothing */ }
}
if (OB_SUCC(ret) && part_cnt_ > 0) {
if (OB_FAIL(replace_expr_action(to_replace_exprs, hash_sortkey_.expr_))) {
LOG_WARN("failed to resolve ref params of hash sortkey", K(hash_sortkey_), K(ret));
} else { /* Do nothing */ }
}
return ret;
}
const char* ObLogSort::get_name() const
const char *ObLogSort::get_name() const
{
return NULL != topn_count_ ? "TOP-N SORT" : log_op_def::get_op_name(type_);
const char *ret = NULL;
if (NULL != topn_expr_) {
ret = "TOP-N SORT";
} else if (NULL == topk_limit_expr_ && prefix_pos_ <= 0 && part_cnt_ > 0) {
ret = "PARTITION SORT";
}
return NULL != ret ? ret : log_op_def::get_op_name(type_);
}
int ObLogSort::est_width()
{
int ret = OB_SUCCESS;
double width = 0.0;
ObSEArray<ObRawExpr*, 16> output_exprs;
ObLogicalOperator *child = NULL;
if (OB_ISNULL(get_plan()) ||
OB_ISNULL(child = get_child(ObLogicalOperator::first_child))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid plan", K(ret));
} else if (!get_plan()->get_candidate_plans().is_final_sort_) {
width = child->get_width();
set_width(width);
LOG_TRACE("est width for non-final sort", K(output_exprs), K(width));
} else if (OB_FAIL(get_sort_output_exprs(output_exprs))) {
LOG_WARN("failed to get sort output exprs", K(ret));
} else if (OB_FAIL(ObOptEstCost::estimate_width_for_exprs(get_plan()->get_basic_table_metas(),
get_plan()->get_selectivity_ctx(),
output_exprs,
width))) {
LOG_WARN("failed to estimate width for output orderby exprs", K(ret));
} else {
set_width(width);
LOG_TRACE("est width for final sort", K(output_exprs), K(width));
}
return ret;
}
int ObLogSort::get_sort_output_exprs(ObIArray<ObRawExpr *> &output_exprs)
{
int ret = OB_SUCCESS;
ObLogPlan *plan = NULL;
ObSEArray<ObRawExpr*, 16> candi_exprs;
ObSEArray<ObRawExpr*, 16> extracted_col_aggr_winfunc_exprs;
if (OB_ISNULL(plan = get_plan())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid input", K(ret));
} else if (OB_FAIL(append_array_no_dup(candi_exprs, plan->get_select_item_exprs_for_width_est()))) {
LOG_WARN("failed to add into output exprs", K(ret));
} else if (OB_FAIL(ObRawExprUtils::extract_col_aggr_winfunc_exprs(candi_exprs,
extracted_col_aggr_winfunc_exprs))) {
} else if (OB_FAIL(append_array_no_dup(output_exprs, extracted_col_aggr_winfunc_exprs))) {
LOG_WARN("failed to add into output exprs", K(ret));
} else {/*do nothing*/}
return ret;
}
int ObLogSort::est_cost()
{
int ret = OB_SUCCESS;
double sort_cost = 0.0;
ObLogicalOperator* first_child = get_child(ObLogicalOperator::first_child);
if (OB_ISNULL(first_child)) {
double double_topn_count = -1;
ObLogicalOperator *child = get_child(ObLogicalOperator::first_child);
if (OB_ISNULL(child)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("first child is null", K(ret));
LOG_WARN("get unexpected null", K(child), K(ret));
} else if (OB_FAIL(inner_est_cost(child->get_card(), double_topn_count, sort_cost))) {
LOG_WARN("failed to est sort cost", K(ret));
} else {
ObSortCostInfo cost_info(first_child->get_card(), first_child->get_width(), get_prefix_pos(), get_est_sel_info());
if (OB_FAIL(ObOptEstCost::cost_sort(cost_info, get_sort_keys(), sort_cost))) {
LOG_WARN("failed to calc cost", K(ret), K(first_child->get_type()));
set_op_cost(sort_cost);
set_cost(child->get_cost() + sort_cost);
if (double_topn_count >= 0) {
set_card(double_topn_count);
} else {
set_card(child->get_card());
}
LOG_TRACE("cost for sort operator", K(sort_cost), K(get_cost()),
K(get_card()));
}
return ret;
}
int ObLogSort::re_est_cost(EstimateCostInfo &param, double &card, double &cost)
{
int ret = OB_SUCCESS;
double child_card = 0.0;
double child_cost = 0.0;
double double_topn_count = -1;
double sort_cost = 0.0;
card = get_card();
if (param.need_row_count_ >=0 && param.need_row_count_ < card) {
card = param.need_row_count_;
}
ObLogicalOperator *child = get_child(ObLogicalOperator::first_child);
if (OB_ISNULL(child)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (OB_FALSE_IT(param.need_row_count_ = -1)) {
//limit N在sort算子被阻塞
} else if (OB_FAIL(SMART_CALL(child->re_est_cost(param, child_card, child_cost)))) {
LOG_WARN("failed to re est cost", K(ret));
} else if (OB_FAIL(inner_est_cost(child_card, double_topn_count, sort_cost))) {
LOG_WARN("failed to est sort cost", K(ret));
} else {
cost = child_cost + sort_cost;
card = child_card < card ? child_card : card;
if (double_topn_count >= 0 && card > double_topn_count) {
card = double_topn_count;
}
if (param.override_) {
set_op_cost(sort_cost);
set_cost(first_child->get_cost() + sort_cost);
set_card(first_child->get_card());
set_width(first_child->get_width());
LOG_TRACE("cost for sort operator", K(sort_cost), K(get_cost()), K(get_card()), K(get_width()));
set_cost(cost);
set_card(card);
}
}
return ret;
}
int ObLogSort::re_est_cost(const ObLogicalOperator* parent, double need_row_count, bool& re_est)
int ObLogSort::inner_est_cost(double child_card, double &double_topn_count, double &op_cost)
{
int ret = OB_SUCCESS;
re_est = false;
double new_op_cost = op_cost_;
ObLogicalOperator* child = get_child(first_child);
if (OB_ISNULL(child) || OB_ISNULL(parent)) {
int64_t parallel = 0;
int64_t topn_count = -1;
bool is_null_value = false;
double_topn_count = -1;
ObLogicalOperator *child = get_child(ObLogicalOperator::first_child);
if (OB_ISNULL(child) || OB_ISNULL(get_stmt()) ||
OB_ISNULL(get_plan())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(child), K(parent), K(ret));
} else if (need_row_count < get_card()) {
if (parent->get_type() != log_op_def::LOG_LIMIT) {
} else {
ObSortCostInfo cost_info(child->get_card(), child->get_width(), 0, NULL, need_row_count);
if (OB_FAIL(ObOptEstCost::cost_sort(cost_info, get_sort_keys(), new_op_cost))) {
LOG_WARN("failed to cost top-n sort", K(ret), K_(card), K(need_row_count));
} else { /*do nothing*/
}
LOG_WARN("get unexpected null", K(child), K(get_stmt()),
K(get_plan()), K(ret));
} else if (OB_UNLIKELY((parallel = get_parallel()) < 1)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected parallel degree", K(parallel), K(ret));
} else if (NULL != topn_expr_ &&
OB_FAIL(ObTransformUtils::get_limit_value(topn_expr_,
get_stmt(),
get_plan()->get_optimizer_context().get_params(),
get_plan()->get_optimizer_context().get_exec_ctx(),
&get_plan()->get_optimizer_context().get_allocator(),
topn_count,
is_null_value))) {
LOG_WARN("failed to get value", K(ret));
} else {
if (NULL != topn_expr_) {
double_topn_count = std::min(static_cast<double>(topn_count), child_card);
}
if (OB_SUCC(ret)) {
re_est = true;
set_op_cost(new_op_cost);
set_cost(new_op_cost + child->get_cost());
set_card(need_row_count);
LOG_TRACE("succeed to re-estimate cost for sort operator", K(new_op_cost), K(need_row_count), K(get_cost()));
get_plan()->get_selectivity_ctx().init_op_ctx(&child->get_output_equal_sets(), child_card);
ObOptimizerContext &opt_ctx = get_plan()->get_optimizer_context();
ObSortCostInfo cost_info(child_card / parallel,
child->get_width(),
get_prefix_pos(),
get_sort_keys(),
is_local_merge_sort_,
&get_plan()->get_update_table_metas(),
&get_plan()->get_selectivity_ctx(),
double_topn_count,
part_cnt_);
if (OB_FAIL(ObOptEstCost::cost_sort(cost_info, op_cost, opt_ctx.get_cost_model_type()))) {
LOG_WARN("failed to calc cost", K(ret), K(child->get_type()));
}
}
return ret;
@ -445,33 +424,29 @@ int ObLogSort::re_est_cost(const ObLogicalOperator* parent, double need_row_coun
int ObLogSort::compute_op_ordering()
{
int ret = OB_SUCCESS;
if (OB_FAIL(set_op_ordering(sort_keys_))) {
common::ObSEArray<OrderItem, 1> op_ordering;
if (part_cnt_ > 0 && OB_FAIL(op_ordering.push_back(hash_sortkey_))) {
LOG_WARN("failed to push back hash sortkey", K(ret));
} else if (OB_FAIL(append(op_ordering, sort_keys_))) {
LOG_WARN("failed to append sort keys", K(ret));
} else if (OB_FAIL(set_op_ordering(op_ordering))) {
LOG_WARN("failed to set op ordering", K(ret));
} else {
is_local_order_ = false;
}
return ret;
}
int ObLogSort::inner_append_not_produced_exprs(ObRawExprUniqueSet& raw_exprs) const
int ObLogSort::generate_link_sql_post(GenLinkStmtPostContext &link_ctx)
{
UNUSED(link_ctx);
int ret = OB_SUCCESS;
OZ(raw_exprs.append(topn_count_));
OZ(raw_exprs.append(topk_limit_count_));
OZ(raw_exprs.append(topk_offset_count_));
return ret;
}
int ObLogSort::generate_link_sql_pre(GenLinkStmtContext& link_ctx)
{
int ret = OB_SUCCESS;
ObLinkStmt* link_stmt = link_ctx.link_stmt_;
if (OB_ISNULL(link_stmt) || !link_stmt->is_inited()) {
// do nothing.
} else if (dblink_id_ != link_ctx.dblink_id_) {
link_ctx.dblink_id_ = OB_INVALID_ID;
link_ctx.link_stmt_ = NULL;
} else if (OB_FAIL(link_stmt->try_fill_select_strs(output_exprs_))) {
LOG_WARN("failed to fill link stmt select strs", K(ret), K(output_exprs_));
if (0 == dblink_id_) {
// do nothing
} else if (!startup_exprs_.empty()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("set operator have startup filters when reverse spell dblink sql", K(ret));
}
return ret;
}