提交 9eba955a 编写于 作者: O obdev 提交者: ob-robot

fix outline/hint generate plan bug

上级 a0384179
......@@ -476,75 +476,53 @@ int ObLogGroupBy::inner_replace_op_exprs(
int ObLogGroupBy::print_outline_data(PlanText &plan_text)
{
int ret = OB_SUCCESS;
char *buf = plan_text.buf_;
int64_t &buf_len = plan_text.buf_len_;
int64_t &pos = plan_text.pos_;
const ObDMLStmt *stmt = NULL;
ObString qb_name;
const ObLogicalOperator *child = NULL;
const ObLogicalOperator *op = NULL;
const ObLogGroupBy *group_by_op = NULL;
if (is_push_down()) {
/* print outline in top group by */
} else if (OB_ISNULL(get_plan()) || OB_ISNULL(stmt = get_plan()->get_stmt())
|| OB_ISNULL(child = get_child(ObLogicalOperator::first_child))) {
} else if (OB_ISNULL(get_plan()) || OB_ISNULL(stmt = get_plan()->get_stmt())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected NULL", K(ret), K(get_plan()), K(stmt), K(child));
} else if (OB_FAIL(child->get_pushdown_op(log_op_def::LOG_GROUP_BY, op))) {
LOG_WARN("failed to get push down group by", K(ret));
LOG_WARN("unexpected NULL", K(ret), K(get_plan()), K(stmt));
} else if (OB_FAIL(stmt->get_qb_name(qb_name))) {
LOG_WARN("fail to get qb_name", K(ret), K(stmt->get_stmt_id()));
} else if (NULL != op &&
OB_FAIL(BUF_PRINTF("%s%s(@\"%.*s\")",
ObQueryHint::get_outline_indent(plan_text.is_oneline_),
ObHint::get_hint_name(T_GBY_PUSHDOWN),
qb_name.length(), qb_name.ptr()))) {
LOG_WARN("fail to print buffer", K(ret), K(buf), K(buf_len), K(pos));
} else if (OB_FALSE_IT(group_by_op = static_cast<const ObLogGroupBy*>(NULL == op ? this : op))) {
} else if (HASH_AGGREGATE == group_by_op->get_algo() &&
OB_FAIL(BUF_PRINTF("%s%s(@\"%.*s\")",
ObQueryHint::get_outline_indent(plan_text.is_oneline_),
ObHint::get_hint_name(T_USE_HASH_AGGREGATE),
qb_name.length(), qb_name.ptr()))) {
LOG_WARN("fail to print buffer", K(ret), K(buf), K(buf_len), K(pos));
} else {/*do nothing*/}
} else {
if (OB_SUCC(ret) && has_push_down_) {
ObOptHint hint(T_GBY_PUSHDOWN);
hint.set_qb_name(qb_name);
if (OB_FAIL(hint.print_hint(plan_text))) {
LOG_WARN("failed to print hint", K(ret), K(hint));
}
}
if (OB_SUCC(ret) && (use_hash_aggr_ || use_part_sort_)) {
ObAggHint hint(use_hash_aggr_ ? T_USE_HASH_AGGREGATE : T_NO_USE_HASH_AGGREGATE);
hint.set_qb_name(qb_name);
hint.set_use_partition_sort(use_part_sort_);
if (OB_FAIL(hint.print_hint(plan_text))) {
LOG_WARN("failed to print hint", K(ret), K(hint));
}
}
}
return ret;
}
int ObLogGroupBy::print_used_hint(PlanText &plan_text)
{
int ret = OB_SUCCESS;
const ObLogicalOperator *child = NULL;
const ObLogicalOperator *op = NULL;
const ObHint *hint = NULL;
if (is_push_down()) {
/* print outline in top group by */
} else if (OB_ISNULL(get_plan())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected NULL", K(ret), K(get_plan()));
} else if (OB_ISNULL(child = get_child(ObLogicalOperator::first_child))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected NULL", K(ret), K(child));
} else if (OB_FAIL(child->get_pushdown_op(log_op_def::LOG_GROUP_BY, op))) {
LOG_WARN("failed to get push down group by", K(ret));
} else {
const ObHint *use_hash = get_plan()->get_log_plan_hint().get_normal_hint(T_USE_HASH_AGGREGATE);
const ObHint *pushdown = get_plan()->get_log_plan_hint().get_normal_hint(T_GBY_PUSHDOWN);
if (NULL != use_hash) {
// for pushdown group by, need get pushdown algo
const ObLogGroupBy *group_by_op = static_cast<const ObLogGroupBy*>(NULL == op ? this : op);
bool match_hint = (HASH_AGGREGATE == group_by_op->get_algo() && use_hash->is_enable_hint())
|| (MERGE_AGGREGATE == group_by_op->get_algo() && use_hash->is_disable_hint());
if (match_hint && OB_FAIL(use_hash->print_hint(plan_text))) {
LOG_WARN("failed to print used hint for group by", K(ret), K(*use_hash));
}
}
if (OB_SUCC(ret) && NULL != pushdown) {
bool match_hint = NULL == op ? pushdown->is_disable_hint()
: pushdown->is_enable_hint();
if (match_hint && OB_FAIL(pushdown->print_hint(plan_text))) {
LOG_WARN("failed to print used hint for group by", K(ret), K(*pushdown));
}
}
} else if (NULL != (hint = get_plan()->get_log_plan_hint().get_normal_hint(T_GBY_PUSHDOWN))
&& hint->is_enable_hint() == has_push_down_
&& OB_FAIL(hint->print_hint(plan_text))) {
LOG_WARN("failed to print used hint for group by", K(ret), K(*hint));
} else if (NULL != (hint = get_plan()->get_log_plan_hint().get_normal_hint(T_USE_HASH_AGGREGATE))
&& hint->is_enable_hint() == use_hash_aggr_
&& static_cast<const ObAggHint*>(hint)->force_partition_sort() == use_part_sort_
&& OB_FAIL(hint->print_hint(plan_text))) {
LOG_WARN("failed to print used hint for group by", K(ret), K(*hint));
}
return ret;
}
......
......@@ -76,7 +76,10 @@ public:
aggr_stage_(ObThreeStageAggrStage::NONE_STAGE),
three_stage_info_(),
rollup_adaptive_info_(),
force_push_down_(false)
force_push_down_(false),
use_hash_aggr_(false),
has_push_down_(false),
use_part_sort_(false)
{}
virtual ~ObLogGroupBy()
{}
......@@ -197,6 +200,8 @@ public:
{ return ObRollupStatus::ROLLUP_COLLECTOR == rollup_adaptive_info_.rollup_status_; }
inline void set_force_push_down(bool force_push_down)
{ force_push_down_ = force_push_down; }
void set_group_by_outline_info(bool use_hash_aggr, bool has_push_down, bool use_part_sort = false)
{ use_hash_aggr_ = use_hash_aggr; has_push_down_ = has_push_down; use_part_sort_ = use_part_sort; }
virtual int get_plan_item_info(PlanText &plan_text,
ObSqlPlanItem &plan_item) override;
......@@ -230,6 +235,10 @@ private:
// for rollup distributor and collector
ObRollupAdaptiveInfo rollup_adaptive_info_;
bool force_push_down_; // control by _aggregation_optimization_settings
// use print outline
bool use_hash_aggr_;
bool has_push_down_;
bool use_part_sort_;
};
} // end of namespace sql
} // end of namespace oceanbase
......
......@@ -6321,6 +6321,8 @@ int ObLogPlan::create_three_stage_group_plan(const ObIArray<ObRawExpr*> &group_b
} else if (OB_FAIL(third_group_by->set_rollup_info(third_rollup_status,
helper.rollup_id_expr_))) {
LOG_WARN("failed to set rollup parallel info", K(ret));
} else {
third_group_by->set_group_by_outline_info(HASH_AGGREGATE == second_aggr_algo, true);
}
}
return ret;
......@@ -6527,7 +6529,9 @@ int ObLogPlan::create_scala_group_plan(const ObIArray<ObAggFunRawExpr*> &aggr_it
is_from_povit,
origin_child_card))) {
LOG_WARN("failed to allocate scala group by as top", K(ret));
} else { /*do nothing*/ }
} else {
static_cast<ObLogGroupBy*>(top)->set_group_by_outline_info(false, false);
}
} else if (!groupby_helper.distinct_exprs_.empty() &&
OB_FAIL(top->check_sharding_compatible_with_reduce_expr(groupby_helper.distinct_exprs_,
is_partition_wise))) {
......@@ -6566,7 +6570,9 @@ int ObLogPlan::create_scala_group_plan(const ObIArray<ObAggFunRawExpr*> &aggr_it
is_from_povit,
origin_child_card))) {
LOG_WARN("failed to allocate scala group by as top", K(ret));
} else { /*do nothing*/ }
} else {
static_cast<ObLogGroupBy*>(top)->set_group_by_outline_info(false, groupby_helper.can_basic_pushdown_ || is_partition_wise); //zzydebug
}
}
return ret;
......@@ -6645,8 +6651,6 @@ int ObLogPlan::init_groupby_helper(const ObIArray<ObRawExpr*> &group_exprs,
ObSEArray<ObRawExpr*, 4> group_rollup_exprs;
bool push_group = false;
groupby_helper.is_scalar_group_by_ = true;
groupby_helper.force_use_hash_ = get_log_plan_hint().use_hash_aggregate();
groupby_helper.force_use_merge_ = get_log_plan_hint().use_merge_aggregate();
if (OB_FAIL(candidates_.get_best_plan(best_plan))) {
LOG_WARN("failed to get best plan", K(ret));
} else if (OB_ISNULL(best_plan) ||
......@@ -6659,6 +6663,11 @@ int ObLogPlan::init_groupby_helper(const ObIArray<ObRawExpr*> &group_exprs,
} else if (OB_FAIL(append(group_rollup_exprs, group_exprs)) ||
OB_FAIL(append(group_rollup_exprs, rollup_exprs))) {
LOG_WARN("failed to append group rollup exprs", K(ret));
} else if (OB_FAIL(get_log_plan_hint().get_aggregation_info(groupby_helper.force_use_hash_,
groupby_helper.force_use_merge_,
groupby_helper.force_part_sort_,
groupby_helper.force_normal_sort_))) {
LOG_WARN("failed to get aggregation info from hint", K(ret));
} else if (OB_FAIL(check_scalar_groupby_pushdown(aggr_items,
groupby_helper.can_scalar_pushdown_))) {
LOG_WARN("failed to check scalar group by pushdown", K(ret));
......
......@@ -487,6 +487,8 @@ public:
can_rollup_pushdown_(false),
force_use_hash_(false),
force_use_merge_(false),
force_part_sort_(false),
force_normal_sort_(false),
is_scalar_group_by_(false),
distinct_exprs_(),
aggr_code_expr_(NULL),
......@@ -506,6 +508,8 @@ public:
bool can_rollup_pushdown_;
bool force_use_hash_; // has use_hash_aggregation/use_hash_distinct hint
bool force_use_merge_; // has no_use_hash_aggregation/no_use_hash_distinct hint
bool force_part_sort_; // force use partition sort for merge group by
bool force_normal_sort_; // disable use partition sort for merge group by
bool is_scalar_group_by_;
ObSEArray<ObRawExpr*, 8> distinct_exprs_;
......
......@@ -809,7 +809,7 @@ int ObLogSet::print_outline_data(PlanText &plan_text)
LOG_WARN("fail to print buffer", K(ret), K(buf), K(buf_len), K(pos));
} else if (OB_FAIL(construct_pq_set_hint(hint))) {
LOG_WARN("fail to construct pq set hint", K(ret));
} else if (hint.get_dist_methods().empty()) {
} else if (hint.get_dist_methods().empty() && hint.get_left_branch().empty()) {
/*do nothing*/
} else if (OB_FALSE_IT(hint.set_qb_name(qb_name))) {
} else if (hint.print_hint(plan_text)) {
......
......@@ -1760,6 +1760,10 @@ int ObLogTableScan::print_used_hint(PlanText &plan_text)
} else if (NULL != table_hint->parallel_hint_ && table_hint->parallel_hint_->get_parallel() > 1
&& OB_FAIL(table_hint->parallel_hint_->print_hint(plan_text))) {
LOG_WARN("failed to print table parallel hint", K(ret));
} else if (NULL != table_hint->use_das_hint_
&& use_das() == table_hint->use_das_hint_->is_enable_hint()
&& OB_FAIL(table_hint->use_das_hint_->print_hint(plan_text))) {
LOG_WARN("failed to print use das hint", K(ret));
} else if (table_hint->index_list_.empty()) {
/*do nothing*/
} else if (OB_UNLIKELY(table_hint->index_list_.count() != table_hint->index_hints_.count())) {
......
......@@ -85,7 +85,10 @@ private:
const GroupingOpHelper &groupby_helper,
const bool ignore_hint,
bool &use_hash_valid,
bool &use_merge_valid);
bool &use_merge_valid,
bool &part_sort_valid,
bool &normal_sort_valid);
int update_part_sort_method(bool &part_sort_valid, bool &normal_sort_valid);
int candi_allocate_normal_group_by(const ObIArray<ObRawExpr*> &reduce_exprs,
const ObIArray<ObRawExpr*> &group_by_exprs,
const ObIArray<ObOrderDirection> &group_directions,
......@@ -160,6 +163,8 @@ private:
GroupingOpHelper &groupby_helper,
CandidatePlan &candidate_plan,
ObIArray<CandidatePlan> &candidate_plans,
bool part_sort_valid,
bool normal_sort_valid,
bool can_ignore_merge = false);
int generate_merge_group_sort_keys(ObLogicalOperator *top,
......@@ -671,9 +676,8 @@ private:
const bool is_from_povit,
GroupingOpHelper &groupby_helper,
ObLogicalOperator *&top,
bool &is_plan_valid,
bool can_ignore_merge = false,
bool use_part_sort = false);
bool use_part_sort,
bool can_ignore_merge = false);
DISALLOW_COPY_AND_ASSIGN(ObSelectLogPlan);
};
......
......@@ -950,6 +950,8 @@ Timestamp{whitespace}?\"[^\"]*\" {
<hint>USE_PLAN_CACHE { return USE_PLAN_CACHE; }
<hint>USE_HASH_AGGREGATION { return USE_HASH_AGGREGATION; }
<hint>NO_USE_HASH_AGGREGATION { return NO_USE_HASH_AGGREGATION; }
<hint>PARTITION_SORT { return PARTITION_SORT; }
<hint>NO_PARTITION_SORT { return NO_PARTITION_SORT; }
<hint>USE_LATE_MATERIALIZATION { return USE_LATE_MATERIALIZATION; }
<hint>NO_USE_LATE_MATERIALIZATION { return NO_USE_LATE_MATERIALIZATION; }
<hint>TRACE_LOG { return TRACE_LOG; }
......
......@@ -171,7 +171,8 @@ INDEX_SS_HINT INDEX_SS_ASC_HINT INDEX_SS_DESC_HINT
LEADING_HINT ORDERED
USE_NL USE_MERGE USE_HASH NO_USE_HASH NO_USE_MERGE NO_USE_NL
USE_NL_MATERIALIZATION NO_USE_NL_MATERIALIZATION
USE_HASH_AGGREGATION NO_USE_HASH_AGGREGATION
USE_HASH_AGGREGATION NO_USE_HASH_AGGREGATION
PARTITION_SORT NO_PARTITION_SORT
USE_LATE_MATERIALIZATION NO_USE_LATE_MATERIALIZATION
PX_JOIN_FILTER NO_PX_JOIN_FILTER PX_PART_JOIN_FILTER NO_PX_PART_JOIN_FILTER
PQ_MAP PQ_DISTRIBUTE PQ_DISTRIBUTE_WINDOW PQ_SET RANDOM_LOCAL BROADCAST BC2HOST LIST
......@@ -8850,6 +8851,17 @@ INDEX_HINT '(' qb_name_option relation_factor_in_hint NAME_OB ')'
| NO_USE_HASH_AGGREGATION opt_qb_name
{
malloc_non_terminal_node($$, result->malloc_pool_, T_NO_USE_HASH_AGGREGATE, 1, $2);
$$->value_ = -1;
}
| NO_USE_HASH_AGGREGATION '(' qb_name_option NO_PARTITION_SORT ')'
{
malloc_non_terminal_node($$, result->malloc_pool_, T_NO_USE_HASH_AGGREGATE, 1, $3);
$$->value_ = 0;
}
| NO_USE_HASH_AGGREGATION '(' qb_name_option PARTITION_SORT ')'
{
malloc_non_terminal_node($$, result->malloc_pool_, T_NO_USE_HASH_AGGREGATE, 1, $3);
$$->value_ = 1;
}
| USE_LATE_MATERIALIZATION opt_qb_name
{
......
......@@ -11982,8 +11982,6 @@ int ObDMLResolver::resolve_optimize_hint(const ParseNode &hint_node,
}
case T_USE_LATE_MATERIALIZATION:
case T_NO_USE_LATE_MATERIALIZATION:
case T_USE_HASH_AGGREGATE:
case T_NO_USE_HASH_AGGREGATE:
case T_GBY_PUSHDOWN:
case T_NO_GBY_PUSHDOWN:
case T_USE_HASH_DISTINCT:
......@@ -11999,6 +11997,13 @@ int ObDMLResolver::resolve_optimize_hint(const ParseNode &hint_node,
}
break;
}
case T_USE_HASH_AGGREGATE:
case T_NO_USE_HASH_AGGREGATE: {
if (OB_FAIL(resolve_aggregation_hint(hint_node, opt_hint))) {
LOG_WARN("failed to resolve aggregation hint.", K(ret));
}
break;
}
case T_TABLE_PARALLEL: { // PARALLEL(qb_name tablespec 4)
if (OB_FAIL(resolve_table_parallel_hint(hint_node, opt_hint))) {
LOG_WARN("fail to resolve parallel in hint", K(ret));
......@@ -12751,6 +12756,34 @@ int ObDMLResolver::resolve_normal_optimize_hint(const ParseNode &hint_node,
return ret;
}
int ObDMLResolver::resolve_aggregation_hint(const ParseNode &hint_node,
ObOptHint *&hint)
{
int ret = OB_SUCCESS;
hint = NULL;
ObAggHint *agg_hint = NULL;
ObString qb_name;
if (OB_UNLIKELY(1 != hint_node.num_child_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("hint with qb name param has no one children.", K(ret));
} else if (OB_FAIL(ObQueryHint::create_hint(allocator_, hint_node.type_, agg_hint))) {
LOG_WARN("failed to create hint", K(ret));
} else if (OB_FAIL(resolve_qb_name_node(hint_node.children_[0], qb_name))) {
LOG_WARN("failed to resolve qb name node.", K(ret));
} else {
hint = agg_hint;
agg_hint->set_qb_name(qb_name);
if (T_NO_USE_HASH_AGGREGATE == hint_node.type_) {
if (1 == hint_node.value_) {
agg_hint->set_use_partition_sort(true);
} else if (0 == hint_node.value_) {
agg_hint->set_use_partition_sort(false);
}
}
}
return ret;
}
int ObDMLResolver::resolve_monitor_ids(const ParseNode &tracing_node,
ObIArray<ObMonitorHint> &monitoring_ids)
{
......
......@@ -840,6 +840,7 @@ private:
int resolve_pq_distribute_hint(const ParseNode &hint_node, ObOptHint *&opt_hint);
int resolve_pq_set_hint(const ParseNode &hint_node, ObOptHint *&opt_hint);
int resolve_join_filter_hint(const ParseNode &join_node, ObOptHint *&opt_hint);
int resolve_aggregation_hint(const ParseNode &hint_node, ObOptHint *&hint);
int resolve_normal_transform_hint(const ParseNode &hint_node, ObTransHint *&hint);
int resolve_normal_optimize_hint(const ParseNode &hint_node, ObOptHint *&hint);
int resolve_view_merge_hint(const ParseNode &hint_node, ObTransHint *&hint);
......
......@@ -2511,5 +2511,29 @@ int ObWindowDistHint::print_hint_desc(PlanText &plan_text) const
return ret;
}
int ObAggHint::assign(const ObAggHint &other)
{
int ret = OB_SUCCESS;
sort_method_valid_ = other.sort_method_valid_;
use_partition_sort_ = other.use_partition_sort_;
return ret;
}
int ObAggHint::print_hint_desc(PlanText &plan_text) const
{
int ret = OB_SUCCESS;
if (sort_method_valid_) {
char *buf = plan_text.buf_;
int64_t &buf_len = plan_text.buf_len_;
int64_t &pos = plan_text.pos_;
if (use_partition_sort_ && OB_FAIL(BUF_PRINTF("PARTITION_SORT"))) {
LOG_WARN("print failed", K(ret));
} else if (!use_partition_sort_ && OB_FAIL(BUF_PRINTF("NO_PARTITION_SORT"))) {
LOG_WARN("print failed", K(ret));
}
}
return ret;
}
}//end of namespace sql
}//end of namespace oceanbase
......@@ -931,6 +931,31 @@ private:
Algos algos_;
};
class ObAggHint : public ObOptHint
{
public:
ObAggHint(ObItemType hint_type)
: ObOptHint(hint_type),
sort_method_valid_(false),
use_partition_sort_(false)
{
}
int assign(const ObAggHint &other);
virtual ~ObAggHint() {}
virtual int print_hint_desc(PlanText &plan_text) const override;
void set_use_partition_sort(bool use_part_sort) { sort_method_valid_ = is_disable_hint(); use_partition_sort_ = sort_method_valid_ && use_part_sort; }
void reset_use_partition_sort() { sort_method_valid_ = false; use_partition_sort_ = false; }
bool force_partition_sort() const { return is_disable_hint() && sort_method_valid_ && use_partition_sort_; }
bool force_normal_sort() const { return is_disable_hint() && sort_method_valid_ && !use_partition_sort_; }
INHERIT_TO_STRING_KV("ObHint", ObHint, K_(sort_method_valid), K_(use_partition_sort));
private:
bool sort_method_valid_;
bool use_partition_sort_;
};
struct ObDDLSchemaVersionHint
{
ObDDLSchemaVersionHint() : schema_version_(0) {}
......
......@@ -1756,6 +1756,32 @@ bool ObLogPlanHint::has_disable_hint(ObItemType hint_type) const
return NULL != cur_hint ? cur_hint->is_disable_hint() : is_outline_data_;
}
int ObLogPlanHint::get_aggregation_info(bool &force_use_hash,
bool &force_use_merge,
bool &force_part_sort,
bool &force_normal_sort) const
{
int ret = OB_SUCCESS;
force_use_hash = false;
force_use_merge = false;
force_part_sort = false;
force_normal_sort = false;
const ObAggHint *agg_hint = static_cast<const ObAggHint*>(get_normal_hint(T_USE_HASH_AGGREGATE));
if (NULL != agg_hint) {
force_use_hash = agg_hint->is_enable_hint();
force_use_merge = agg_hint->is_disable_hint();
force_part_sort = agg_hint->force_partition_sort();
force_normal_sort = agg_hint->force_normal_sort();
if (force_use_merge && !force_part_sort && !force_normal_sort && is_outline_data_) {
force_normal_sort = true;
}
} else if (is_outline_data_) {
force_use_merge = true;
force_normal_sort = true;
}
return ret;
}
const ObWindowDistHint *ObLogPlanHint::get_window_dist() const
{
return static_cast<const ObWindowDistHint*>(
......
......@@ -435,6 +435,10 @@ struct ObLogPlanHint
bool has_disable_hint(ObItemType hint_type) const;
bool use_join_filter(const ObRelIds &table_set) const;
bool no_use_join_filter(const ObRelIds &table_set) const;
int get_aggregation_info(bool &force_use_hash,
bool &force_use_merge,
bool &force_part_sort,
bool &force_normal_sort) const;
bool use_late_material() const { return has_enable_hint(T_USE_LATE_MATERIALIZATION); }
bool no_use_late_material() const { return has_disable_hint(T_USE_LATE_MATERIALIZATION); }
......
......@@ -305,7 +305,7 @@ int ObTransformGroupByPullup::check_groupby_pullup_validity(ObDMLStmt *stmt,
} else if (OB_FAIL(check_hint_valid(*stmt, *table->ref_query_, hint_valid))) {
LOG_WARN("check hint failed", K(ret));
} else if (!hint_valid) {
is_valid = false;
// can not set is_valid as false, may pullup other table
OPT_TRACE("hint reject transform");
} else if (OB_FALSE_IT(myhint = static_cast<const ObViewMergeHint*>(sub_stmt->get_stmt_hint().get_normal_hint(T_MERGE_HINT)))) {
} else if (ignore_tables.has_member(stmt->get_table_bit_index(table->table_id_))) {
......@@ -1071,12 +1071,12 @@ int ObTransformGroupByPullup::need_transform(const common::ObIArray<ObParentDMLS
} else {
need_trans = query_hint->is_valid_outline_transform(ctx_->trans_list_loc_,
get_hint(table->ref_query_->get_stmt_hint()));
if (!need_trans) {
OPT_TRACE("outline reject transform");
}
LOG_DEBUG("need trans pullup0", K(need_trans));
}
}
if (OB_SUCC(ret) && !need_trans) {
OPT_TRACE("outline reject transform");
}
}
LOG_DEBUG("need trans pullup", K(need_trans));
return ret;
......
......@@ -43,21 +43,14 @@ private:
const ObDMLStmt &stmt,
bool &need_trans) override;
int sort_transed_stmts();
int check_outline_valid_to_transform(const ObDMLStmt &stmt, bool &need_trans);
int pullup_predicates(ObDMLStmt *stmt,
ObIArray<int64_t> &select_list,
ObIArray<ObRawExpr *> &properties);
int preprocess(ObDMLStmt &stmt);
int preprocess_semi_info(ObDMLStmt &stmt,
SemiInfo *semi_info,
ObIArray<ObRawExpr*> &upper_conds);
int preprocess_joined_table(ObDMLStmt &stmt,
JoinedTable *join_table,
ObIArray<ObRawExpr*> &upper_conds);
int pullup_predicates_from_view(ObDMLStmt &stmt,
ObIArray<int64_t> &sel_ids,
ObIArray<ObRawExpr *> &input_pullup_preds);
......@@ -93,6 +86,12 @@ private:
ObIArray<int64_t> &sel_ids,
ObIArray<ObRawExpr *> &input_pullup_preds,
ObIArray<ObRawExpr *> &output_pullup_preds);
int gather_pullup_preds_from_semi_outer_join(ObDMLStmt &stmt,
ObIArray<ObRawExpr*> &preds,
bool remove_preds = false);
int gather_pullup_preds_from_join_table(TableItem *table,
ObIArray<ObRawExpr*> &preds,
bool remove_preds);
int remove_pullup_union_predicates(ObIArray<ObRawExpr *> &exprs);
......@@ -220,9 +219,18 @@ private:
ObIArray<ObRawExpr *> &pullup_preds,
ObIArray<ObRawExpr *> &pushdown_preds);
int check_transform_happened(ObDMLStmt *stmt,
ObIArray<ObRawExpr *> &old_conditions,
ObIArray<ObRawExpr *> &new_conditions);
int store_all_preds(const ObDMLStmt &stmt, ObIArray<ObSEArray<ObRawExpr*, 16>> &all_preds);
int store_join_conds(const TableItem *table, ObIArray<ObSEArray<ObRawExpr*, 16>> &all_preds);
int check_transform_happened(const ObDMLStmt &stmt,
const ObIArray<ObSEArray<ObRawExpr*, 16>> &all_preds,
bool &is_happened);
int check_join_conds_deduced(const TableItem *table,
const ObIArray<ObSEArray<ObRawExpr*, 16>> &all_preds,
uint64_t &idx,
bool &is_happened);
int check_conds_deduced(const ObIArray<ObRawExpr *> &old_conditions,
const ObIArray<ObRawExpr *> &new_conditions,
bool &is_happened);
int pushdown_through_winfunc(ObSelectStmt &stmt,
ObIArray<ObRawExpr *> &predicates,
......@@ -277,7 +285,7 @@ private:
ObIArray<int64_t> &sel_items,
ObIArray<ObRawExpr *> &columns);
int create_equal_exprs_for_insert(ObDelUpdStmt *del_upd_stmt, bool &is_happened);
int create_equal_exprs_for_insert(ObDelUpdStmt *del_upd_stmt);
int print_debug_info(const char *str, ObDMLStmt *stmt, ObIArray<ObRawExpr *> &preds);
......@@ -325,6 +333,7 @@ private:
Ob2DArray<PullupPreds *> stmt_pullup_preds_;
ObSEArray<ObDMLStmt *, 8> transed_stmts_;
ObSEArray<ObHint *, 4> applied_hints_;
bool real_happened_;
};
}
......
......@@ -360,15 +360,15 @@ insert into t1 (c1, c2, c3) values (2, 3, 2);
insert into t1 (c1, c2, c3) values (2, 3, 3);
select c2, avg(distinct c3) from t1 group by c2;
c2 avg(distinct c3)
2 2.5000
1 1.5000
2 2.5000
3 2.5000
select c1,c2, group_concat(distinct c3 order by c3 desc) from t1 group by c1,c2;
c1 c2 group_concat(distinct c3 order by c3 desc)
1 1 1
2 3 3,2
2 1 2
1 2 3,2
2 1 2
2 3 3,2
select c1,c2, group_concat(distinct c3 order by c3 desc) from t1 group by c1,c2 with rollup;
c1 c2 group_concat(distinct c3 order by c3 desc)
1 1 1
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册