Fix temp table related bugs
This commit is contained in:
parent
558f74a2d6
commit
88430a9d96
@ -11,6 +11,10 @@
|
||||
*/
|
||||
|
||||
#include "sql/ob_sql_temp_table.h"
|
||||
#include "sql/rewrite/ob_predicate_deduce.h"
|
||||
#include "sql/ob_sql_context.h"
|
||||
#include "sql/optimizer/ob_optimizer_util.h"
|
||||
#define USING_LOG_PREFIX SQL_OPT
|
||||
|
||||
using namespace oceanbase::sql;
|
||||
|
||||
@ -22,3 +26,280 @@ OB_SERIALIZE_MEMBER(ObSqlTempTableCtx,
|
||||
interm_result_infos_,
|
||||
temp_table_id_,
|
||||
is_local_interm_result_);
|
||||
|
||||
int ObSqlTempTableInfo::collect_temp_tables(ObIAllocator &allocator,
|
||||
ObDMLStmt &stmt,
|
||||
ObIArray<ObSqlTempTableInfo*> &temp_table_infos,
|
||||
ObQueryCtx *query_ctx,
|
||||
bool do_collect_filter)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSqlTempTableInfo *temp_table_info = NULL;
|
||||
void *ptr = NULL;
|
||||
TableItem *table = NULL;
|
||||
ObSEArray<ObSelectStmt*, 4> child_stmts;
|
||||
if (OB_FAIL(stmt.get_child_stmts(child_stmts))) {
|
||||
LOG_WARN("failed to get child stmts", K(ret));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < child_stmts.count(); i++) {
|
||||
if (OB_ISNULL(child_stmts.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret));
|
||||
} else if (OB_FAIL(SMART_CALL(collect_temp_tables(allocator, *child_stmts.at(i),
|
||||
temp_table_infos, query_ctx, do_collect_filter)))) {
|
||||
LOG_WARN("failed to add all temp tables", K(ret));
|
||||
}
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < stmt.get_table_items().count(); i++) {
|
||||
bool find = true;
|
||||
if (OB_ISNULL(table = stmt.get_table_items().at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret));
|
||||
} else if (!table->is_temp_table()) {
|
||||
//do nothing
|
||||
} else {
|
||||
find = false;
|
||||
for (int64_t j = 0; OB_SUCC(ret) && !find && j < temp_table_infos.count(); j++) {
|
||||
ObSqlTempTableInfo* info = temp_table_infos.at(j);
|
||||
if (OB_ISNULL(info)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect null info", K(ret));
|
||||
} else if (info->table_query_ == table->ref_query_) {
|
||||
find = true;
|
||||
table->ref_id_ = info->temp_table_id_;
|
||||
|
||||
TableInfo table_info;
|
||||
table_info.upper_stmt_ = &stmt;
|
||||
table_info.table_item_ = table;
|
||||
if (do_collect_filter &&
|
||||
OB_FAIL(collect_temp_table_filters(table_info.upper_stmt_,
|
||||
table_info.table_item_,
|
||||
table_info.table_filters_,
|
||||
table_info.filter_conditions_))) {
|
||||
LOG_WARN("failed to collect temp table info", K(ret));
|
||||
} else if (OB_FAIL(info->table_infos_.push_back(table_info))) {
|
||||
LOG_WARN("failed to push back table info", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && !find) {
|
||||
if (OB_ISNULL(table->ref_query_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret));
|
||||
} else if (OB_FAIL(SMART_CALL(collect_temp_tables(allocator, *table->ref_query_,
|
||||
temp_table_infos, query_ctx, do_collect_filter)))) {
|
||||
LOG_WARN("failed to add all temp tables", K(ret));
|
||||
} else if (OB_ISNULL(ptr = allocator.alloc(sizeof(ObSqlTempTableInfo)))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret));
|
||||
} else {
|
||||
temp_table_info = new (ptr) ObSqlTempTableInfo();
|
||||
table->ref_id_ = (NULL == query_ctx) ? OB_INVALID_ID : query_ctx->available_tb_id_--;
|
||||
temp_table_info->temp_table_id_ = table->ref_id_;
|
||||
temp_table_info->table_name_ = table->table_name_;
|
||||
temp_table_info->table_query_ = table->ref_query_;
|
||||
|
||||
TableInfo table_info;
|
||||
table_info.upper_stmt_ = &stmt;
|
||||
table_info.table_item_ = table;
|
||||
if (do_collect_filter &&
|
||||
OB_FAIL(collect_temp_table_filters(table_info.upper_stmt_,
|
||||
table_info.table_item_,
|
||||
table_info.table_filters_,
|
||||
table_info.filter_conditions_))) {
|
||||
LOG_WARN("failed to collect temp table info", K(ret));
|
||||
} else if (OB_FAIL(temp_table_info->table_infos_.push_back(table_info))) {
|
||||
LOG_WARN("failed to push back table item", K(ret));
|
||||
} else if (OB_FAIL(temp_table_infos.push_back(temp_table_info))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSqlTempTableInfo::collect_temp_table_filters(ObDMLStmt *stmt,
|
||||
TableItem *table,
|
||||
ObIArray<ObRawExpr*> &table_filters,
|
||||
ObIArray<ObRawExprCondition*> &filter_conditions)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSqlBitSet<> table_ids;
|
||||
int32_t table_idx = OB_INVALID_INDEX;
|
||||
uint64_t table_id = OB_INVALID_ID;
|
||||
if (OB_ISNULL(stmt) || OB_ISNULL(table) || OB_ISNULL(table->ref_query_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect null param", K(ret));
|
||||
} else if (OB_FALSE_IT(table_idx = stmt->get_table_bit_index(table->table_id_))) {
|
||||
} else if (OB_FAIL(table_ids.add_member(table_idx))) {
|
||||
LOG_WARN("failed to add member", K(table_idx), K(ret));
|
||||
} else if (OB_FAIL(get_candi_exprs(table_ids,
|
||||
stmt->get_condition_exprs(),
|
||||
table_filters,
|
||||
filter_conditions))) {
|
||||
LOG_WARN("failed to get candi exprs", K(ret));
|
||||
} else {
|
||||
table_id = table->table_id_;
|
||||
}
|
||||
//如果是joined table内部表,如果在左侧,则可以使用where condition、
|
||||
//如果在右侧,则不能使用where condition,选择可以使用的on condition
|
||||
bool find = false;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && !find && i < stmt->get_from_item_size(); ++i) {
|
||||
FromItem &from = stmt->get_from_item(i);
|
||||
if (from.table_id_ == table_id) {
|
||||
find = true;
|
||||
} else if (from.is_joined_) {
|
||||
JoinedTable *joined_table = stmt->get_joined_table(from.table_id_);
|
||||
if (OB_ISNULL(joined_table)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect null table item", K(ret));
|
||||
} else if (!ObOptimizerUtil::find_item(joined_table->single_table_ids_, table_id)) {
|
||||
//do nothing
|
||||
} else if (OB_FAIL(collect_table_filters_in_joined_table(joined_table,
|
||||
table_id,
|
||||
table_ids,
|
||||
table_filters,
|
||||
filter_conditions))) {
|
||||
LOG_WARN("failed to get table filters", K(ret));
|
||||
} else {
|
||||
find = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSqlTempTableInfo::collect_table_filters_in_joined_table(JoinedTable *table,
|
||||
uint64_t table_id,
|
||||
const ObSqlBitSet<> &table_ids,
|
||||
ObIArray<ObRawExpr*> &table_filters,
|
||||
ObIArray<ObRawExprCondition*> &filter_conditions)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSEArray<ObRawExpr*, 8> candi_filters;
|
||||
bool in_left = false;
|
||||
bool in_right = false;
|
||||
if (OB_ISNULL(table) || OB_ISNULL(table->left_table_) ||
|
||||
OB_ISNULL(table->right_table_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect null table item", K(ret));
|
||||
} else if (table->left_table_->is_joined_table()) {
|
||||
JoinedTable *joined_table = static_cast<JoinedTable*>(table->left_table_);
|
||||
if (ObOptimizerUtil::find_item(joined_table->single_table_ids_, table_id)) {
|
||||
in_left = true;
|
||||
}
|
||||
} else if (!table->left_table_->is_joined_table()) {
|
||||
if (table_id == table->left_table_->table_id_) {
|
||||
in_left = true;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && !in_left) {
|
||||
if (table->right_table_->is_joined_table()) {
|
||||
JoinedTable *joined_table = static_cast<JoinedTable*>(table->right_table_);
|
||||
if (ObOptimizerUtil::find_item(joined_table->single_table_ids_, table_id)) {
|
||||
in_right = true;
|
||||
}
|
||||
} else if (!table->right_table_->is_joined_table()) {
|
||||
if (table_id == table->right_table_->table_id_) {
|
||||
in_right = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && in_left) {
|
||||
if (INNER_JOIN == table->joined_type_) {
|
||||
if (OB_FAIL(get_candi_exprs(table_ids,
|
||||
table->join_conditions_,
|
||||
table_filters,
|
||||
filter_conditions))) {
|
||||
LOG_WARN("failed to get candi exprs", K(ret));
|
||||
}
|
||||
} else if (LEFT_OUTER_JOIN == table->joined_type_) {
|
||||
//do nothing
|
||||
} else if (RIGHT_OUTER_JOIN == table->joined_type_) {
|
||||
table_filters.reuse();
|
||||
filter_conditions.reuse();
|
||||
if (OB_FAIL(get_candi_exprs(table_ids,
|
||||
table->join_conditions_,
|
||||
table_filters,
|
||||
filter_conditions))) {
|
||||
LOG_WARN("failed to get candi exprs", K(ret));
|
||||
}
|
||||
} else {
|
||||
table_filters.reuse();
|
||||
filter_conditions.reuse();
|
||||
}
|
||||
if (OB_SUCC(ret) && table->left_table_->is_joined_table()) {
|
||||
JoinedTable *joined_table = static_cast<JoinedTable*>(table->left_table_);
|
||||
if (OB_FAIL(SMART_CALL(collect_table_filters_in_joined_table(joined_table,
|
||||
table_id,
|
||||
table_ids,
|
||||
table_filters,
|
||||
filter_conditions)))) {
|
||||
LOG_WARN("failed to get table filters", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && in_right) {
|
||||
if (INNER_JOIN == table->joined_type_) {
|
||||
if (OB_FAIL(get_candi_exprs(table_ids,
|
||||
table->join_conditions_,
|
||||
table_filters,
|
||||
filter_conditions))) {
|
||||
LOG_WARN("failed to get candi exprs", K(ret));
|
||||
}
|
||||
} else if (LEFT_OUTER_JOIN == table->joined_type_) {
|
||||
table_filters.reuse();
|
||||
filter_conditions.reuse();
|
||||
if (OB_FAIL(get_candi_exprs(table_ids,
|
||||
table->join_conditions_,
|
||||
table_filters,
|
||||
filter_conditions))) {
|
||||
LOG_WARN("failed to get candi exprs", K(ret));
|
||||
}
|
||||
} else if (RIGHT_OUTER_JOIN == table->joined_type_) {
|
||||
//do nothing
|
||||
} else {
|
||||
table_filters.reuse();
|
||||
filter_conditions.reuse();
|
||||
}
|
||||
if (OB_SUCC(ret) && table->right_table_->is_joined_table()) {
|
||||
JoinedTable *joined_table = static_cast<JoinedTable*>(table->right_table_);
|
||||
if (OB_FAIL(SMART_CALL(collect_table_filters_in_joined_table(joined_table,
|
||||
table_id,
|
||||
table_ids,
|
||||
table_filters,
|
||||
filter_conditions)))) {
|
||||
LOG_WARN("failed to get table filters", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSqlTempTableInfo::get_candi_exprs(const ObSqlBitSet<> &table_ids,
|
||||
ObIArray<ObRawExpr*> &exprs,
|
||||
ObIArray<ObRawExpr*> &candi_exprs,
|
||||
ObIArray<ObRawExprCondition*> &candi_conditions)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); ++i) {
|
||||
ObRawExpr *expr = exprs.at(i);
|
||||
if (OB_ISNULL(expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect null expr", K(ret));
|
||||
} else if (ObPredicateDeduce::contain_special_expr(*expr)) {
|
||||
// do nothing
|
||||
} else if (expr->has_flag(CNT_DYNAMIC_PARAM)) {
|
||||
//do nothing
|
||||
} else if (!expr->get_relation_ids().is_subset(table_ids)) {
|
||||
//do nothing
|
||||
} else if (OB_FAIL(candi_exprs.push_back(expr))) {
|
||||
LOG_WARN("failed to push back expr", K(ret));
|
||||
} else if (OB_FAIL(candi_conditions.push_back(&exprs))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
@ -59,6 +59,27 @@ public:
|
||||
bool is_local_interm_result_;
|
||||
};
|
||||
|
||||
typedef ObIArray<ObRawExpr *> ObRawExprCondition;
|
||||
struct TableInfo {
|
||||
TableInfo()
|
||||
:table_filters_(),
|
||||
table_item_(NULL),
|
||||
upper_stmt_(NULL)
|
||||
{}
|
||||
|
||||
virtual ~TableInfo(){}
|
||||
|
||||
TO_STRING_KV(
|
||||
K_(table_filters),
|
||||
K_(table_item),
|
||||
K_(upper_stmt)
|
||||
);
|
||||
ObSEArray<ObRawExpr *, 4, common::ModulePageAllocator, true> table_filters_;
|
||||
ObSEArray<ObRawExprCondition *, 4, common::ModulePageAllocator, true> filter_conditions_;
|
||||
TableItem *table_item_;
|
||||
ObDMLStmt* upper_stmt_;
|
||||
};
|
||||
|
||||
class ObSqlTempTableInfo
|
||||
{
|
||||
public:
|
||||
@ -69,13 +90,34 @@ public:
|
||||
virtual ~ObSqlTempTableInfo() {}
|
||||
|
||||
TO_STRING_KV(K_(temp_table_id),
|
||||
K_(table_name));
|
||||
K_(table_name),
|
||||
K_(table_infos));
|
||||
|
||||
static int collect_temp_tables(ObIAllocator &allocator,
|
||||
ObDMLStmt &stmt,
|
||||
ObIArray<ObSqlTempTableInfo*> &temp_table_infos,
|
||||
ObQueryCtx *query_ctx,
|
||||
bool do_collect_filter);
|
||||
static int collect_temp_table_filters(ObDMLStmt *stmt,
|
||||
TableItem *table,
|
||||
ObIArray<ObRawExpr*> &table_filters,
|
||||
ObIArray<ObRawExprCondition*> &filter_conditions);
|
||||
static int collect_table_filters_in_joined_table(JoinedTable *table,
|
||||
uint64_t table_id,
|
||||
const ObSqlBitSet<> &table_ids,
|
||||
ObIArray<ObRawExpr*> &table_filters,
|
||||
ObIArray<ObRawExprCondition*> &filter_conditions);
|
||||
static int get_candi_exprs(const ObSqlBitSet<> &table_ids,
|
||||
ObIArray<ObRawExpr*> &exprs,
|
||||
ObIArray<ObRawExpr*> &candi_exprs,
|
||||
ObIArray<ObRawExprCondition*> &candi_conditions);
|
||||
public:
|
||||
uint64_t temp_table_id_;
|
||||
common::ObString table_name_;
|
||||
ObSelectStmt *table_query_;
|
||||
ObLogicalOperator *table_plan_;
|
||||
|
||||
ObSEArray<TableInfo, 8, common::ModulePageAllocator, true> table_infos_;
|
||||
};
|
||||
|
||||
} /* ns sql*/
|
||||
|
@ -7461,22 +7461,15 @@ int ObJoinOrder::generate_normal_subquery_paths()
|
||||
} else if (OB_ISNULL(helper.child_stmt_ = table_item->ref_query_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("stmt is null", K(ret));
|
||||
} else if (OB_FAIL(ObOptimizerUtil::pushdown_filter_into_subquery(*parent_stmt,
|
||||
*helper.child_stmt_,
|
||||
get_plan()->get_optimizer_context(),
|
||||
get_restrict_infos(),
|
||||
candi_pushdown_quals,
|
||||
helper.filters_,
|
||||
can_pushdown))) {
|
||||
LOG_WARN("failed to pushdown filter into subquery", K(ret));
|
||||
} else if (OB_FAIL(ObOptimizerUtil::rename_pushdown_filter(*parent_stmt,
|
||||
*helper.child_stmt_,
|
||||
table_id_,
|
||||
session_info,
|
||||
*expr_factory,
|
||||
candi_pushdown_quals,
|
||||
helper.pushdown_filters_))) {
|
||||
LOG_WARN("failed to rename pushdown filter", K(ret));
|
||||
} else if (OB_FAIL(ObOptimizerUtil::pushdown_and_rename_filter_into_subquery(*parent_stmt,
|
||||
*helper.child_stmt_,
|
||||
table_id_,
|
||||
get_plan()->get_optimizer_context(),
|
||||
get_restrict_infos(),
|
||||
helper.pushdown_filters_,
|
||||
helper.filters_,
|
||||
/*check_match_index*/false))) {
|
||||
LOG_WARN("failed to push down filter into subquery", K(ret));
|
||||
} else if (OB_FAIL(generate_subquery_paths(helper))) {
|
||||
LOG_WARN("failed to generate subquery path", K(ret));
|
||||
}
|
||||
|
@ -10368,6 +10368,9 @@ int ObLogPlan::add_candidate_plan(ObIArray<CandidatePlan> ¤t_plans,
|
||||
should_add = false;
|
||||
OPT_TRACE("containt match all fake cte, but not remote plan, will not add plan");
|
||||
}
|
||||
} else if (new_plan.plan_tree_->get_contains_match_all_fake_cte() &&
|
||||
!new_plan.plan_tree_->is_remote()) {
|
||||
should_add = false;
|
||||
}
|
||||
for (int64_t i = current_plans.count() - 1;
|
||||
OB_SUCC(ret) && should_add && i >= 0; --i) {
|
||||
|
@ -25,6 +25,7 @@
|
||||
#include "sql/engine/cmd/ob_table_direct_insert_service.h"
|
||||
#include "sql/dblink/ob_dblink_utils.h"
|
||||
#include "sql/resolver/dml/ob_merge_stmt.h"
|
||||
#include "sql/optimizer/ob_log_temp_table_insert.h"
|
||||
using namespace oceanbase;
|
||||
using namespace sql;
|
||||
using namespace oceanbase::common;
|
||||
@ -114,11 +115,74 @@ int ObOptimizer::get_optimization_cost(ObDMLStmt &stmt,
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizer::get_cte_optimization_cost(ObDMLStmt &root_stmt,
|
||||
ObSelectStmt *cte_query,
|
||||
ObIArray<ObSelectStmt *> &stmts,
|
||||
double &cte_cost,
|
||||
ObIArray<double> &costs)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
cte_cost = 0.0;
|
||||
ctx_.set_cost_evaluation();
|
||||
costs.reuse();
|
||||
if (OB_ISNULL(ctx_.get_query_ctx()) ||
|
||||
OB_ISNULL(ctx_.get_session_info())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("query ctx or session info is null", K(ret));
|
||||
} else if (OB_FAIL(init_env_info(root_stmt))) {
|
||||
LOG_WARN("failed to init env info", K(ret));
|
||||
} else if (OB_FAIL(generate_plan_for_temp_table(root_stmt))) {
|
||||
LOG_WARN("failed to generate plan for temp table", K(ret));
|
||||
}
|
||||
if (OB_SUCC(ret) && NULL != cte_query) {
|
||||
bool find = false;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && !find && i < ctx_.get_temp_table_infos().count(); i ++) {
|
||||
ObSqlTempTableInfo *temp_table_info = ctx_.get_temp_table_infos().at(i);
|
||||
ObLogPlan *plan = NULL;
|
||||
if (OB_ISNULL(temp_table_info) || OB_ISNULL(temp_table_info->table_plan_)
|
||||
|| OB_ISNULL(plan = temp_table_info->table_plan_->get_plan())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null", K(ret));
|
||||
} else if (cte_query == temp_table_info->table_query_) {
|
||||
find = true;
|
||||
if (OB_FAIL(plan->allocate_temp_table_insert_as_top(temp_table_info->table_plan_,
|
||||
temp_table_info))) {
|
||||
LOG_WARN("failed to allocate temp table insert", K(ret));
|
||||
} else {
|
||||
cte_cost = temp_table_info->table_plan_->get_cost();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < stmts.count(); i ++) {
|
||||
ObLogPlan *plan = NULL;
|
||||
ObSelectStmt *stmt = stmts.at(i);
|
||||
ObLogicalOperator *best_plan = NULL;
|
||||
if (OB_ISNULL(stmt)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null", K(ret));
|
||||
} else if (OB_ISNULL(plan = ctx_.get_log_plan_factory().create(ctx_, *stmt))) {
|
||||
ret = OB_ALLOCATE_MEMORY_FAILED;
|
||||
LOG_ERROR("failed to create plan", "stmt", ctx_.get_query_ctx()->get_sql_stmt(), K(ret));
|
||||
} else if (OB_FAIL(plan->generate_raw_plan())) {
|
||||
LOG_WARN("failed to perform optimization", K(ret));
|
||||
} else if (OB_FAIL(plan->init_candidate_plans(plan->get_candidate_plans().candidate_plans_))) {
|
||||
LOG_WARN("failed to do candi into", K(ret));
|
||||
} else if (OB_FAIL(plan->get_candidate_plans().get_best_plan(best_plan))) {
|
||||
LOG_WARN("failed to get best plan", K(ret));
|
||||
} else if (OB_FAIL(costs.push_back(best_plan->get_cost()))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizer::generate_plan_for_temp_table(ObDMLStmt &stmt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObIArray<ObSqlTempTableInfo*> &temp_table_infos = ctx_.get_temp_table_infos();
|
||||
if (OB_FAIL(collect_temp_tables(ctx_.get_allocator(), stmt, temp_table_infos))) {
|
||||
if (OB_FAIL(ObSqlTempTableInfo::collect_temp_tables(ctx_.get_allocator(), stmt, temp_table_infos,
|
||||
ctx_.get_query_ctx(), true))) {
|
||||
LOG_WARN("failed to add all temp tables", K(ret));
|
||||
} else if (temp_table_infos.empty()) {
|
||||
//do nothing
|
||||
@ -128,6 +192,9 @@ int ObOptimizer::generate_plan_for_temp_table(ObDMLStmt &stmt)
|
||||
ObSelectLogPlan *temp_plan = NULL;
|
||||
ObLogicalOperator *temp_op = NULL;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < temp_table_infos.count(); i++) {
|
||||
ObRawExpr *temp_table_nonwhere_filter = NULL;
|
||||
ObRawExpr *temp_table_where_filter = NULL;
|
||||
bool can_push_to_where = true;
|
||||
if (OB_ISNULL(temp_table_info = temp_table_infos.at(i)) ||
|
||||
OB_ISNULL(ref_query = temp_table_info->table_query_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
@ -141,6 +208,13 @@ int ObOptimizer::generate_plan_for_temp_table(ObDMLStmt &stmt)
|
||||
OPT_TRACE_TITLE("begin generate plan for temp table ", temp_table_info->table_name_);
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(try_push_down_temp_table_filter(*temp_table_info,
|
||||
temp_table_nonwhere_filter,
|
||||
temp_table_where_filter))) {
|
||||
LOG_WARN("failed to push down filter for temp table", K(ret));
|
||||
} else if (NULL != temp_table_where_filter &&
|
||||
OB_FAIL(temp_plan->get_pushdown_filters().push_back(temp_table_where_filter))) {
|
||||
LOG_WARN("failed to push down filter", K(ret));
|
||||
} else if (OB_FAIL(temp_plan->generate_raw_plan())) {
|
||||
LOG_WARN("Failed to generate temp_plan for sub_stmt", K(ret));
|
||||
} else if (OB_FAIL(temp_plan->get_candidate_plans().get_best_plan(temp_op))) {
|
||||
@ -148,6 +222,9 @@ int ObOptimizer::generate_plan_for_temp_table(ObDMLStmt &stmt)
|
||||
} else if (OB_ISNULL(temp_op)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret));
|
||||
} else if (NULL != temp_table_nonwhere_filter &&
|
||||
OB_FAIL(temp_op->get_filter_exprs().push_back(temp_table_nonwhere_filter))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else {
|
||||
temp_table_info->table_plan_ = temp_op;
|
||||
OPT_TRACE_TITLE("end generate plan for temp table ", temp_table_info->table_name_);
|
||||
@ -157,72 +234,131 @@ int ObOptimizer::generate_plan_for_temp_table(ObDMLStmt &stmt)
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizer::collect_temp_tables(ObIAllocator &allocator,
|
||||
ObDMLStmt &stmt,
|
||||
ObIArray<ObSqlTempTableInfo*> &temp_table_infos)
|
||||
/**
|
||||
* If every appearance of cte is accompanied by some filter,
|
||||
* We can combine these filters to reduce the data materialized by cte.
|
||||
* We try to push these filters to where condition.
|
||||
* If all filters can be push to where condition, nonwhere_filter will be NULL.
|
||||
* Otherwise, we might have both where_filter and nonwhere_filter.
|
||||
* e.g.
|
||||
* with cte as (select a,count(*) as cnt from t1 group by a)
|
||||
* select * from cte where a = 1 and cnt = 1 union all select * from cte where a = 2 and cnt = 2;
|
||||
* nonwhere_filter : (a = 1 and cnt = 1) or (a = 2 and cnt = 2)
|
||||
* where_filter : (a = 1) or (a = 2)
|
||||
*/
|
||||
int ObOptimizer::try_push_down_temp_table_filter(ObSqlTempTableInfo &info,
|
||||
ObRawExpr *&nonwhere_filter,
|
||||
ObRawExpr *&where_filter)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSqlTempTableInfo *temp_table_info = NULL;
|
||||
void *ptr = NULL;
|
||||
TableItem *table = NULL;
|
||||
ObSEArray<ObSelectStmt*, 4> child_stmts;
|
||||
if (OB_ISNULL(ctx_.get_query_ctx())) {
|
||||
bool have_filter = true;
|
||||
nonwhere_filter = NULL;
|
||||
where_filter = NULL;
|
||||
ObSEArray<ObIArray<ObRawExpr *> *, 4> temp_table_filters;
|
||||
ObSEArray<const ObDMLStmt *, 4> parent_stmts;
|
||||
ObSEArray<const ObSelectStmt *, 4> subqueries;
|
||||
ObSEArray<int64_t, 4> table_ids;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && have_filter && i < info.table_infos_.count(); ++i) {
|
||||
TableItem *table = info.table_infos_.at(i).table_item_;
|
||||
ObDMLStmt *stmt = info.table_infos_.at(i).upper_stmt_;
|
||||
if (OB_ISNULL(table) || OB_ISNULL(stmt)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null", K(ret));
|
||||
} else if (info.table_infos_.at(i).table_filters_.empty()) {
|
||||
have_filter = false;
|
||||
} else if (OB_FAIL(temp_table_filters.push_back(&info.table_infos_.at(i).table_filters_))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else if (OB_FAIL(parent_stmts.push_back(stmt))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else if (OB_FAIL(subqueries.push_back(table->ref_query_))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else if (OB_FAIL(table_ids.push_back(table->table_id_))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && have_filter) {
|
||||
OPT_TRACE("pushdown filter into temp table:", info.table_query_);
|
||||
bool can_push_all = false;
|
||||
if (OB_FAIL(ObOptimizerUtil::split_or_filter_into_subquery(parent_stmts,
|
||||
subqueries,
|
||||
table_ids,
|
||||
temp_table_filters,
|
||||
ctx_,
|
||||
where_filter,
|
||||
can_push_all,
|
||||
/*check_match_index = */false))) {
|
||||
LOG_WARN("failed to split filter", K(ret));
|
||||
} else if (can_push_all) {
|
||||
// do nothing
|
||||
} else if (OB_FAIL(push_down_temp_table_filter(info, nonwhere_filter))) {
|
||||
LOG_WARN("failed to push down remain temp table filter", K(ret));
|
||||
}
|
||||
if (NULL != where_filter) {
|
||||
OPT_TRACE("succeed to pushdown filter to where:", where_filter);
|
||||
}
|
||||
if (NULL != nonwhere_filter) {
|
||||
OPT_TRACE("succeed to pushdown filter into the top of temp table:", nonwhere_filter);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizer::push_down_temp_table_filter(ObSqlTempTableInfo &info,
|
||||
ObRawExpr *&temp_table_filter)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObRawExprFactory &expr_factory = ctx_.get_expr_factory();
|
||||
ObSQLSessionInfo *session_info = NULL;
|
||||
temp_table_filter = NULL;
|
||||
ObSEArray<ObRawExpr *, 8> and_exprs;
|
||||
ObRawExpr *or_expr = NULL;
|
||||
bool have_temp_table_filter = true;
|
||||
if (OB_ISNULL(session_info = ctx_.get_session_info()) ||
|
||||
OB_ISNULL(info.table_query_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret));
|
||||
} else if (OB_FAIL(stmt.get_child_stmts(child_stmts))) {
|
||||
LOG_WARN("failed to get child stmts", K(ret));
|
||||
LOG_WARN("unexpect null param", K(session_info), K(ret));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < child_stmts.count(); i++) {
|
||||
if (OB_ISNULL(child_stmts.at(i))) {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && have_temp_table_filter && i < info.table_infos_.count(); ++i) {
|
||||
have_temp_table_filter &= !info.table_infos_.at(i).table_filters_.empty();
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && have_temp_table_filter && i < info.table_infos_.count(); ++i) {
|
||||
ObDMLStmt *upper_stmt = info.table_infos_.at(i).upper_stmt_;
|
||||
TableItem *table = info.table_infos_.at(i).table_item_;
|
||||
ObIArray<ObRawExpr *> &table_filters = info.table_infos_.at(i).table_filters_;
|
||||
ObSEArray<ObRawExpr *, 8> rename_exprs;
|
||||
ObRawExpr *and_expr = NULL;
|
||||
if (table_filters.empty() || OB_ISNULL(upper_stmt) ||
|
||||
OB_ISNULL(table)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret));
|
||||
} else if (OB_FAIL(SMART_CALL(collect_temp_tables(allocator, *child_stmts.at(i),
|
||||
temp_table_infos)))) {
|
||||
LOG_WARN("failed to add all temp tables", K(ret));
|
||||
LOG_WARN("unexpect null table info", K(ret));
|
||||
} else if (OB_FAIL(ObOptimizerUtil::rename_pushdown_filter(*upper_stmt,
|
||||
*info.table_query_,
|
||||
table->table_id_,
|
||||
session_info,
|
||||
expr_factory,
|
||||
table_filters,
|
||||
rename_exprs))) {
|
||||
LOG_WARN("failed to rename push down preds", K(ret));
|
||||
} else if (OB_FAIL(ObRawExprUtils::build_and_expr(expr_factory,
|
||||
rename_exprs,
|
||||
and_expr))) {
|
||||
LOG_WARN("failed to build and expr", K(ret));
|
||||
}
|
||||
if (OB_SUCC(ret) && OB_FAIL(and_exprs.push_back(and_expr))) {
|
||||
LOG_WARN("failed to push back expr", K(ret));
|
||||
}
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < stmt.get_table_items().count(); i++) {
|
||||
bool find = true;
|
||||
if (OB_ISNULL(table = stmt.get_table_items().at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret));
|
||||
} else if (!table->is_temp_table()) {
|
||||
//do nothing
|
||||
} else {
|
||||
ObIArray<ObSqlTempTableInfo*> &temp_table_infos_ = ctx_.get_temp_table_infos();
|
||||
find = false;
|
||||
for (int64_t j = 0; OB_SUCC(ret) && !find && j < temp_table_infos_.count(); j++) {
|
||||
ObSqlTempTableInfo* info = temp_table_infos_.at(j);
|
||||
if (OB_ISNULL(info)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect null info", K(ret));
|
||||
} else if (info->table_query_ == table->ref_query_) {
|
||||
find = true;
|
||||
table->ref_id_ = info->temp_table_id_;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && !find) {
|
||||
if (OB_ISNULL(table->ref_query_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret));
|
||||
} else if (OB_FAIL(SMART_CALL(collect_temp_tables(allocator, *table->ref_query_,
|
||||
temp_table_infos)))) {
|
||||
LOG_WARN("failed to add all temp tables", K(ret));
|
||||
} else if (OB_ISNULL(ptr = allocator.alloc(sizeof(ObSqlTempTableInfo)))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret));
|
||||
} else {
|
||||
temp_table_info = new (ptr) ObSqlTempTableInfo();
|
||||
table->ref_id_ = ctx_.get_query_ctx()->available_tb_id_--;
|
||||
temp_table_info->temp_table_id_ = table->ref_id_;
|
||||
temp_table_info->table_name_ = table->table_name_;
|
||||
temp_table_info->table_query_ = table->ref_query_;
|
||||
if (OB_FAIL(temp_table_infos.push_back(temp_table_info))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret) || !have_temp_table_filter) {
|
||||
} else if (OB_FAIL(ObRawExprUtils::build_or_exprs(expr_factory,
|
||||
and_exprs,
|
||||
or_expr))) {
|
||||
LOG_WARN("failed to build or expr", K(ret));
|
||||
} else if (OB_FAIL(or_expr->formalize(session_info))) {
|
||||
LOG_WARN("failed to formalize expr", K(ret));
|
||||
} else if (OB_FAIL(or_expr->pull_relation_id())) {
|
||||
LOG_WARN("failed to pull relation id and levels", K(ret));
|
||||
} else {
|
||||
temp_table_filter = or_expr;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
@ -183,6 +183,11 @@ namespace sql
|
||||
virtual int get_optimization_cost(ObDMLStmt &stmt,
|
||||
ObLogPlan *&plan,
|
||||
double &cost);
|
||||
virtual int get_cte_optimization_cost(ObDMLStmt &root_stmt,
|
||||
ObSelectStmt *cte_query,
|
||||
ObIArray<ObSelectStmt *> &stmts,
|
||||
double &cte_cost,
|
||||
ObIArray<double> &costs);
|
||||
int update_column_usage_infos();
|
||||
private:
|
||||
int generate_plan_for_temp_table(ObDMLStmt &stmt);
|
||||
@ -217,6 +222,13 @@ namespace sql
|
||||
int check_whether_contain_nested_sql(const ObDMLStmt &stmt);
|
||||
int check_force_default_stat();
|
||||
int calc_link_stmt_count(const ObDMLStmt &stmt, int64_t &count);
|
||||
|
||||
int try_push_down_temp_table_filter(ObSqlTempTableInfo &temp_table_info,
|
||||
ObRawExpr *&temp_table_filter,
|
||||
ObRawExpr *&where_filter);
|
||||
int push_down_temp_table_filter(ObSqlTempTableInfo &temp_table_info,
|
||||
ObRawExpr *&temp_table_filter);
|
||||
|
||||
private:
|
||||
ObOptimizerContext &ctx_;
|
||||
DISALLOW_COPY_AND_ASSIGN(ObOptimizer);
|
||||
|
@ -6393,7 +6393,8 @@ int ObOptimizerUtil::pushdown_filter_into_subquery(const ObDMLStmt &parent_stmt,
|
||||
ObIArray<ObRawExpr*> &pushdown_filters,
|
||||
ObIArray<ObRawExpr*> &candi_filters,
|
||||
ObIArray<ObRawExpr*> &remain_filters,
|
||||
bool &can_pushdown)
|
||||
bool &can_pushdown,
|
||||
bool check_match_index/* = true*/)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool has_rownum = false;
|
||||
@ -6407,7 +6408,8 @@ int ObOptimizerUtil::pushdown_filter_into_subquery(const ObDMLStmt &parent_stmt,
|
||||
opt_ctx,
|
||||
pushdown_filters,
|
||||
candi_filters,
|
||||
remain_filters))) {
|
||||
remain_filters,
|
||||
check_match_index))) {
|
||||
LOG_WARN("failed to check pushdown filter", K(ret));
|
||||
} else if (candi_filters.empty()) {
|
||||
//do thing
|
||||
@ -6424,6 +6426,8 @@ int ObOptimizerUtil::pushdown_filter_into_subquery(const ObDMLStmt &parent_stmt,
|
||||
LOG_WARN("failed to check stmt has rownum", K(ret));
|
||||
} else if (subquery.has_limit() || subquery.has_sequence() ||
|
||||
subquery.is_contains_assignment() ||
|
||||
subquery.is_unpivot_select() ||
|
||||
subquery.is_dblink_stmt() ||
|
||||
has_rollup || has_rownum) {
|
||||
//can not pushdown do nothing
|
||||
} else if (OB_FAIL(check_pushdown_filter(parent_stmt,
|
||||
@ -6431,7 +6435,8 @@ int ObOptimizerUtil::pushdown_filter_into_subquery(const ObDMLStmt &parent_stmt,
|
||||
opt_ctx,
|
||||
pushdown_filters,
|
||||
candi_filters,
|
||||
remain_filters))) {
|
||||
remain_filters,
|
||||
check_match_index))) {
|
||||
LOG_WARN("failed to check pushdown filter", K(ret));
|
||||
} else if (candi_filters.empty()) {
|
||||
//do thing
|
||||
@ -6450,19 +6455,24 @@ int ObOptimizerUtil::check_pushdown_filter(const ObDMLStmt &parent_stmt,
|
||||
ObOptimizerContext &opt_ctx,
|
||||
ObIArray<ObRawExpr*> &pushdown_filters,
|
||||
ObIArray<ObRawExpr*> &candi_filters,
|
||||
ObIArray<ObRawExpr*> &remain_filters)
|
||||
ObIArray<ObRawExpr*> &remain_filters,
|
||||
bool check_match_index/* = true*/)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool is_valid = false;
|
||||
ObSEArray<ObRawExpr *, 4> common_exprs;
|
||||
if (!parent_stmt.is_set_stmt() && subquery.is_set_stmt()) {
|
||||
//如果子查询是set stmt,只需要检查是否有下推的谓词overlap index
|
||||
if (OB_FAIL(check_pushdown_filter_overlap_index(parent_stmt,
|
||||
opt_ctx,
|
||||
pushdown_filters,
|
||||
candi_filters,
|
||||
remain_filters))) {
|
||||
LOG_WARN("failed to check pushdown filter overlap index", K(ret));
|
||||
if (check_match_index) {
|
||||
if (OB_FAIL(check_pushdown_filter_overlap_index(parent_stmt,
|
||||
opt_ctx,
|
||||
pushdown_filters,
|
||||
candi_filters,
|
||||
remain_filters))) {
|
||||
LOG_WARN("failed to check pushdown filter overlap index", K(ret));
|
||||
}
|
||||
} else {
|
||||
ret = candi_filters.assign(pushdown_filters);
|
||||
}
|
||||
} else if (OB_FAIL(get_groupby_win_func_common_exprs(subquery,
|
||||
common_exprs,
|
||||
@ -6486,7 +6496,8 @@ int ObOptimizerUtil::check_pushdown_filter(const ObDMLStmt &parent_stmt,
|
||||
common_exprs,
|
||||
pushdown_filters,
|
||||
candi_filters,
|
||||
remain_filters))) {
|
||||
remain_filters,
|
||||
check_match_index))) {
|
||||
LOG_WARN("failed to check pushdown filter for subquery", K(ret));
|
||||
}
|
||||
}
|
||||
@ -6604,7 +6615,8 @@ int ObOptimizerUtil::check_pushdown_filter_for_subquery(const ObDMLStmt &parent_
|
||||
ObIArray<ObRawExpr*> &common_exprs,
|
||||
ObIArray<ObRawExpr*> &pushdown_filters,
|
||||
ObIArray<ObRawExpr*> &candi_filters,
|
||||
ObIArray<ObRawExpr*> &remain_filters)
|
||||
ObIArray<ObRawExpr*> &remain_filters,
|
||||
bool check_match_index/* = true*/)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (parent_stmt.is_set_stmt()) {
|
||||
@ -6644,7 +6656,8 @@ int ObOptimizerUtil::check_pushdown_filter_for_subquery(const ObDMLStmt &parent_
|
||||
expr->has_flag(CNT_AGG) ||
|
||||
expr->has_flag(CNT_SUB_QUERY)) {
|
||||
is_simple_expr = false;
|
||||
} else if (OB_FAIL(ObTransformUtils::check_column_match_index(opt_ctx.get_root_stmt(),
|
||||
} else if (check_match_index &&
|
||||
OB_FAIL(ObTransformUtils::check_column_match_index(opt_ctx.get_root_stmt(),
|
||||
&parent_stmt,
|
||||
opt_ctx.get_sql_schema_guard(),
|
||||
static_cast<ObColumnRefRawExpr*>(column_exprs.at(j)),
|
||||
@ -6675,7 +6688,7 @@ int ObOptimizerUtil::check_pushdown_filter_for_subquery(const ObDMLStmt &parent_
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
//do nothing
|
||||
} else if (!is_match_index) {
|
||||
} else if (check_match_index && !is_match_index) {
|
||||
candi_filters.reset();
|
||||
ret = remain_filters.assign(pushdown_filters);
|
||||
}
|
||||
@ -8720,3 +8733,236 @@ int ObOptimizerUtil::is_in_range_optimization_enabled(const ObGlobalHint &global
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerUtil::pushdown_and_rename_filter_into_subquery(const ObDMLStmt &parent_stmt,
|
||||
const ObSelectStmt &subquery,
|
||||
int64_t table_id,
|
||||
ObOptimizerContext &opt_ctx,
|
||||
ObIArray<ObRawExpr *> &input_filters,
|
||||
ObIArray<ObRawExpr *> &push_filters,
|
||||
ObIArray<ObRawExpr *> &remain_filters,
|
||||
bool check_match_index)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSEArray<ObRawExpr *, 4> candi_filters;
|
||||
bool can_pushdown = false;
|
||||
ObSQLSessionInfo *session_info = opt_ctx.get_session_info();
|
||||
ObRawExprFactory &expr_factory = opt_ctx.get_expr_factory();
|
||||
if (!input_filters.empty() &&
|
||||
OB_FAIL(pushdown_filter_into_subquery(parent_stmt,
|
||||
subquery,
|
||||
opt_ctx,
|
||||
input_filters,
|
||||
candi_filters,
|
||||
remain_filters,
|
||||
can_pushdown,
|
||||
check_match_index))) {
|
||||
LOG_WARN("pushdown filters into left query failed", K(ret));
|
||||
} else if (!candi_filters.empty() &&
|
||||
OB_FAIL(rename_pushdown_filter(parent_stmt, subquery,
|
||||
table_id, session_info,
|
||||
expr_factory,
|
||||
candi_filters,
|
||||
push_filters))) {
|
||||
LOG_WARN("failed to rename pushdown filter", K(ret));
|
||||
}
|
||||
for (int64_t i = 0 ; OB_SUCC(ret) && i < remain_filters.count(); i ++){
|
||||
ObRawExpr *part_push_filter = NULL;
|
||||
bool can_pushdown_all = false;
|
||||
if (OB_FAIL(split_or_filter_into_subquery(parent_stmt,
|
||||
subquery,
|
||||
table_id,
|
||||
opt_ctx,
|
||||
remain_filters.at(i),
|
||||
part_push_filter,
|
||||
can_pushdown_all,
|
||||
check_match_index))) {
|
||||
LOG_WARN("failed to push part of the filter", K(ret));
|
||||
} else if (OB_UNLIKELY(can_pushdown_all)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("can not push the whole remain filter", K(ret), KPC(remain_filters.at(i)));
|
||||
} else if (NULL != part_push_filter &&
|
||||
OB_FAIL(push_filters.push_back(part_push_filter))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
/**
|
||||
* If the filter is an or predicate,
|
||||
* we try to push part of it into the subquery
|
||||
* e.g.
|
||||
* select * from (select a, count(*) as cnt from t1 group by a) v where a = 1 and cnt = 1 or a = 2;
|
||||
* We can not push the whole filter `a = 1 and cnt = 1 or a = 2`,
|
||||
* But we can push `a = 1 or a = 2` into the subquery.
|
||||
*/
|
||||
int ObOptimizerUtil::split_or_filter_into_subquery(const ObDMLStmt &parent_stmt,
|
||||
const ObSelectStmt &subquery,
|
||||
int64_t table_id,
|
||||
ObOptimizerContext &opt_ctx,
|
||||
ObRawExpr *filter,
|
||||
ObRawExpr *&push_filter,
|
||||
bool &can_pushdown_all,
|
||||
bool check_match_index)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
push_filter = NULL;
|
||||
ObRawExprFactory &expr_factory = opt_ctx.get_expr_factory();
|
||||
ObSQLSessionInfo *session_info = opt_ctx.get_session_info();
|
||||
if (OB_ISNULL(filter)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null expr", K(ret));
|
||||
} else if (T_OP_OR != filter->get_expr_type()) {
|
||||
// do nothing
|
||||
} else {
|
||||
ObOpRawExpr *or_pred = static_cast<ObOpRawExpr *>(filter);
|
||||
ObSEArray<ObIArray<ObRawExpr*> *, 4> or_filter_params;
|
||||
ObSEArray<const ObDMLStmt *, 4> parent_stmts;
|
||||
ObSEArray<const ObSelectStmt *, 4> subqueries;
|
||||
ObSEArray<int64_t, 4> table_ids;
|
||||
ObArenaAllocator tmp_allocator;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < or_pred->get_param_count(); ++i) {
|
||||
ObRawExpr *cur_expr = or_pred->get_param_expr(i);
|
||||
ObIArray<ObRawExpr*> *param_exprs = NULL;
|
||||
void *ptr = NULL;
|
||||
if (OB_ISNULL(cur_expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null expr", K(ret), KPC(or_pred));
|
||||
} else if (OB_ISNULL(ptr = tmp_allocator.alloc(sizeof(ObSEArray<ObRawExpr*, 4>)))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("get unexpected null", K(ret));
|
||||
} else if (FALSE_IT((param_exprs = new (ptr)ObSEArray<ObRawExpr*, 4>()))) {
|
||||
} else if (T_OP_AND == cur_expr->get_expr_type()) {
|
||||
ObOpRawExpr *and_pred = static_cast<ObOpRawExpr *>(cur_expr);
|
||||
if (OB_FAIL(param_exprs->assign(and_pred->get_param_exprs()))) {
|
||||
LOG_WARN("failed to assgin predicates", K(ret));
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(param_exprs->push_back(cur_expr))) {
|
||||
LOG_WARN("failed to push back predicate", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(or_filter_params.push_back(param_exprs))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else if (OB_FAIL(parent_stmts.push_back(&parent_stmt))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else if (OB_FAIL(subqueries.push_back(&subquery))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
} else if (OB_FAIL(table_ids.push_back(table_id))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) &&
|
||||
OB_FAIL(split_or_filter_into_subquery(parent_stmts,
|
||||
subqueries,
|
||||
table_ids,
|
||||
or_filter_params,
|
||||
opt_ctx,
|
||||
push_filter,
|
||||
can_pushdown_all,
|
||||
check_match_index))) {
|
||||
LOG_WARN("failed to split or fitler", K(ret));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < or_filter_params.count(); i ++) {
|
||||
if (OB_NOT_NULL(or_filter_params.at(i))) {
|
||||
or_filter_params.at(i)->~ObIArray();
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObOptimizerUtil::split_or_filter_into_subquery(ObIArray<const ObDMLStmt *> &parent_stmts,
|
||||
ObIArray<const ObSelectStmt *> &subqueries,
|
||||
ObIArray<int64_t> &table_ids,
|
||||
ObIArray<ObIArray<ObRawExpr *>*> &or_filter_params,
|
||||
ObOptimizerContext &opt_ctx,
|
||||
ObRawExpr *&push_filter,
|
||||
bool &can_pushdown_all,
|
||||
bool check_match_index)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
push_filter = NULL;
|
||||
ObRawExprFactory &expr_factory = opt_ctx.get_expr_factory();
|
||||
ObSQLSessionInfo *session_info = opt_ctx.get_session_info();
|
||||
can_pushdown_all = true;
|
||||
bool have_push_filter = true;
|
||||
ObSEArray<ObSEArray<ObRawExpr*, 4>, 2> final_push_filters;
|
||||
if (OB_UNLIKELY(parent_stmts.count() != subqueries.count()) ||
|
||||
OB_UNLIKELY(subqueries.count() != table_ids.count()) ||
|
||||
OB_UNLIKELY(table_ids.count() != or_filter_params.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected param count", K(ret));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && have_push_filter && i < or_filter_params.count(); ++i) {
|
||||
ObSEArray<ObRawExpr*, 4> push_filters;
|
||||
ObSEArray<ObRawExpr*, 4> remain_filters;
|
||||
bool can_push_to_where = false;
|
||||
if (OB_ISNULL(parent_stmts.at(i)) ||
|
||||
OB_ISNULL(subqueries.at(i)) ||
|
||||
OB_ISNULL(or_filter_params.at(i)) ||
|
||||
OB_UNLIKELY(or_filter_params.at(i)->empty())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null stmt", K(ret));
|
||||
} else if (OB_FAIL(pushdown_filter_into_subquery(*parent_stmts.at(i),
|
||||
*subqueries.at(i),
|
||||
opt_ctx,
|
||||
*or_filter_params.at(i),
|
||||
push_filters,
|
||||
remain_filters,
|
||||
can_push_to_where,
|
||||
check_match_index))) {
|
||||
LOG_WARN("failed to pushdown filter", K(ret));
|
||||
} else if (push_filters.empty()){
|
||||
// AND pred can not be pushed
|
||||
have_push_filter = false;
|
||||
can_pushdown_all = false;
|
||||
} else {
|
||||
// Part/All of AND pred can be pushed
|
||||
can_pushdown_all &= remain_filters.empty();
|
||||
if (OB_FAIL(final_push_filters.push_back(push_filters))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (OB_SUCC(ret) && have_push_filter) {
|
||||
ObSEArray<ObRawExpr *, 4> rename_and_exprs;
|
||||
ObRawExpr *new_or_expr = NULL;
|
||||
for (int64_t i = 0 ; OB_SUCC(ret) && i < final_push_filters.count() ; i ++) {
|
||||
ObSEArray<ObRawExpr*, 4> rename_exprs;
|
||||
ObRawExpr *rename_and_expr = NULL;
|
||||
if (OB_FAIL(rename_pushdown_filter(*parent_stmts.at(i),
|
||||
*subqueries.at(i),
|
||||
table_ids.at(i),
|
||||
session_info,
|
||||
expr_factory,
|
||||
final_push_filters.at(i),
|
||||
rename_exprs))) {
|
||||
LOG_WARN("failed to rename push down preds", K(ret));
|
||||
} else if (OB_FAIL(ObRawExprUtils::build_and_expr(expr_factory,
|
||||
rename_exprs,
|
||||
rename_and_expr))) {
|
||||
LOG_WARN("failed to build and expr", K(ret));
|
||||
} else if (OB_FAIL(rename_and_exprs.push_back(rename_and_expr))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(ObRawExprUtils::build_or_exprs(expr_factory,
|
||||
rename_and_exprs,
|
||||
new_or_expr))) {
|
||||
LOG_WARN("failed to build and expr", K(ret));
|
||||
} else if (OB_FAIL(new_or_expr->formalize(session_info))) {
|
||||
LOG_WARN("failed to formalize expr", K(ret));
|
||||
} else if (OB_FAIL(new_or_expr->pull_relation_id())) {
|
||||
LOG_WARN("failed to pull relation id and levels", K(ret));
|
||||
} else {
|
||||
push_filter = new_or_expr;
|
||||
}
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
@ -1131,7 +1131,8 @@ public:
|
||||
ObIArray<ObRawExpr*> &pushdown_filters,
|
||||
ObIArray<ObRawExpr*> &candi_filters,
|
||||
ObIArray<ObRawExpr*> &remain_filters,
|
||||
bool &can_pushdown);
|
||||
bool &can_pushdown,
|
||||
bool check_match_index = true);
|
||||
|
||||
/**
|
||||
* @brief check_pushdown_filter
|
||||
@ -1146,7 +1147,8 @@ public:
|
||||
ObOptimizerContext &opt_ctx,
|
||||
ObIArray<ObRawExpr*> &pushdown_filters,
|
||||
ObIArray<ObRawExpr*> &candi_filters,
|
||||
ObIArray<ObRawExpr*> &remain_filters);
|
||||
ObIArray<ObRawExpr*> &remain_filters,
|
||||
bool check_match_index = true);
|
||||
|
||||
static int check_pushdown_filter_overlap_index(const ObDMLStmt &stmt,
|
||||
ObOptimizerContext &opt_ctx,
|
||||
@ -1176,7 +1178,8 @@ public:
|
||||
ObIArray<ObRawExpr*> &common_exprs,
|
||||
ObIArray<ObRawExpr*> &pushdown_filters,
|
||||
ObIArray<ObRawExpr*> &candi_filters,
|
||||
ObIArray<ObRawExpr*> &remain_filters);
|
||||
ObIArray<ObRawExpr*> &remain_filters,
|
||||
bool check_match_index = true);
|
||||
|
||||
/**
|
||||
* @brief get_groupby_win_func_common_exprs
|
||||
@ -1456,6 +1459,32 @@ public:
|
||||
ObOpPseudoColumnRawExpr *&expr);
|
||||
|
||||
static int is_in_range_optimization_enabled(const ObGlobalHint &global_hint, ObSQLSessionInfo *session_info, bool &is_enabled);
|
||||
|
||||
static int pushdown_and_rename_filter_into_subquery(const ObDMLStmt &parent_stmt,
|
||||
const ObSelectStmt &subquery,
|
||||
int64_t table_id,
|
||||
ObOptimizerContext &opt_ctx,
|
||||
ObIArray<ObRawExpr *> &input_filters,
|
||||
ObIArray<ObRawExpr *> &push_filters,
|
||||
ObIArray<ObRawExpr *> &remain_filters,
|
||||
bool check_match_index = true);
|
||||
static int split_or_filter_into_subquery(const ObDMLStmt &parent_stmt,
|
||||
const ObSelectStmt &subquery,
|
||||
int64_t table_id,
|
||||
ObOptimizerContext &opt_ctx,
|
||||
ObRawExpr *filter,
|
||||
ObRawExpr *&push_filter,
|
||||
bool &can_pushdown_all,
|
||||
bool check_match_index = true);
|
||||
static int split_or_filter_into_subquery(ObIArray<const ObDMLStmt *> &parent_stmts,
|
||||
ObIArray<const ObSelectStmt *> &subqueries,
|
||||
ObIArray<int64_t> &table_ids,
|
||||
ObIArray<ObIArray<ObRawExpr *>*> &or_filter_params,
|
||||
ObOptimizerContext &opt_ctx,
|
||||
ObRawExpr *&push_filter,
|
||||
bool &can_pushdown_all,
|
||||
bool check_match_index = true);
|
||||
|
||||
private:
|
||||
//disallow construct
|
||||
ObOptimizerUtil();
|
||||
|
@ -1744,15 +1744,12 @@ int ObSelectLogPlan::generate_raw_plan_for_set()
|
||||
const int64_t child_size = child_stmts.count();
|
||||
const bool is_set_distinct = select_stmt->is_set_distinct();
|
||||
ObSEArray<ObRawExpr *, 8> child_input_filters;
|
||||
ObSEArray<ObRawExpr *, 8> child_candi_filters;
|
||||
ObSEArray<ObRawExpr *, 8> child_rename_filters;
|
||||
ObSEArray<ObRawExpr *, 8> child_remain_filters;
|
||||
bool can_pushdown = false;
|
||||
const ObSelectStmt *child_stmt = NULL;
|
||||
ObSelectLogPlan *child_plan = NULL;
|
||||
for (int64 i = 0; OB_SUCC(ret) && i < child_size; ++i) {
|
||||
child_input_filters.reuse();
|
||||
child_candi_filters.reuse();
|
||||
child_rename_filters.reuse();
|
||||
child_remain_filters.reuse();
|
||||
if (OB_ISNULL(child_stmt = child_stmts.at(i))) {
|
||||
@ -1761,27 +1758,20 @@ int ObSelectLogPlan::generate_raw_plan_for_set()
|
||||
} else if (!pushdown_filters_.empty() &&
|
||||
OB_FAIL(child_input_filters.assign(pushdown_filters_))) {
|
||||
LOG_WARN("failed to copy exprs", K(ret));
|
||||
} else if (!child_input_filters.empty() &&
|
||||
OB_FAIL(ObOptimizerUtil::pushdown_filter_into_subquery(*select_stmt,
|
||||
*child_stmt,
|
||||
get_optimizer_context(),
|
||||
child_input_filters,
|
||||
child_candi_filters,
|
||||
child_remain_filters,
|
||||
can_pushdown))) {
|
||||
LOG_WARN("pushdown filters into left query failed", K(ret));
|
||||
} else if (OB_FAIL(ObOptimizerUtil::pushdown_and_rename_filter_into_subquery(*select_stmt,
|
||||
*child_stmt,
|
||||
OB_INVALID_ID,
|
||||
get_optimizer_context(),
|
||||
child_input_filters,
|
||||
child_rename_filters,
|
||||
child_remain_filters,
|
||||
false))) {
|
||||
LOG_WARN("failed to push down filter into subquery", K(ret));
|
||||
} else if (OB_FAIL(ObOptimizerUtil::get_set_op_remain_filter(*select_stmt,
|
||||
child_remain_filters,
|
||||
remain_filters,
|
||||
0 == i))) {
|
||||
LOG_WARN("get remain filters failed", K(ret));
|
||||
} else if (!child_candi_filters.empty() &&
|
||||
OB_FAIL(ObOptimizerUtil::rename_pushdown_filter(*select_stmt, *child_stmt,
|
||||
OB_INVALID, session_info,
|
||||
*expr_factory,
|
||||
child_candi_filters,
|
||||
child_rename_filters))) {
|
||||
LOG_WARN("failed to rename pushdown filter", K(ret));
|
||||
} else if (OB_FAIL(generate_child_plan_for_set(child_stmt, child_plan,
|
||||
child_rename_filters, i,
|
||||
select_stmt->is_set_distinct()))) {
|
||||
|
@ -3986,6 +3986,8 @@ int ObDMLStmt::collect_temp_table_infos(ObIArray<TempTableInfo> &temp_table_info
|
||||
if (table->ref_query_ == info.temp_table_query_) {
|
||||
if (OB_FAIL(info.table_items_.push_back(table))) {
|
||||
LOG_WARN("failed to push back table item", K(ret));
|
||||
} else if (OB_FAIL(info.upper_stmts_.push_back(this))) {
|
||||
LOG_WARN("failed to push back stmt", K(ret));
|
||||
} else {
|
||||
find = true;
|
||||
}
|
||||
@ -3998,6 +4000,8 @@ int ObDMLStmt::collect_temp_table_infos(ObIArray<TempTableInfo> &temp_table_info
|
||||
LOG_WARN("failed to collect temp table infos", K(ret));
|
||||
} else if (OB_FAIL(info.table_items_.push_back(table))) {
|
||||
LOG_WARN("failed to push back table item", K(ret));
|
||||
} else if (OB_FAIL(info.upper_stmts_.push_back(this))) {
|
||||
LOG_WARN("failed to push back stmt", K(ret));
|
||||
} else if (OB_FAIL(temp_table_infos.push_back(info))) {
|
||||
LOG_WARN("failed to push back temp table info", K(ret));
|
||||
}
|
||||
|
@ -1099,6 +1099,7 @@ public:
|
||||
struct TempTableInfo {
|
||||
TempTableInfo()
|
||||
:table_items_(),
|
||||
upper_stmts_(),
|
||||
temp_table_query_(NULL)
|
||||
{}
|
||||
|
||||
@ -1109,6 +1110,7 @@ public:
|
||||
);
|
||||
|
||||
ObSEArray<TableItem*, 8> table_items_;
|
||||
ObSEArray<ObDMLStmt*, 8> upper_stmts_;
|
||||
ObSelectStmt *temp_table_query_;
|
||||
};
|
||||
int collect_temp_table_infos(ObIArray<TempTableInfo> &temp_table_infos);
|
||||
|
@ -739,6 +739,11 @@ bool ObOptParamHint::is_param_val_valid(const OptParamType param_type, const ObO
|
||||
|| 0 == val.get_varchar().case_compare("false"));
|
||||
break;
|
||||
}
|
||||
case XSOLAPI_GENERATE_WITH_CLAUSE: {
|
||||
is_valid = val.is_varchar() && (0 == val.get_varchar().case_compare("true")
|
||||
|| 0 == val.get_varchar().case_compare("false"));
|
||||
break;
|
||||
}
|
||||
default:
|
||||
LOG_TRACE("invalid opt param val", K(param_type), K(val));
|
||||
break;
|
||||
@ -1313,6 +1318,59 @@ bool QbNameList::is_equal(const ObIArray<ObString> &qb_name_list) const
|
||||
return bret;
|
||||
}
|
||||
|
||||
bool QbNameList::is_subset(const ObIArray<ObSelectStmt*> &stmts) const
|
||||
{
|
||||
bool bret = false;
|
||||
if (qb_names_.count() <= stmts.count()) {
|
||||
bool all_found = true;
|
||||
for (int i = 0; all_found && i < qb_names_.count(); ++i) {
|
||||
bool find = false;
|
||||
ObString stmt_qb_name;
|
||||
for (int j = 0; !find && j < stmts.count(); j ++) {
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(stmts.at(j))) {
|
||||
LOG_WARN("unexpected null stmt");
|
||||
} else if (OB_FAIL(stmts.at(j)->get_qb_name(stmt_qb_name))) {
|
||||
LOG_WARN("failed to get qb name");
|
||||
} else if (0 == stmt_qb_name.case_compare(qb_names_.at(i))) {
|
||||
find = true;
|
||||
}
|
||||
}
|
||||
if (!find) {
|
||||
all_found = false;
|
||||
}
|
||||
}
|
||||
if (all_found) {
|
||||
bret = true;
|
||||
}
|
||||
}
|
||||
return bret;
|
||||
}
|
||||
|
||||
bool QbNameList::is_subset(const ObIArray<ObString> &qb_name_list) const
|
||||
{
|
||||
bool bret = false;
|
||||
if (qb_names_.count() <= qb_name_list.count()) {
|
||||
bool all_found = true;
|
||||
for (int i = 0; all_found && i < qb_names_.count(); ++i) {
|
||||
bool find = false;
|
||||
ObString stmt_qb_name;
|
||||
for (int j = 0; !find && j < qb_name_list.count(); j ++) {
|
||||
if (0 == qb_name_list.at(j).case_compare(qb_names_.at(i))) {
|
||||
find = true;
|
||||
}
|
||||
}
|
||||
if (!find) {
|
||||
all_found = false;
|
||||
}
|
||||
}
|
||||
if (all_found) {
|
||||
bret = true;
|
||||
}
|
||||
}
|
||||
return bret;
|
||||
}
|
||||
|
||||
int ObCountToExistsHint::assign(const ObCountToExistsHint &other)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
|
@ -92,6 +92,7 @@ struct ObOptParamHint
|
||||
DEF(USE_DEFAULT_OPT_STAT,) \
|
||||
DEF(USE_FORCE_BLOCK_SAMPLE,) \
|
||||
DEF(ENABLE_IN_RANGE_OPTIMIZATION,) \
|
||||
DEF(XSOLAPI_GENERATE_WITH_CLAUSE,) \
|
||||
|
||||
DECLARE_ENUM(OptParamType, opt_param, OPT_PARAM_TYPE_DEF, static);
|
||||
|
||||
@ -686,6 +687,8 @@ struct QbNameList {
|
||||
bool has_qb_name(const ObString &qb_name) const;
|
||||
bool is_equal(const ObIArray<ObSelectStmt*> &stmts) const;
|
||||
bool is_equal(const ObIArray<ObString> &qb_name_list) const;
|
||||
bool is_subset(const ObIArray<ObSelectStmt*> &stmts) const;
|
||||
bool is_subset(const ObIArray<ObString> &qb_name_list) const;
|
||||
bool empty() const { return qb_names_.empty(); }
|
||||
TO_STRING_KV(K_(qb_names));
|
||||
common::ObSEArray<ObString, 4, common::ModulePageAllocator, true> qb_names_;
|
||||
|
@ -2331,7 +2331,7 @@ public:
|
||||
|
||||
int64_t get_ref_id() const;
|
||||
void set_ref_id(int64_t id);
|
||||
ObSelectStmt *get_ref_stmt() { return ref_stmt_; }
|
||||
ObSelectStmt *&get_ref_stmt() { return ref_stmt_; }
|
||||
const ObSelectStmt *get_ref_stmt() const { return ref_stmt_; }
|
||||
void set_ref_stmt(ObSelectStmt *ref_stmt)
|
||||
{
|
||||
|
@ -1289,6 +1289,7 @@ int ObStmtComparer::compare_set_stmt(const ObSelectStmt *first,
|
||||
relation = QueryRelation::QUERY_LEFT_SUBSET;
|
||||
}
|
||||
}
|
||||
|
||||
// check order by exprs
|
||||
if (OB_SUCC(ret) && QueryRelation::QUERY_UNCOMPARABLE != relation) {
|
||||
int64_t first_count = first->get_order_item_size();
|
||||
|
@ -58,7 +58,8 @@ int ObTransformMinMax::transform_one_stmt(common::ObIArray<ObParentDMLStmt> &par
|
||||
} else if (!stmt->is_select_stmt()) {
|
||||
//do nothing
|
||||
OPT_TRACE("not select stmt");
|
||||
} else if (OB_FAIL(check_transform_validity(static_cast<ObSelectStmt *>(stmt),
|
||||
} else if (OB_FAIL(check_transform_validity(*ctx_,
|
||||
static_cast<ObSelectStmt *>(stmt),
|
||||
aggr_expr,
|
||||
is_valid))) {
|
||||
LOG_WARN("failed to check transform validity", K(ret));
|
||||
@ -75,7 +76,8 @@ int ObTransformMinMax::transform_one_stmt(common::ObIArray<ObParentDMLStmt> &par
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTransformMinMax::check_transform_validity(ObSelectStmt *select_stmt,
|
||||
int ObTransformMinMax::check_transform_validity(ObTransformerCtx &ctx,
|
||||
ObSelectStmt *select_stmt,
|
||||
ObAggFunRawExpr *&aggr_expr,
|
||||
bool &is_valid)
|
||||
{
|
||||
@ -99,7 +101,7 @@ int ObTransformMinMax::check_transform_validity(ObSelectStmt *select_stmt,
|
||||
} else if ((T_FUN_MAX != expr->get_expr_type() && T_FUN_MIN != expr->get_expr_type()) ||
|
||||
expr->get_real_param_count() != 1) {
|
||||
OPT_TRACE("aggr expr is not min/max expr");
|
||||
} else if (OB_FAIL(is_valid_index_column(select_stmt, expr->get_param_expr(0), is_valid))) {
|
||||
} else if (OB_FAIL(is_valid_index_column(ctx, select_stmt, expr->get_param_expr(0), is_valid))) {
|
||||
LOG_WARN("failed to check is valid index column", K(ret));
|
||||
} else if (!is_valid) {
|
||||
OPT_TRACE("aggr expr is not include index column");
|
||||
@ -275,7 +277,8 @@ int ObTransformMinMax::find_unexpected_having_expr(const ObAggFunRawExpr *aggr_e
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTransformMinMax::is_valid_index_column(const ObSelectStmt *stmt,
|
||||
int ObTransformMinMax::is_valid_index_column(ObTransformerCtx &ctx,
|
||||
const ObSelectStmt *stmt,
|
||||
const ObRawExpr *expr,
|
||||
bool &is_valid)
|
||||
{
|
||||
@ -284,10 +287,10 @@ int ObTransformMinMax::is_valid_index_column(const ObSelectStmt *stmt,
|
||||
const ObColumnRefRawExpr *col_expr = NULL;
|
||||
bool is_match_index = false;
|
||||
ObArenaAllocator alloc;
|
||||
EqualSets &equal_sets = ctx_->equal_sets_;
|
||||
EqualSets &equal_sets = ctx.equal_sets_;
|
||||
ObSEArray<ObRawExpr *, 4> const_exprs;
|
||||
is_valid = false;
|
||||
if (OB_ISNULL(stmt) || OB_ISNULL(expr) || OB_ISNULL(ctx_)) {
|
||||
if (OB_ISNULL(stmt) || OB_ISNULL(expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("invalid argument", K(ret), K(stmt), K(expr));
|
||||
} else if (!expr->is_column_ref_expr()) {
|
||||
@ -303,7 +306,7 @@ int ObTransformMinMax::is_valid_index_column(const ObSelectStmt *stmt,
|
||||
} else if (OB_FAIL(ObOptimizerUtil::compute_const_exprs(stmt->get_condition_exprs(),
|
||||
const_exprs))) {
|
||||
LOG_WARN("failed to compute const equivalent exprs", K(ret));
|
||||
} else if (OB_FAIL(ObTransformUtils::is_match_index(ctx_->sql_schema_guard_,
|
||||
} else if (OB_FAIL(ObTransformUtils::is_match_index(ctx.sql_schema_guard_,
|
||||
stmt,
|
||||
col_expr,
|
||||
is_match_index,
|
||||
|
@ -63,28 +63,29 @@ public:
|
||||
virtual int transform_one_stmt(common::ObIArray<ObParentDMLStmt> &parent_stmts,
|
||||
ObDMLStmt *&stmt,
|
||||
bool &trans_happened) override;
|
||||
static int check_transform_validity(ObTransformerCtx &ctx,
|
||||
ObSelectStmt *stmt,
|
||||
ObAggFunRawExpr *&aggr_expr,
|
||||
bool &is_valid);
|
||||
|
||||
private:
|
||||
|
||||
int check_transform_validity(ObSelectStmt *stmt,
|
||||
ObAggFunRawExpr *&aggr_expr,
|
||||
bool &is_valid);
|
||||
|
||||
int do_transform(ObSelectStmt *select_stmt, ObAggFunRawExpr *aggr_expr);
|
||||
|
||||
int is_valid_index_column(const ObSelectStmt *stmt,
|
||||
const ObRawExpr *expr,
|
||||
bool &is_expected_index);
|
||||
static int is_valid_index_column(ObTransformerCtx &ctx,
|
||||
const ObSelectStmt *stmt,
|
||||
const ObRawExpr *expr,
|
||||
bool &is_expected_index);
|
||||
|
||||
int is_valid_having(const ObSelectStmt *stmt,
|
||||
const ObAggFunRawExpr *column_aggr_expr,
|
||||
bool &is_expected);
|
||||
static int is_valid_having(const ObSelectStmt *stmt,
|
||||
const ObAggFunRawExpr *column_aggr_expr,
|
||||
bool &is_expected);
|
||||
|
||||
int is_valid_aggr_expr(const ObSelectStmt &stmt,
|
||||
const ObRawExpr *expr,
|
||||
const ObAggFunRawExpr *aggr_expr,
|
||||
bool &is_valid);
|
||||
static int is_valid_aggr_expr(const ObSelectStmt &stmt,
|
||||
const ObRawExpr *expr,
|
||||
const ObAggFunRawExpr *aggr_expr,
|
||||
bool &is_valid);
|
||||
|
||||
int find_unexpected_having_expr(const ObAggFunRawExpr *aggr_expr,
|
||||
static int find_unexpected_having_expr(const ObAggFunRawExpr *aggr_expr,
|
||||
const ObRawExpr *cur_expr,
|
||||
bool &is_unexpected);
|
||||
|
||||
@ -96,7 +97,7 @@ private:
|
||||
* @brief: check whether there is any valid select_item
|
||||
* request stmt has only one valid aggr expr, and select_items are exprs combainded const expr or that aggr_expr
|
||||
*/
|
||||
int is_valid_select_list(const ObSelectStmt &stmt, const ObAggFunRawExpr *aggr_expr, bool &is_valid);
|
||||
static int is_valid_select_list(const ObSelectStmt &stmt, const ObAggFunRawExpr *aggr_expr, bool &is_valid);
|
||||
DISALLOW_COPY_AND_ASSIGN(ObTransformMinMax);
|
||||
};
|
||||
|
||||
|
@ -68,12 +68,23 @@ int ObTransformPreProcess::transform_one_stmt(common::ObIArray<ObParentDMLStmt>
|
||||
OB_FAIL(formalize_limit_expr(*stmt))) {
|
||||
LOG_WARN("formalize stmt fialed", K(ret));
|
||||
} else {
|
||||
if (OB_FAIL(transform_udt_columns(parent_stmts, stmt, is_happened))) {
|
||||
LOG_WARN("failed to transform for transform for cast multiset", K(ret));
|
||||
} else {
|
||||
trans_happened |= is_happened;
|
||||
OPT_TRACE("transform for udt columns", is_happened);
|
||||
LOG_TRACE("succeed to transform for udt columns", K(is_happened), K(ret));
|
||||
if (OB_SUCC(ret) && parent_stmts.empty()) {
|
||||
if (OB_FAIL(expand_correlated_cte(stmt, is_happened))) {
|
||||
LOG_WARN("failed to expand correlated cte", K(ret));
|
||||
} else {
|
||||
trans_happened |= is_happened;
|
||||
OPT_TRACE("expand correlated cte", is_happened);
|
||||
LOG_TRACE("succeed to expand correlated cte", K(is_happened), K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(transform_udt_columns(parent_stmts, stmt, is_happened))) {
|
||||
LOG_WARN("failed to transform for transform for cast multiset", K(ret));
|
||||
} else {
|
||||
trans_happened |= is_happened;
|
||||
OPT_TRACE("transform for udt columns", is_happened);
|
||||
LOG_TRACE("succeed to transform for udt columns", K(is_happened), K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(transform_cast_multiset_for_stmt(stmt, is_happened))) {
|
||||
@ -9451,5 +9462,134 @@ int ObTransformPreProcess::remove_last_insert_id(ObRawExpr *&expr) {
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTransformPreProcess::expand_correlated_cte(ObDMLStmt *stmt, bool& trans_happened)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSEArray<ObDMLStmt::TempTableInfo, 8> temp_table_infos;
|
||||
if (OB_ISNULL(stmt)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("stmt is NULL", K(ret));
|
||||
} else if (OB_FAIL(stmt->collect_temp_table_infos(temp_table_infos))) {
|
||||
LOG_WARN("failed to collect temp table infos", K(ret));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < temp_table_infos.count(); i ++) {
|
||||
bool is_correlated = false;
|
||||
ObSEArray<ObSelectStmt *, 4> dummy;
|
||||
if (OB_FAIL(check_is_correlated_cte(temp_table_infos.at(i).temp_table_query_, dummy, is_correlated))) {
|
||||
LOG_WARN("failed to check is correlated cte", K(ret));
|
||||
} else if (!is_correlated) {
|
||||
//do nothing
|
||||
} else if (OB_FAIL(ObTransformUtils::expand_temp_table(ctx_, temp_table_infos.at(i)))) {
|
||||
LOG_WARN("failed to extend temp table", K(ret));
|
||||
} else {
|
||||
trans_happened = true;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTransformPreProcess::check_exec_param_correlated(const ObRawExpr *expr, bool &is_correlated)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
is_correlated = false;
|
||||
if (OB_ISNULL(expr)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("expr is null", K(ret), K(expr));
|
||||
} else if (expr->is_exec_param_expr()) {
|
||||
if (!expr->has_flag(BE_USED)) {
|
||||
is_correlated = true;
|
||||
}
|
||||
} else if (expr->has_flag(CNT_DYNAMIC_PARAM)) {
|
||||
for (int64_t i = 0; OB_SUCC(ret) && !is_correlated && i < expr->get_param_count(); ++i) {
|
||||
if (OB_FAIL(SMART_CALL(check_exec_param_correlated(expr->get_param_expr(i),
|
||||
is_correlated)))) {
|
||||
LOG_WARN("failed to check exec param correlated", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTransformPreProcess::check_is_correlated_cte(ObSelectStmt *stmt, ObIArray<ObSelectStmt *> &visited_cte, bool &is_correlated)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObArray<ObSelectStmt *> child_stmts;
|
||||
ObArray<ObRawExpr *> relation_exprs;
|
||||
is_correlated = false;
|
||||
if (OB_ISNULL(stmt)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("stmt is NULL", K(ret));
|
||||
} else if (OB_FAIL(stmt->get_relation_exprs(relation_exprs))) {
|
||||
LOG_WARN("failed to get relation exprs", K(ret));
|
||||
} else if (OB_FAIL(stmt->get_child_stmts(child_stmts))) {
|
||||
LOG_WARN("failed to get child stmts", K(ret));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && !is_correlated && i < relation_exprs.count(); ++i) {
|
||||
ObRawExpr *expr = relation_exprs.at(i);
|
||||
if (OB_FAIL(check_exec_param_correlated(expr, is_correlated))) {
|
||||
LOG_WARN("failed to check exec param level", K(ret));
|
||||
}
|
||||
}
|
||||
if (!is_correlated) {
|
||||
// add flag to mark the exec param refer the same table
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < stmt->get_subquery_expr_size(); ++i) {
|
||||
ObQueryRefRawExpr *query_ref = stmt->get_subquery_exprs().at(i);
|
||||
if (OB_ISNULL(query_ref)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("query ref is null", K(ret), K(query_ref));
|
||||
}
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < query_ref->get_exec_params().count(); ++j) {
|
||||
ObRawExpr *exec_param = query_ref->get_exec_params().at(j);
|
||||
if (OB_ISNULL(exec_param)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("exec param is null", K(ret));
|
||||
} else if (OB_FAIL(exec_param->add_flag(BE_USED))) {
|
||||
LOG_WARN("failed to add flag", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
for (int64_t i = 0; OB_SUCC(ret) && !is_correlated && i < child_stmts.count(); ++i) {
|
||||
if (OB_FAIL(SMART_CALL(check_is_correlated_cte(child_stmts.at(i), visited_cte, is_correlated)))) {
|
||||
LOG_WARN("failed to get non correlated subquery", K(ret));
|
||||
}
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < stmt->get_table_size(); ++i) {
|
||||
TableItem *table = stmt->get_table_item(i);
|
||||
if (OB_ISNULL(table)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect null table item", K(ret));
|
||||
} else if (!table->is_temp_table()) {
|
||||
//do nothing
|
||||
} else if (ObOptimizerUtil::find_item(visited_cte, table->ref_query_)) {
|
||||
//do nothing
|
||||
} else if (OB_FAIL(visited_cte.push_back(table->ref_query_))) {
|
||||
LOG_WARN("failed to push back stmt", K(ret));
|
||||
} else if (OB_FAIL(SMART_CALL(check_is_correlated_cte(table->ref_query_, visited_cte, is_correlated)))) {
|
||||
LOG_WARN("failed to get non correlated subquery", K(ret));
|
||||
}
|
||||
}
|
||||
|
||||
// clear flag
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < stmt->get_subquery_expr_size(); ++i) {
|
||||
ObQueryRefRawExpr *query_ref = stmt->get_subquery_exprs().at(i);
|
||||
if (OB_ISNULL(query_ref)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("query ref is null", K(ret), K(query_ref));
|
||||
}
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < query_ref->get_exec_params().count(); ++j) {
|
||||
ObRawExpr *exec_param = query_ref->get_exec_params().at(j);
|
||||
if (OB_ISNULL(exec_param)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("exec param is null", K(ret));
|
||||
} else {
|
||||
exec_param->clear_flag(BE_USED);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
} // end namespace sql
|
||||
} // end namespace oceanbase
|
||||
|
@ -600,6 +600,10 @@ struct DistinctObjMeta
|
||||
int expand_last_insert_id_for_join(ObDMLStmt &stmt, JoinedTable *join_table, bool &is_happened);
|
||||
int remove_last_insert_id(ObRawExpr *&expr);
|
||||
int check_last_insert_id_removable(const ObRawExpr *expr, bool &is_removable);
|
||||
|
||||
int expand_correlated_cte(ObDMLStmt *stmt, bool& trans_happened);
|
||||
int check_exec_param_correlated(const ObRawExpr *expr, bool &is_correlated);
|
||||
int check_is_correlated_cte(ObSelectStmt *stmt, ObIArray<ObSelectStmt *> &visited_cte, bool &is_correlated);
|
||||
int convert_join_preds_vector_to_scalar(JoinedTable &joined_table, bool &trans_happened);
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObTransformPreProcess);
|
||||
|
@ -42,17 +42,66 @@ using namespace oceanbase::common;
|
||||
*/
|
||||
int ObTransformPredicateMoveAround::transform_one_stmt(
|
||||
common::ObIArray<ObParentDMLStmt> &parent_stmts, ObDMLStmt *&stmt, bool &trans_happened)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
UNUSED(parent_stmts);
|
||||
bool is_happened = false;
|
||||
if (OB_ISNULL(stmt)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("null stmt", K(ret));
|
||||
} else if (OB_FAIL(inner_do_transfrom(stmt, is_happened))) {
|
||||
LOG_WARN("failed to do predicate move around", K(ret));
|
||||
} else {
|
||||
trans_happened |= is_happened;
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < temp_table_infos_.count(); i ++) {
|
||||
is_happened = false;
|
||||
if (OB_FAIL(inner_do_transfrom(temp_table_infos_.at(i)->table_query_, is_happened))) {
|
||||
LOG_WARN("failed to do predicate move around", K(ret));
|
||||
} else {
|
||||
trans_happened |= is_happened;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_FAIL(push_down_cte_filter(temp_table_infos_, is_happened))) {
|
||||
LOG_WARN("failed to push down filter", K(ret));
|
||||
} else {
|
||||
trans_happened |= is_happened;
|
||||
OPT_TRACE("push down common filter for temp table:", is_happened);
|
||||
LOG_TRACE("succeed to do push down common filter for temp table", K(temp_table_infos_), K(is_happened));
|
||||
}
|
||||
}
|
||||
for (int64_t i = 0; i < temp_table_infos_.count(); i ++) {
|
||||
if (OB_NOT_NULL(temp_table_infos_.at(i))) {
|
||||
temp_table_infos_.at(i)->~ObSqlTempTableInfo();
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret) || transed_stmts_.empty() || !trans_happened) {
|
||||
// transform not happened actually
|
||||
} else if (OB_FAIL(adjust_transed_stmts())) {
|
||||
LOG_WARN("sort sort transed stmts failed", K(ret));
|
||||
} else if (OB_FAIL(add_transform_hint(*stmt, &transed_stmts_))) {
|
||||
LOG_WARN("add transform hint failed", K(ret));
|
||||
}
|
||||
transed_stmts_.reuse();
|
||||
temp_table_infos_.reuse();
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTransformPredicateMoveAround::inner_do_transfrom(ObDMLStmt *stmt, bool &trans_happened)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObArray<ObRawExpr *> dummy_pullup;
|
||||
ObArray<ObRawExpr *> dummy_pushdown;
|
||||
ObArray<int64_t> dummy_list;
|
||||
transed_stmts_.reset();
|
||||
real_happened_ = false;
|
||||
UNUSED(parent_stmts);
|
||||
if (OB_ISNULL(stmt)) {
|
||||
trans_happened = false;
|
||||
const ObQueryHint *query_hint = NULL;
|
||||
if (OB_ISNULL(stmt) || OB_ISNULL(query_hint = stmt->get_stmt_hint().query_hint_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("stmt is null", K(ret), K(stmt));
|
||||
LOG_WARN("stmt or hint is null", K(ret), K(stmt));
|
||||
} else if (is_normal_disabled_transform(*stmt)) {
|
||||
// do nothing
|
||||
} else if (!stmt_map_.created() && OB_FAIL(stmt_map_.create(20, ObModIds::OB_SQL_COMPILE))) {
|
||||
LOG_WARN("failed to create stmt map", K(ret));
|
||||
} else if (OB_FAIL(pullup_predicates(stmt, dummy_list, dummy_pullup))) {
|
||||
@ -62,29 +111,23 @@ int ObTransformPredicateMoveAround::transform_one_stmt(
|
||||
LOG_WARN("failed to create equal exprs for insert", K(ret));
|
||||
} else if (OB_FAIL(pushdown_predicates(stmt, dummy_pushdown))) {
|
||||
LOG_WARN("failed to push down predicates", K(ret));
|
||||
} else if (OB_FAIL(adjust_transed_stmts())) {
|
||||
LOG_WARN("sort sort transed stmts failed", K(ret));
|
||||
} else if (transed_stmts_.empty() || !real_happened_) {
|
||||
// transform not happened actually
|
||||
transed_stmts_.reuse();
|
||||
} else if (OB_FAIL(add_transform_hint(*stmt, &transed_stmts_))) {
|
||||
LOG_WARN("add transform hint failed", K(ret));
|
||||
} else {
|
||||
trans_happened = true;
|
||||
trans_happened = real_happened_;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
const static auto cmp_func = [](ObDMLStmt* l_stmt, ObDMLStmt* r_stmt) {
|
||||
if (OB_ISNULL(l_stmt) || OB_ISNULL(r_stmt)) {
|
||||
return false;
|
||||
} else {
|
||||
return l_stmt->get_stmt_id() < r_stmt->get_stmt_id();
|
||||
}
|
||||
};
|
||||
|
||||
int ObTransformPredicateMoveAround::adjust_transed_stmts()
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
auto cmp_func = [](ObDMLStmt* l_stmt, ObDMLStmt* r_stmt) {
|
||||
if (OB_ISNULL(l_stmt) || OB_ISNULL(r_stmt)) {
|
||||
return false;
|
||||
} else {
|
||||
return l_stmt->get_stmt_id() < r_stmt->get_stmt_id();
|
||||
}
|
||||
};
|
||||
std::sort(transed_stmts_.begin(), transed_stmts_.end(), cmp_func);
|
||||
ObDMLStmt *stmt = NULL;
|
||||
for (int64_t i = transed_stmts_.count() - 1; OB_SUCC(ret) && i >= 0; --i) {
|
||||
@ -128,24 +171,24 @@ int ObTransformPredicateMoveAround::need_transform(const common::ObIArray<ObPare
|
||||
int ret = OB_SUCCESS;
|
||||
const ObQueryHint *query_hint = stmt.get_stmt_hint().query_hint_;
|
||||
need_trans = !is_normal_disabled_transform(stmt);
|
||||
UNUSED(current_level);
|
||||
ObDMLStmt *parent = NULL;
|
||||
temp_table_infos_.reuse();
|
||||
applied_hints_.reuse();
|
||||
if (!need_trans) {
|
||||
//do nothing
|
||||
} else if (OB_ISNULL(ctx_) || OB_ISNULL(query_hint = stmt.get_stmt_hint().query_hint_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null", K(ret), K(ctx_), K(query_hint));
|
||||
} else if (parent_stmts.empty()) {
|
||||
// do nothing
|
||||
} else {
|
||||
// the stmt is a generated table
|
||||
} else if (!parent_stmts.empty() || current_level != 0) {
|
||||
need_trans = false;
|
||||
OPT_TRACE("generate table do not need transform");
|
||||
}
|
||||
if (OB_SUCC(ret) && need_trans && query_hint->has_outline_data()) {
|
||||
if (OB_FAIL(check_outline_valid_to_transform(stmt, need_trans))) {
|
||||
LOG_WARN("check outlien valid failed", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(ObSqlTempTableInfo::collect_temp_tables(allocator_,
|
||||
const_cast<ObDMLStmt &>(stmt),
|
||||
temp_table_infos_,
|
||||
NULL,
|
||||
false))) {
|
||||
LOG_WARN("failed to add all temp tables", K(ret));
|
||||
} else if (check_outline_valid_to_transform(stmt, need_trans)) {
|
||||
LOG_WARN("failed to check outline", K(ret));
|
||||
}
|
||||
LOG_DEBUG("IF NO PRED DEDUCE", K(need_trans));
|
||||
return ret;
|
||||
@ -156,7 +199,6 @@ int ObTransformPredicateMoveAround::check_outline_valid_to_transform(const ObDML
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const ObQueryHint *query_hint = stmt.get_stmt_hint().query_hint_;
|
||||
applied_hints_.reset();
|
||||
if (OB_ISNULL(ctx_) || OB_ISNULL(query_hint)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null", K(ret), K(ctx_), K(query_hint));
|
||||
@ -167,7 +209,16 @@ int ObTransformPredicateMoveAround::check_outline_valid_to_transform(const ObDML
|
||||
if (OB_FAIL(get_stmt_to_trans(const_cast<ObDMLStmt *>(&stmt), views))) {
|
||||
LOG_WARN("get stmt to trans failed", K(ret));
|
||||
}
|
||||
int64_t tmp_trans_list_loc = ctx_->trans_list_loc_;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < temp_table_infos_.count(); i ++) {
|
||||
if (OB_ISNULL(temp_table_infos_.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null temp table info", K(ret));
|
||||
} else if (OB_FAIL(get_stmt_to_trans(temp_table_infos_.at(i)->table_query_, views))) {
|
||||
LOG_WARN("get stmt to trans failed", K(ret));
|
||||
}
|
||||
}
|
||||
std::sort(views.begin(), views.end(), cmp_func);
|
||||
int tmp_trans_list_loc = ctx_->trans_list_loc_;
|
||||
while (OB_SUCC(ret) && i <views.count()) {
|
||||
ObDMLStmt *view = views.at(i);
|
||||
const ObHint *trans_hint = NULL;
|
||||
@ -220,7 +271,6 @@ int ObTransformPredicateMoveAround::transform_one_stmt_with_outline(ObIArray<ObP
|
||||
int ObTransformPredicateMoveAround::get_stmt_to_trans(ObDMLStmt *stmt, ObIArray<ObDMLStmt *> &stmt_to_trans)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bool no_pred_deduce = false;
|
||||
ObSEArray<ObSelectStmt*, 4> child_stmts;
|
||||
if (OB_ISNULL(stmt)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
@ -1184,8 +1234,13 @@ int ObTransformPredicateMoveAround::pushdown_predicates(
|
||||
LOG_WARN("failed to push down predicates into where", K(ret));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < from_items.count(); ++i) {
|
||||
if (OB_FAIL(pushdown_into_table(stmt, stmt->get_table_item(from_items.at(i)),
|
||||
*pullup_preds, stmt->get_condition_exprs()))) {
|
||||
ObSEArray<ObRawExprCondition *, 4> pred_conditions;
|
||||
if (OB_FAIL(append_condition_array(pred_conditions,
|
||||
stmt->get_condition_size(),
|
||||
&stmt->get_condition_exprs()))) {
|
||||
LOG_WARN("failed to prepare allocate", K(ret));
|
||||
} else if (OB_FAIL(pushdown_into_table(stmt, stmt->get_table_item(from_items.at(i)),
|
||||
*pullup_preds, stmt->get_condition_exprs(), pred_conditions))) {
|
||||
LOG_WARN("failed to push down predicates", K(ret));
|
||||
}
|
||||
}
|
||||
@ -2260,7 +2315,8 @@ int ObTransformPredicateMoveAround::pushdown_into_joined_table(
|
||||
ObDMLStmt *stmt,
|
||||
JoinedTable *joined_table,
|
||||
ObIArray<ObRawExpr *> &pullup_preds,
|
||||
ObIArray<ObRawExpr *> &pushdown_preds)
|
||||
ObIArray<ObRawExpr *> &pushdown_preds,
|
||||
ObIArray<ObRawExprCondition *> &pred_conditions)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSEArray<ObRawExpr *, 4> all_preds;
|
||||
@ -2355,6 +2411,8 @@ int ObTransformPredicateMoveAround::pushdown_into_joined_table(
|
||||
if (OB_SUCC(ret)) {
|
||||
ObSEArray<ObRawExpr*,8> left_down;
|
||||
ObSEArray<ObRawExpr*,8> right_down;
|
||||
ObSEArray<ObRawExprCondition *, 8> left_conditions;
|
||||
ObSEArray<ObRawExprCondition *, 8> right_conditions;
|
||||
// consider the left table of the joined table
|
||||
// full outer join, can not push anything down
|
||||
// left outer join, push preds down
|
||||
@ -2365,12 +2423,24 @@ int ObTransformPredicateMoveAround::pushdown_into_joined_table(
|
||||
LOG_WARN("failed to append preds", K(ret));
|
||||
} else if (OB_FAIL(append(right_down, joined_table->join_conditions_))) {
|
||||
LOG_WARN("failed to append preds", K(ret));
|
||||
} else if (OB_FAIL(append(left_conditions, pred_conditions))) {
|
||||
LOG_WARN("failed to append preds", K(ret));
|
||||
} else if (OB_FAIL(append_condition_array(right_conditions,
|
||||
joined_table->join_conditions_.count(),
|
||||
&joined_table->join_conditions_))) {
|
||||
LOG_WARN("failed to prepare allocate", K(ret));
|
||||
}
|
||||
} else if (joined_table->is_right_join()) {
|
||||
if (OB_FAIL(append(left_down, joined_table->join_conditions_))) {
|
||||
LOG_WARN("failed to append preds", K(ret));
|
||||
} else if (OB_FAIL(append(right_down, pushdown_preds))) {
|
||||
LOG_WARN("failed to append preds", K(ret));
|
||||
} else if (OB_FAIL(append_condition_array(left_conditions,
|
||||
joined_table->join_conditions_.count(),
|
||||
&joined_table->join_conditions_))) {
|
||||
LOG_WARN("failed to prepare allocate", K(ret));
|
||||
} else if (OB_FAIL(append(right_conditions, pred_conditions))) {
|
||||
LOG_WARN("failed to append preds", K(ret));
|
||||
}
|
||||
} else if (joined_table->is_inner_join()) {
|
||||
if (OB_FAIL(append(left_down, pushdown_preds))) {
|
||||
@ -2379,6 +2449,14 @@ int ObTransformPredicateMoveAround::pushdown_into_joined_table(
|
||||
LOG_WARN("failed to append preds", K(ret));
|
||||
} else if (OB_FAIL(append(right_down, left_down))) {
|
||||
LOG_WARN("failed to append preds", K(ret));
|
||||
} else if (OB_FAIL(append(left_conditions, pred_conditions))) {
|
||||
LOG_WARN("failed to append preds", K(ret));
|
||||
} else if (OB_FAIL(append_condition_array(left_conditions,
|
||||
joined_table->join_conditions_.count(),
|
||||
&joined_table->join_conditions_))) {
|
||||
LOG_WARN("failed to append preds", K(ret));
|
||||
} else if (OB_FAIL(append(right_conditions, left_conditions))) {
|
||||
LOG_WARN("failed to append preds", K(ret));
|
||||
}
|
||||
} else {
|
||||
//can pushdown nothing
|
||||
@ -2387,12 +2465,14 @@ int ObTransformPredicateMoveAround::pushdown_into_joined_table(
|
||||
} else if (OB_FAIL(pushdown_into_table(stmt,
|
||||
joined_table->left_table_,
|
||||
pullup_preds,
|
||||
left_down))) {
|
||||
left_down,
|
||||
left_conditions))) {
|
||||
LOG_WARN("failed to push down predicates", K(ret));
|
||||
} else if (OB_FAIL(pushdown_into_table(stmt,
|
||||
joined_table->right_table_,
|
||||
pullup_preds,
|
||||
right_down))) {
|
||||
right_down,
|
||||
right_conditions))) {
|
||||
LOG_WARN("failed to push down predicates", K(ret));
|
||||
} else {
|
||||
//删除下推的谓词
|
||||
@ -2444,7 +2524,9 @@ int ObTransformPredicateMoveAround::pushdown_into_joined_table(
|
||||
int ObTransformPredicateMoveAround::get_pushdown_predicates(
|
||||
ObDMLStmt &stmt, TableItem &table,
|
||||
ObIArray<ObRawExpr *> &preds,
|
||||
ObIArray<ObRawExpr *> &table_filters)
|
||||
ObIArray<ObRawExpr *> &table_filters,
|
||||
ObIArray<ObRawExprCondition *> *pred_conditions,
|
||||
ObIArray<ObRawExprCondition *> *table_conditions)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSqlBitSet<> table_set;
|
||||
@ -2458,13 +2540,14 @@ int ObTransformPredicateMoveAround::get_pushdown_predicates(
|
||||
} else if (joined_table.is_inner_join()) {
|
||||
target_table = &joined_table;
|
||||
}
|
||||
} else if (table.is_generated_table()) {
|
||||
} else if (table.is_generated_table() || table.is_temp_table()) {
|
||||
target_table = &table;
|
||||
}
|
||||
if (OB_FAIL(ret) || NULL == target_table) {
|
||||
} else if (OB_FAIL(stmt.get_table_rel_ids(*target_table, table_set))) {
|
||||
LOG_WARN("failed to get table set", K(ret));
|
||||
} else if (OB_FAIL(get_pushdown_predicates(stmt, table_set, preds, table_filters))) {
|
||||
} else if (OB_FAIL(get_pushdown_predicates(stmt, table_set, preds, table_filters,
|
||||
pred_conditions, table_conditions))) {
|
||||
LOG_WARN("failed to get push down predicates", K(ret));
|
||||
}
|
||||
return ret;
|
||||
@ -2473,7 +2556,9 @@ int ObTransformPredicateMoveAround::get_pushdown_predicates(
|
||||
int ObTransformPredicateMoveAround::get_pushdown_predicates(ObDMLStmt &stmt,
|
||||
ObSqlBitSet<> &table_set,
|
||||
ObIArray<ObRawExpr *> &preds,
|
||||
ObIArray<ObRawExpr *> &table_filters)
|
||||
ObIArray<ObRawExpr *> &table_filters,
|
||||
ObIArray<ObRawExprCondition *> *pred_conditions,
|
||||
ObIArray<ObRawExprCondition *> *table_conditions)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < preds.count(); ++i) {
|
||||
@ -2482,13 +2567,19 @@ int ObTransformPredicateMoveAround::get_pushdown_predicates(ObDMLStmt &stmt,
|
||||
LOG_WARN("predicate is null", K(ret));
|
||||
} else if (ObPredicateDeduce::contain_special_expr(*preds.at(i))) {
|
||||
// do nothing
|
||||
} else if (!preds.at(i)->has_flag(CNT_COLUMN)
|
||||
|| !table_set.is_superset2(preds.at(i)->get_relation_ids())) {
|
||||
} else if (!table_set.is_superset2(preds.at(i)->get_relation_ids())) {
|
||||
// shall we push down a predicate containing exec param ?
|
||||
// a subquery may not be unnested after pushing down such a predicate.
|
||||
// do nothing
|
||||
} else if (OB_FAIL(table_filters.push_back(preds.at(i)))) {
|
||||
LOG_WARN("failed to push back predicate", K(ret));
|
||||
} else if (NULL != pred_conditions &&
|
||||
OB_UNLIKELY(i >= pred_conditions->count() || NULL == pred_conditions->at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected pred conditions", K(ret), KPC(pred_conditions));
|
||||
} else if (NULL != pred_conditions && NULL != table_conditions &&
|
||||
OB_FAIL(table_conditions->push_back(pred_conditions->at(i)))) {
|
||||
LOG_WARN("failed to push back conditions", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -2518,6 +2609,7 @@ int ObTransformPredicateMoveAround::pushdown_into_semi_info(ObDMLStmt *stmt,
|
||||
ObSEArray<ObRawExpr*, 4> all_cols;
|
||||
ObSEArray<ObRawExpr*, 4> new_preds;
|
||||
ObSEArray<ObRawExpr *, 16, common::ModulePageAllocator, true> empty;
|
||||
ObSEArray<ObRawExprCondition *, 4> pred_conditions;
|
||||
OPT_TRACE("try to transform semi conditions");
|
||||
if (OB_ISNULL(stmt) || OB_ISNULL(semi_info) ||
|
||||
OB_ISNULL(right_table = stmt->get_table_item_by_id(semi_info->right_table_id_))) {
|
||||
@ -2547,7 +2639,12 @@ int ObTransformPredicateMoveAround::pushdown_into_semi_info(ObDMLStmt *stmt,
|
||||
} else if (OB_FAIL(accept_predicates(*stmt, semi_info->semi_conditions_,
|
||||
properites, new_preds))) {
|
||||
LOG_WARN("failed to check different", K(ret));
|
||||
} else if (OB_FAIL(pushdown_into_table(stmt, right_table, pullup_preds, semi_info->semi_conditions_))) {
|
||||
} else if (OB_FAIL(append_condition_array(pred_conditions,
|
||||
semi_info->semi_conditions_.count(),
|
||||
&semi_info->semi_conditions_))) {
|
||||
LOG_WARN("failed to append preds", K(ret));
|
||||
} else if (OB_FAIL(pushdown_into_table(stmt, right_table, pullup_preds,
|
||||
semi_info->semi_conditions_, pred_conditions))) {
|
||||
LOG_WARN("failed to push down predicates", K(ret));
|
||||
} else if (OB_FAIL(pushdown_semi_info_right_filter(stmt, ctx_, semi_info))) {
|
||||
LOG_WARN("failed to pushdown semi info right filter", K(ret));
|
||||
@ -2651,6 +2748,19 @@ int ObTransformPredicateMoveAround::pushdown_semi_info_right_filter(ObDMLStmt *s
|
||||
LOG_WARN("failed to create view with table", K(ret));
|
||||
} else {
|
||||
real_happened_ = true;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < temp_table_infos_.count(); i ++) {
|
||||
ObSqlTempTableInfo *info = NULL;
|
||||
if (OB_ISNULL(info = temp_table_infos_.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected temp table info", K(ret));
|
||||
}
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < info->table_infos_.count(); ++j) {
|
||||
ObDMLStmt *&upper_stmt = info->table_infos_.at(j).upper_stmt_;
|
||||
if (upper_stmt == stmt) {
|
||||
upper_stmt = view_table->ref_query_;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
@ -2668,13 +2778,15 @@ int ObTransformPredicateMoveAround::pushdown_semi_info_right_filter(ObDMLStmt *s
|
||||
int ObTransformPredicateMoveAround::pushdown_into_table(ObDMLStmt *stmt,
|
||||
TableItem *table_item,
|
||||
ObIArray<ObRawExpr *> &pullup_preds,
|
||||
ObIArray<ObRawExpr *> &preds)
|
||||
ObIArray<ObRawExpr *> &preds,
|
||||
ObIArray<ObRawExprCondition *> &pred_conditions)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSEArray<ObRawExpr*, 8> rename_preds;
|
||||
ObSEArray<ObRawExpr*, 8> table_preds;
|
||||
ObSEArray<ObRawExpr*, 8> candi_preds;
|
||||
ObSEArray<ObRawExpr*, 8> table_pullup_preds;
|
||||
ObSEArray<ObRawExprCondition*, 8> table_conditions;
|
||||
bool is_stack_overflow = false;
|
||||
OPT_TRACE("try to pushdown preds into", table_item);
|
||||
if (OB_FAIL(check_stack_overflow(is_stack_overflow))) {
|
||||
@ -2686,13 +2798,14 @@ int ObTransformPredicateMoveAround::pushdown_into_table(ObDMLStmt *stmt,
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("params have null", K(ret), K(stmt), K(table_item));
|
||||
} else if (OB_FAIL(get_pushdown_predicates(
|
||||
*stmt, *table_item, preds, table_preds))) {
|
||||
*stmt, *table_item, preds, table_preds, &pred_conditions, &table_conditions))) {
|
||||
LOG_WARN("failed to get push down predicates", K(ret));
|
||||
} else if (OB_ISNULL(table_item)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected null after create_view_with_table", K(ret), K(table_item));
|
||||
} else if (!table_item->is_joined_table() &&
|
||||
!table_item->is_generated_table()) {
|
||||
!table_item->is_generated_table() &&
|
||||
!table_item->is_temp_table()) {
|
||||
// do nothing
|
||||
} else if (OB_FAIL(rename_preds.assign(table_preds))) {
|
||||
LOG_WARN("failed to assgin exprs", K(ret));
|
||||
@ -2729,11 +2842,40 @@ int ObTransformPredicateMoveAround::pushdown_into_table(ObDMLStmt *stmt,
|
||||
stmt,
|
||||
static_cast<JoinedTable*>(table_item),
|
||||
table_pullup_preds,
|
||||
candi_preds))) {
|
||||
candi_preds,
|
||||
table_conditions))) {
|
||||
LOG_WARN("failed to push down predicates", K(ret));
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_SUCC(ret) && table_item->is_temp_table()) {
|
||||
bool find_temp_table_info = false;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && !find_temp_table_info && i < temp_table_infos_.count(); i ++) {
|
||||
ObSqlTempTableInfo *temp_table_info = NULL;
|
||||
if (OB_ISNULL(temp_table_info = temp_table_infos_.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected temp table info", K(ret));
|
||||
} else if (temp_table_info->table_query_ == table_item->ref_query_) {
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < temp_table_info->table_infos_.count(); j ++) {
|
||||
TableInfo &table_info = temp_table_info->table_infos_.at(j);
|
||||
if (table_info.table_item_ != table_item) {
|
||||
// do nothing
|
||||
} else if (OB_FAIL(append(table_info.table_filters_, rename_preds))) {
|
||||
LOG_WARN("failed to append", K(ret));
|
||||
} else if (OB_FAIL(append(table_info.filter_conditions_, table_conditions))) {
|
||||
LOG_WARN("failed to append", K(ret));
|
||||
} else if (OB_UNLIKELY(table_info.table_filters_.count() != table_info.filter_conditions_.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected filter count",
|
||||
K(table_info.table_filters_.count()), K(table_info.filter_conditions_.count()));
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
find_temp_table_info = true;;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && (table_item->is_generated_table() || table_item->is_joined_table())) {
|
||||
// remove a pred from preds if it is pushed into a joined table or a generated table
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < rename_preds.count(); ++i) {
|
||||
// check whether a table filter is pushed into a view
|
||||
@ -3569,4 +3711,162 @@ int ObTransformPredicateMoveAround::extract_filter_column_exprs_for_insert(ObDel
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTransformPredicateMoveAround::push_down_cte_filter(ObIArray<ObSqlTempTableInfo *> &temp_table_info,
|
||||
bool &trans_happened)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
trans_happened = false;
|
||||
real_happened_ = false;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < temp_table_info.count(); ++i) {
|
||||
ObSqlTempTableInfo *info = temp_table_info.at(i);
|
||||
uint64_t filter_count = 0;
|
||||
bool have_common_filter = true;
|
||||
bool is_valid = true;
|
||||
ObSEArray<ObRawExpr *, 4> common_filters;
|
||||
bool enable_no_pred_deduce = false;
|
||||
if (OB_ISNULL(info) || OB_ISNULL(info->table_query_) ||
|
||||
OB_ISNULL(info->table_query_->get_query_ctx())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect null ref query", K(ret));
|
||||
} else if (OB_FAIL(check_enable_no_pred_deduce(*info->table_query_, enable_no_pred_deduce))) {
|
||||
LOG_WARN("check_enable_no_pred_deduce failed", K(ret));
|
||||
} else if (enable_no_pred_deduce) {
|
||||
OPT_TRACE("hint reject push down filter into ", info->table_name_);
|
||||
continue;
|
||||
}
|
||||
for (int64_t j = 0; OB_SUCC(ret) && have_common_filter && j < info->table_infos_.count(); ++j) {
|
||||
ObIArray<ObRawExpr *> &table_filters = info->table_infos_.at(j).table_filters_;
|
||||
if (table_filters.empty()) {
|
||||
have_common_filter = false;
|
||||
} else if (j == 0) {
|
||||
if (OB_FAIL(common_filters.assign(table_filters))) {
|
||||
LOG_WARN("failed to assign", K(ret));
|
||||
}
|
||||
} else {
|
||||
ObSEArray<ObRawExpr *, 4> new_common_filters;
|
||||
ObTempTableColumnCheckContext check_context;
|
||||
check_context.init(info->table_infos_.at(0).table_item_->table_id_,
|
||||
info->table_infos_.at(j).table_item_->table_id_,
|
||||
&info->table_query_->get_query_ctx()->calculable_items_);
|
||||
for (int64_t k = 0; OB_SUCC(ret) && k < common_filters.count(); ++k) {
|
||||
bool find = false;
|
||||
if (OB_FAIL(ObTransformUtils::find_expr(table_filters, common_filters.at(k), find, &check_context))) {
|
||||
LOG_WARN("failed to find expr", K(ret));
|
||||
} else if (find && OB_FAIL(new_common_filters.push_back(common_filters.at(k)))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
have_common_filter = !new_common_filters.empty();
|
||||
if (OB_SUCC(ret) && have_common_filter && OB_FAIL(common_filters.assign(new_common_filters))) {
|
||||
LOG_WARN("failed to assign", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && have_common_filter) {
|
||||
OPT_TRACE("try to pushdown filter into temp table:", info->table_query_);
|
||||
//当所有的引用表都有可以相同的下推的谓词时才下推谓词
|
||||
ObDMLStmt *orig_stmt = info->table_query_;
|
||||
if (OB_FAIL(inner_push_down_cte_filter(*info, common_filters))) {
|
||||
LOG_WARN("failed to pushdown preds into temp table", K(ret));
|
||||
} else {
|
||||
trans_happened = true;
|
||||
real_happened_ = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTransformPredicateMoveAround::inner_push_down_cte_filter(ObSqlTempTableInfo& info, ObIArray<ObRawExpr *> &filters)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObRawExprFactory *expr_factory = NULL;
|
||||
ObSQLSessionInfo *session_info = NULL;
|
||||
ObSEArray<ObRawExpr *, 8> rename_exprs;
|
||||
if (OB_ISNULL(info.table_query_) ||
|
||||
OB_ISNULL(ctx_) ||
|
||||
OB_ISNULL(expr_factory = ctx_->expr_factory_) ||
|
||||
OB_ISNULL(session_info = ctx_->session_info_) ||
|
||||
OB_ISNULL(info.table_query_->get_query_ctx())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect null param", K(info), K(expr_factory), K(ret));
|
||||
} else if (OB_FAIL(add_var_to_array_no_dup(transed_stmts_, static_cast<ObDMLStmt *>(info.table_query_)))) {
|
||||
LOG_WARN("append transed stmt failed", K(ret));
|
||||
} else if (!info.table_query_->is_spj() && OB_FAIL(ObTransformUtils::pack_stmt(ctx_, info.table_query_))) {
|
||||
LOG_WARN("failed to create spj", K(ret));
|
||||
} else if (OB_FAIL(ObOptimizerUtil::rename_pushdown_filter(*info.table_infos_.at(0).upper_stmt_,
|
||||
*info.table_query_,
|
||||
info.table_infos_.at(0).table_item_->table_id_,
|
||||
session_info,
|
||||
*expr_factory,
|
||||
filters,
|
||||
rename_exprs))) {
|
||||
LOG_WARN("failed to rename push down preds", K(ret));
|
||||
} else if (OB_FAIL(append(info.table_query_->get_condition_exprs(), rename_exprs))) {
|
||||
LOG_WARN("failed to append exprs", K(ret));
|
||||
}
|
||||
|
||||
// remove the filter in upper stmts
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < info.table_infos_.count(); i ++) {
|
||||
ObIArray<ObRawExpr *> &table_filters = info.table_infos_.at(i).table_filters_;
|
||||
ObIArray<ObRawExprCondition *> &filter_conditions = info.table_infos_.at(i).filter_conditions_;
|
||||
ObTempTableColumnCheckContext check_context;
|
||||
check_context.init(info.table_infos_.at(0).table_item_->table_id_,
|
||||
info.table_infos_.at(i).table_item_->table_id_,
|
||||
&info.table_query_->get_query_ctx()->calculable_items_);
|
||||
if (OB_UNLIKELY(table_filters.count() != filter_conditions.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected filter count", K(table_filters.count()), K(filter_conditions.count()));
|
||||
}
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < table_filters.count(); j ++) {
|
||||
bool find = false;
|
||||
if (OB_FAIL(ObTransformUtils::find_expr(filters, table_filters.at(j), find, &check_context))) {
|
||||
LOG_WARN("failed to find expr", K(ret));
|
||||
} else if (!find) {
|
||||
// do nothing
|
||||
} else if (OB_ISNULL(filter_conditions.at(j))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpecte null filter conditions", K(ret));
|
||||
} else if (OB_FAIL(ObOptimizerUtil::remove_item(*filter_conditions.at(j), table_filters.at(j)))) {
|
||||
LOG_WARN("failed to remove condition", K(ret));
|
||||
} else {
|
||||
OPT_TRACE("succeed to remove filter in upper stmt : ", table_filters.at(j));
|
||||
}
|
||||
}
|
||||
if (OB_FAIL(ret)) {
|
||||
} else if (OB_FAIL(add_var_to_array_no_dup(transed_stmts_, info.table_infos_.at(i).upper_stmt_))) {
|
||||
LOG_WARN("append transed stmt failed", K(ret));
|
||||
} else if (OB_FAIL(append(ctx_->equal_param_constraints_, check_context.equal_param_info_))) {
|
||||
LOG_WARN("failed to append equal param constraints", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTransformPredicateMoveAround::append_condition_array(ObIArray<ObRawExprCondition *> &conditions,
|
||||
int count, ObRawExprCondition *value)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < count; i++) {
|
||||
if (OB_FAIL(conditions.push_back(value))) {
|
||||
LOG_WARN("failed to push back", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
bool ObTempTableColumnCheckContext::compare_column(const ObColumnRefRawExpr &left,
|
||||
const ObColumnRefRawExpr &right)
|
||||
{
|
||||
bool bret = false;
|
||||
if (left.get_expr_type() != right.get_expr_type()) {
|
||||
} else if (left.get_column_id() != right.get_column_id()) {
|
||||
} else if (left.get_table_id() == right.get_table_id() ||
|
||||
(left.get_table_id() == first_temp_table_id_ && right.get_table_id() == second_temp_table_id_) ||
|
||||
(left.get_table_id() == second_temp_table_id_ && right.get_table_id() == first_temp_table_id_)) {
|
||||
bret = true;
|
||||
}
|
||||
return bret;
|
||||
}
|
@ -20,6 +20,24 @@ namespace oceanbase
|
||||
namespace sql
|
||||
{
|
||||
class ObDelUpdStmt;
|
||||
|
||||
struct ObTempTableColumnCheckContext : public ObStmtCompareContext {
|
||||
ObTempTableColumnCheckContext() :
|
||||
ObStmtCompareContext() {
|
||||
override_column_compare_ = true;
|
||||
}
|
||||
virtual bool compare_column(const ObColumnRefRawExpr &left,
|
||||
const ObColumnRefRawExpr &right) override;
|
||||
void init(int64_t first, int64_t second,
|
||||
const ObIArray<ObHiddenColumnItem> *calculable_items) {
|
||||
first_temp_table_id_ = first;
|
||||
second_temp_table_id_ = second;
|
||||
ObStmtCompareContext::init(calculable_items);
|
||||
}
|
||||
int64_t first_temp_table_id_;
|
||||
int64_t second_temp_table_id_;
|
||||
};
|
||||
|
||||
class ObTransformPredicateMoveAround : public ObTransformRule
|
||||
{
|
||||
public:
|
||||
@ -43,6 +61,8 @@ private:
|
||||
const ObDMLStmt &stmt,
|
||||
bool &need_trans) override;
|
||||
|
||||
int inner_do_transfrom(ObDMLStmt *stmt, bool &trans_happened);
|
||||
|
||||
int adjust_transed_stmts();
|
||||
|
||||
int check_outline_valid_to_transform(const ObDMLStmt &stmt, bool &need_trans);
|
||||
@ -202,22 +222,28 @@ private:
|
||||
int pushdown_into_table(ObDMLStmt *stmt,
|
||||
TableItem *table,
|
||||
ObIArray<ObRawExpr *> &pullup_preds,
|
||||
ObIArray<ObRawExpr *> &preds);
|
||||
ObIArray<ObRawExpr *> &preds,
|
||||
ObIArray<ObRawExprCondition *> &pred_conditions);
|
||||
|
||||
int get_pushdown_predicates(ObDMLStmt &stmt,
|
||||
TableItem &table,
|
||||
ObIArray<ObRawExpr *> &preds,
|
||||
ObIArray<ObRawExpr *> &table_filters);
|
||||
ObIArray<ObRawExpr *> &table_filters,
|
||||
ObIArray<ObRawExprCondition *> *pred_conditions = NULL,
|
||||
ObIArray<ObRawExprCondition *> *table_conditions = NULL);
|
||||
|
||||
int get_pushdown_predicates(ObDMLStmt &stmt,
|
||||
ObSqlBitSet<> &table_set,
|
||||
ObIArray<ObRawExpr *> &preds,
|
||||
ObIArray<ObRawExpr *> &table_filters);
|
||||
ObIArray<ObRawExpr *> &table_filters,
|
||||
ObIArray<ObRawExprCondition *> *pred_conditions = NULL,
|
||||
ObIArray<ObRawExprCondition *> *table_conditions = NULL);
|
||||
|
||||
int pushdown_into_joined_table(ObDMLStmt *stmt,
|
||||
JoinedTable *joined_table,
|
||||
ObIArray<ObRawExpr *> &pullup_preds,
|
||||
ObIArray<ObRawExpr *> &pushdown_preds);
|
||||
ObIArray<ObRawExpr *> &pushdown_preds,
|
||||
ObIArray<ObRawExprCondition *> &pred_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);
|
||||
@ -327,6 +353,12 @@ private:
|
||||
|
||||
int extract_filter_column_exprs_for_insert(ObDelUpdStmt &del_upd_stmt, ObIArray<ObRawExpr *> &columns);
|
||||
|
||||
int inner_push_down_cte_filter(ObSqlTempTableInfo& info, ObIArray<ObRawExpr *> &filters);
|
||||
|
||||
int push_down_cte_filter(ObIArray<ObSqlTempTableInfo *> &temp_table_info, bool &trans_happened);
|
||||
|
||||
int append_condition_array(ObIArray<ObRawExprCondition *> &conditions, int count, ObRawExprCondition *value);
|
||||
|
||||
private:
|
||||
typedef ObSEArray<ObRawExpr *, 4> PullupPreds;
|
||||
ObArenaAllocator allocator_;
|
||||
@ -334,6 +366,7 @@ private:
|
||||
Ob2DArray<PullupPreds *> stmt_pullup_preds_;
|
||||
ObSEArray<ObDMLStmt *, 8> transed_stmts_;
|
||||
ObSEArray<ObHint *, 4> applied_hints_;
|
||||
ObSEArray<ObSqlTempTableInfo *, 2> temp_table_infos_;
|
||||
bool real_happened_;
|
||||
};
|
||||
|
||||
|
@ -58,7 +58,6 @@ void ObTransformerCtx::reset()
|
||||
happened_cost_based_trans_ = 0;
|
||||
equal_sets_.reset();
|
||||
ignore_semi_infos_.reset();
|
||||
used_table_filters_.reset();
|
||||
temp_table_ignore_stmts_.reset();
|
||||
trans_list_loc_ = 0;
|
||||
src_qb_name_.reset();
|
||||
|
@ -58,7 +58,6 @@ struct ObTransformerCtx
|
||||
happened_cost_based_trans_(0),
|
||||
equal_sets_(),
|
||||
ignore_semi_infos_(),
|
||||
used_table_filters_(),
|
||||
temp_table_ignore_stmts_(),
|
||||
eval_cost_(false),
|
||||
trans_list_loc_(0),
|
||||
@ -116,8 +115,6 @@ struct ObTransformerCtx
|
||||
EqualSets equal_sets_;
|
||||
//记录semi to inner改写中,代价竞争失败的semi info,避免下一轮迭代重复检查代价
|
||||
ObSEArray<uint64_t, 8, common::ModulePageAllocator, true> ignore_semi_infos_;
|
||||
//记录temp table优化过程中使用过的table filter
|
||||
ObSEArray<ObRawExpr*, 8, common::ModulePageAllocator, true> used_table_filters_;
|
||||
ObSEArray<ObSelectStmt*, 8, common::ModulePageAllocator, true> temp_table_ignore_stmts_;
|
||||
bool eval_cost_;
|
||||
/* used for hint and outline below */
|
||||
@ -377,6 +374,12 @@ protected:
|
||||
{ return hint.get_normal_hint(hint_type_); }
|
||||
ObItemType get_hint_type() const { return hint_type_; }
|
||||
|
||||
int deep_copy_temp_table(ObDMLStmt &stmt,
|
||||
ObStmtFactory &stmt_factory,
|
||||
ObRawExprFactory &expr_factory,
|
||||
ObIArray<ObSelectStmt*> &old_temp_table_stmts,
|
||||
ObIArray<ObSelectStmt*> &new_temp_table_stmts);
|
||||
|
||||
private:
|
||||
// pre-order transformation
|
||||
int transform_pre_order(common::ObIArray<ObParentDMLStmt> &parent_stmts,
|
||||
@ -404,12 +407,6 @@ private:
|
||||
ObDMLStmt *&orgin_stmt,
|
||||
ObDMLStmt *&root_stmt);
|
||||
|
||||
int deep_copy_temp_table(ObDMLStmt &stmt,
|
||||
ObStmtFactory &stmt_factory,
|
||||
ObRawExprFactory &expr_factory,
|
||||
ObIArray<ObSelectStmt*> &old_temp_table_stmts,
|
||||
ObIArray<ObSelectStmt*> &new_temp_table_stmts);
|
||||
|
||||
int evaluate_cost(common::ObIArray<ObParentDMLStmt> &parent_stms,
|
||||
ObDMLStmt *&stmt,
|
||||
bool is_trans_stmt,
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -39,43 +39,6 @@ public:
|
||||
|
||||
virtual int construct_transform_hint(ObDMLStmt &stmt, void *trans_params) override;
|
||||
|
||||
struct TableInfo {
|
||||
TableInfo()
|
||||
:table_filters_(),
|
||||
column_ids_(),
|
||||
table_item_(NULL),
|
||||
upper_stmt_(NULL)
|
||||
{}
|
||||
|
||||
virtual ~TableInfo(){}
|
||||
|
||||
TO_STRING_KV(
|
||||
K_(table_filters),
|
||||
K_(column_ids),
|
||||
K_(table_item),
|
||||
K_(upper_stmt)
|
||||
);
|
||||
ObSEArray<ObRawExpr*, 8> table_filters_;
|
||||
ObSqlBitSet<> column_ids_;
|
||||
TableItem *table_item_;
|
||||
ObDMLStmt* upper_stmt_;
|
||||
};
|
||||
struct TempTableInfo {
|
||||
TempTableInfo()
|
||||
:table_infos_(),
|
||||
temp_table_query_(NULL)
|
||||
{}
|
||||
|
||||
virtual ~TempTableInfo(){}
|
||||
|
||||
TO_STRING_KV(
|
||||
K_(table_infos),
|
||||
K_(temp_table_query)
|
||||
);
|
||||
|
||||
ObSEArray<TableInfo, 8> table_infos_;
|
||||
ObSelectStmt *temp_table_query_;
|
||||
};
|
||||
struct StmtClassifyHelper {
|
||||
StmtClassifyHelper()
|
||||
:stmts_(),
|
||||
@ -114,42 +77,61 @@ public:
|
||||
ObSelectStmt *trans_stmt_;
|
||||
ObItemType trans_type_;
|
||||
};
|
||||
|
||||
/**
|
||||
* @brief expand_temp_table
|
||||
* 如果temp table只被引用一次
|
||||
* 还原成generate table
|
||||
*/
|
||||
int expand_temp_table(ObIArray<TempTableInfo> &temp_table_info,
|
||||
int expand_temp_table(ObIArray<ObDMLStmt::TempTableInfo> &temp_table_info,
|
||||
bool &trans_happened);
|
||||
|
||||
int check_stmt_size(ObDMLStmt *stmt, int64_t &total_size, bool &stmt_oversize);
|
||||
|
||||
int inner_expand_temp_table(TempTableInfo &helper);
|
||||
int inner_expand_temp_table(ObDMLStmt::TempTableInfo &helper);
|
||||
|
||||
int check_stmt_can_materialize(ObSelectStmt *stmt, bool &is_valid);
|
||||
int check_stmt_can_materialize(ObSelectStmt *stmt, bool is_existing_cte, bool &is_valid);
|
||||
|
||||
int check_stmt_has_cross_product(ObSelectStmt *stmt, bool &has_cross_product);
|
||||
|
||||
int generate_with_clause(ObDMLStmt *&stmt, bool &trans_happened);
|
||||
|
||||
int extract_common_subquery_as_cte(ObDMLStmt *stmt,
|
||||
ObIArray<ObSelectStmt*> &stmts,
|
||||
hash::ObHashMap<uint64_t, ObDMLStmt *> &parent_map,
|
||||
hash::ObHashMap<uint64_t, ObParentDMLStmt> &parent_map,
|
||||
bool &trans_happened);
|
||||
|
||||
int inner_extract_common_subquery_as_cte(ObDMLStmt &root_stmt,
|
||||
ObIArray<ObSelectStmt*> &stmts,
|
||||
hash::ObHashMap<uint64_t, ObDMLStmt *> &parent_map,
|
||||
hash::ObHashMap<uint64_t,ObParentDMLStmt> &parent_map,
|
||||
bool &trans_happened);
|
||||
|
||||
int add_materialize_stmts(const ObIArray<ObSelectStmt*> &stms);
|
||||
|
||||
int check_has_stmt(ObSelectStmt *left_stmt,
|
||||
ObSelectStmt *right_stmt,
|
||||
hash::ObHashMap<uint64_t, ObDMLStmt *> &parent_map,
|
||||
hash::ObHashMap<uint64_t, ObParentDMLStmt> &parent_map,
|
||||
bool &has_stmt);
|
||||
|
||||
bool is_similar_stmt(ObSelectStmt& stmt,
|
||||
const ObStmtMapInfo &map_info,
|
||||
QueryRelation relation);
|
||||
int check_has_stmt(const ObIArray<ObSelectStmt *> &left_stmt,
|
||||
ObSelectStmt *right_stmt,
|
||||
hash::ObHashMap<uint64_t, ObParentDMLStmt> &parent_map,
|
||||
bool &has_stmt);
|
||||
|
||||
int check_stmt_can_extract_temp_table(ObSelectStmt *first,
|
||||
ObSelectStmt *second,
|
||||
const ObStmtMapInfo &map_info,
|
||||
QueryRelation relation,
|
||||
bool check_basic,
|
||||
bool &is_valid);
|
||||
int check_equal_join_condition_match(ObSelectStmt &first,
|
||||
ObSelectStmt &second,
|
||||
const ObStmtMapInfo &map_info,
|
||||
bool &is_match);
|
||||
int check_index_condition_match(ObSelectStmt &first,
|
||||
ObSelectStmt &second,
|
||||
const ObStmtMapInfo &map_info,
|
||||
bool &is_match);
|
||||
|
||||
int remove_simple_stmts(ObIArray<ObSelectStmt*> &stmts);
|
||||
|
||||
@ -159,6 +141,9 @@ public:
|
||||
ObIArray<ObSelectStmt *> &non_correlated_stmts,
|
||||
uint64_t &min_param_level);
|
||||
|
||||
int get_non_correlated_subquery(ObDMLStmt *stmt,
|
||||
ObIArray<ObSelectStmt *> &non_correlated_stmts);
|
||||
|
||||
int check_exec_param_level(const ObRawExpr *expr,
|
||||
const hash::ObHashMap<uint64_t, uint64_t> ¶m_level,
|
||||
uint64_t &min_param_level);
|
||||
@ -168,7 +153,10 @@ public:
|
||||
int classify_stmts(ObIArray<ObSelectStmt*> &stmts,
|
||||
ObIArray<StmtClassifyHelper> &stmt_groups);
|
||||
|
||||
int create_temp_table(StmtCompareHelper& compare_info);
|
||||
int create_temp_table(ObDMLStmt &root_stmt,
|
||||
StmtCompareHelper& compare_info,
|
||||
hash::ObHashMap<uint64_t, ObParentDMLStmt> &parent_map,
|
||||
bool &trans_happened);
|
||||
|
||||
int compute_common_map_info(ObIArray<ObStmtMapInfo>& map_infos,
|
||||
ObStmtMapInfo &common_map_info);
|
||||
@ -200,39 +188,15 @@ public:
|
||||
const uint64_t &view_table_id,
|
||||
uint64_t &table_id);
|
||||
|
||||
int collect_temp_table_infos(ObDMLStmt *stmt,
|
||||
ObIArray<TempTableInfo> &temp_table_infos);
|
||||
|
||||
int collect_temp_table_infos(ObIArray<ObSelectStmt*> &stmts,
|
||||
ObIArray<TempTableInfo> &temp_table_infos);
|
||||
|
||||
int inner_collect_temp_table_info(TableInfo &table_info);
|
||||
|
||||
int get_table_filters(ObDMLStmt *stmt, TableItem *table, ObIArray<ObRawExpr*> &table_filters);
|
||||
|
||||
int get_table_filters_in_joined_table(JoinedTable *table,
|
||||
uint64_t table_id,
|
||||
const ObSqlBitSet<> &table_ids,
|
||||
ObIArray<ObRawExpr*> &table_filters);
|
||||
|
||||
int get_candi_exprs(const ObSqlBitSet<> &table_ids,
|
||||
const ObIArray<ObRawExpr*> &exprs,
|
||||
ObIArray<ObRawExpr*> &candi_exprs);
|
||||
|
||||
int project_pruning(ObIArray<TempTableInfo> &temp_table_info,
|
||||
int project_pruning(ObIArray<ObDMLStmt::TempTableInfo> &temp_table_info,
|
||||
bool &trans_happened);
|
||||
|
||||
int get_remove_select_item(TempTableInfo &info,
|
||||
int get_remove_select_item(ObDMLStmt::TempTableInfo &info,
|
||||
ObSqlBitSet<> &removed_idx);
|
||||
|
||||
int remove_select_items(TempTableInfo &info,
|
||||
int remove_select_items(ObDMLStmt::TempTableInfo &info,
|
||||
ObSqlBitSet<> &removed_idxs);
|
||||
|
||||
int push_down_filter(ObIArray<TempTableInfo> &temp_table_info,
|
||||
bool &trans_happened);
|
||||
|
||||
int inner_push_down_filter(TempTableInfo& info);
|
||||
|
||||
int add_normal_temp_table_trans_hint(ObDMLStmt &stmt, ObItemType type);
|
||||
|
||||
virtual int need_transform(const common::ObIArray<ObParentDMLStmt> &parent_stmts,
|
||||
@ -255,8 +219,36 @@ public:
|
||||
|
||||
int sort_materialize_stmts(Ob2DArray<MaterializeStmts *> &materialize_stmts);
|
||||
|
||||
int pushdown_shared_subqueries(ObSelectStmt *parent_stmt, ObIArray<ObRawExpr*> &candi_exprs);
|
||||
|
||||
int get_stmt_pointers(ObDMLStmt &root_stmt,
|
||||
ObIArray<ObSelectStmt *> &stmts,
|
||||
hash::ObHashMap<uint64_t, ObParentDMLStmt> &parent_map,
|
||||
ObIArray<ObSelectStmtPointer> &stmt_ptrs);
|
||||
int accept_cte_transform(ObDMLStmt &origin_root_stmt,
|
||||
TableItem *temp_table,
|
||||
common::ObIArray<ObSelectStmt *> &origin_stmts,
|
||||
common::ObIArray<ObSelectStmt *> &trans_stmts,
|
||||
common::ObIArray<ObSelectStmt *> &accept_stmts,
|
||||
hash::ObHashMap<uint64_t, ObParentDMLStmt> &parent_map,
|
||||
bool force_accept,
|
||||
bool &trans_happened);
|
||||
int prepare_eval_cte_cost_stmt(ObDMLStmt &root_stmt,
|
||||
ObIArray<ObSelectStmt *> &trans_stmts,
|
||||
ObSelectStmt *cte_query,
|
||||
ObIArray<ObSelectStmtPointer> &stmt_ptrs,
|
||||
ObDMLStmt *&copied_stmt,
|
||||
ObIArray<ObSelectStmt *> &copied_trans_stmts,
|
||||
ObSelectStmt *&copied_cte_query,
|
||||
bool is_trans_stmt);
|
||||
int evaluate_cte_cost(ObDMLStmt &root_stmt,
|
||||
bool is_trans_stmt,
|
||||
ObIArray<ObSelectStmt *> &stmts,
|
||||
ObIArray<ObSelectStmtPointer> &stmt_ptrs,
|
||||
ObIArray<double> &costs,
|
||||
TableItem *temp_table,
|
||||
double &temp_table_cost);
|
||||
int adjust_transformed_stmt(ObIArray<ObSelectStmtPointer> &stmt_ptrs,
|
||||
ObIArray<ObSelectStmt *> &stmts,
|
||||
ObIArray<ObSelectStmt *> *origin_stmts);
|
||||
private:
|
||||
ObArenaAllocator allocator_;
|
||||
TempTableTransParam *trans_param_;
|
||||
|
@ -2719,7 +2719,8 @@ bool ObTransformUtils::is_not_null_deduce_type(const ObItemType type)
|
||||
|
||||
int ObTransformUtils::find_expr(const ObIArray<const ObRawExpr *> &source,
|
||||
const ObRawExpr *target,
|
||||
bool &bret)
|
||||
bool &bret,
|
||||
ObExprEqualCheckContext *check_context)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bret = false;
|
||||
@ -2731,7 +2732,7 @@ int ObTransformUtils::find_expr(const ObIArray<const ObRawExpr *> &source,
|
||||
if (OB_ISNULL(source.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("expr in source is null", K(ret));
|
||||
} else if (source.at(i) == target || source.at(i)->same_as(*target)) {
|
||||
} else if (source.at(i) == target || source.at(i)->same_as(*target, check_context)) {
|
||||
bret = true;
|
||||
}
|
||||
}
|
||||
@ -2740,7 +2741,8 @@ int ObTransformUtils::find_expr(const ObIArray<const ObRawExpr *> &source,
|
||||
|
||||
int ObTransformUtils::find_expr(ObIArray<ObRawExpr *> &source,
|
||||
ObRawExpr *target,
|
||||
bool &bret)
|
||||
bool &bret,
|
||||
ObExprEqualCheckContext *check_context)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
bret = false;
|
||||
@ -2752,7 +2754,7 @@ int ObTransformUtils::find_expr(ObIArray<ObRawExpr *> &source,
|
||||
if (OB_ISNULL(source.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("expr in source is null", K(ret));
|
||||
} else if (source.at(i) == target || source.at(i)->same_as(*target)) {
|
||||
} else if (source.at(i) == target || source.at(i)->same_as(*target, check_context)) {
|
||||
bret = true;
|
||||
}
|
||||
}
|
||||
@ -10369,7 +10371,7 @@ int ObTransformUtils::add_param_bool_constraint(ObTransformerCtx *ctx,
|
||||
|
||||
int ObTransformUtils::get_all_child_stmts(ObDMLStmt *stmt,
|
||||
ObIArray<ObSelectStmt*> &child_stmts,
|
||||
hash::ObHashMap<uint64_t, ObDMLStmt *> *parent_map)
|
||||
hash::ObHashMap<uint64_t, ObParentDMLStmt> *parent_map)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSEArray<ObSelectStmt*, 8> temp_stmts;
|
||||
@ -10379,6 +10381,7 @@ int ObTransformUtils::get_all_child_stmts(ObDMLStmt *stmt,
|
||||
} else if (OB_FAIL(stmt->get_child_stmts(temp_stmts))) {
|
||||
LOG_WARN("failed to get child stmts", K(ret));
|
||||
}
|
||||
int64_t child_stmt_size = temp_stmts.count();
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < stmt->get_table_size(); ++i) {
|
||||
TableItem *table = stmt->get_table_item(i);
|
||||
if (OB_ISNULL(table)) {
|
||||
@ -10400,10 +10403,18 @@ int ObTransformUtils::get_all_child_stmts(ObDMLStmt *stmt,
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret)&& i < temp_stmts.count(); ++i) {
|
||||
uint64_t key = reinterpret_cast<uint64_t>(temp_stmts.at(i));
|
||||
ObParentDMLStmt parent_stmt;
|
||||
if (i < child_stmt_size) {
|
||||
parent_stmt.stmt_ = stmt;
|
||||
parent_stmt.pos_ = i;
|
||||
} else {
|
||||
parent_stmt.stmt_ = NULL;
|
||||
parent_stmt.pos_ = OB_INVALID_ID;
|
||||
}
|
||||
if (OB_ISNULL(temp_stmts.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("temp stmt is null", K(ret));
|
||||
} else if (parent_map != NULL && OB_FAIL(parent_map->set_refactored(key, stmt))) {
|
||||
} else if (parent_map != NULL && OB_FAIL(parent_map->set_refactored(key, parent_stmt))) {
|
||||
LOG_WARN("failed to add parent child relation", K(ret));
|
||||
} else if (OB_FAIL(SMART_CALL(get_all_child_stmts(temp_stmts.at(i), child_stmts, parent_map)))) {
|
||||
LOG_WARN("failed to get all child stmts", K(ret));
|
||||
@ -11748,43 +11759,6 @@ int ObTransformUtils::is_batch_stmt_write_table(uint64_t table_id,
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTransformUtils::move_expr_into_view(ObRawExprFactory &expr_factory,
|
||||
ObDMLStmt &stmt,
|
||||
TableItem &view,
|
||||
ObIArray<ObRawExpr *> &conds,
|
||||
ObIArray<ObRawExpr *> &new_exprs,
|
||||
ObIArray<ObQueryRefRawExpr *> *moved_query_refs)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSEArray<ObQueryRefRawExpr *, 4> query_refs;
|
||||
ObSEArray<ObRawExpr *, 4> select_list;
|
||||
ObSEArray<ObRawExpr *, 4> column_list;
|
||||
ObRawExprCopier copier(expr_factory);
|
||||
if (OB_UNLIKELY(!view.is_generated_table()) || OB_ISNULL(view.ref_query_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("table item is not a view", K(ret), K(view));
|
||||
} else if (OB_FAIL(extract_query_ref_expr(conds, query_refs))) {
|
||||
LOG_WARN("failed to extract subquery", K(ret));
|
||||
} else if (OB_FAIL(stmt.get_view_output(view,
|
||||
select_list,
|
||||
column_list))) {
|
||||
LOG_WARN("failed to get view output", K(ret));
|
||||
} else if (OB_FAIL(copier.add_replaced_expr(column_list, select_list))) {
|
||||
LOG_WARN("failed to add replace pair", K(ret));
|
||||
} else if (OB_FAIL(copier.add_skipped_expr(query_refs, false))) {
|
||||
LOG_WARN("failed to add skipped expr", K(ret));
|
||||
} else if (OB_FAIL(copier.copy(conds, new_exprs))) {
|
||||
LOG_WARN("failed to copy on replace conditions", K(ret));
|
||||
} else if (OB_FAIL(ObOptimizerUtil::remove_item(stmt.get_subquery_exprs(), query_refs))) {
|
||||
LOG_WARN("failed to remove subquery exprs", K(ret));
|
||||
} else if (OB_FAIL(append(view.ref_query_->get_subquery_exprs(), query_refs))) {
|
||||
LOG_WARN("failed to append subquery exprs", K(ret));
|
||||
} else if (NULL != moved_query_refs && OB_FAIL(moved_query_refs->assign(query_refs))) {
|
||||
LOG_WARN("failed to assign moved query refs", K(ret));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
//new heap table not add partition key in rowkey and the tablet id is unique in partition
|
||||
//we need add partition key.
|
||||
int ObTransformUtils::add_part_column_exprs_for_heap_table(const ObDMLStmt *stmt,
|
||||
@ -13311,6 +13285,194 @@ int ObTransformUtils::check_is_index_part_key(ObTransformerCtx &ctx,
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTransformUtils::get_stmt_map_after_copy(ObDMLStmt *origin_stmt,
|
||||
ObDMLStmt *new_stmt,
|
||||
hash::ObHashMap<uint64_t, ObDMLStmt *> &stmt_map)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSEArray<ObSelectStmt *, 4> orgi_child_stmts;
|
||||
ObSEArray<ObSelectStmt *, 4> new_child_stmts;
|
||||
uint64_t key = reinterpret_cast<uint64_t>(origin_stmt);
|
||||
if (OB_ISNULL(origin_stmt) || OB_ISNULL(new_stmt)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("stmt is null", K(ret));
|
||||
} else if (OB_UNLIKELY(new_stmt->get_stmt_type() != origin_stmt->get_stmt_type())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("stmt type does not match", K(ret));
|
||||
} else if (OB_FAIL(origin_stmt->get_child_stmts(orgi_child_stmts))) {
|
||||
LOG_WARN("failed to get child stmts", K(ret));
|
||||
} else if (OB_FAIL(new_stmt->get_child_stmts(new_child_stmts))) {
|
||||
LOG_WARN("failed to get child stmts", K(ret));
|
||||
} else if (OB_UNLIKELY(orgi_child_stmts.count() != new_child_stmts.count()) ||
|
||||
OB_UNLIKELY(origin_stmt->get_table_size() != new_stmt->get_table_size())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("child stmt count does not match", K(ret));
|
||||
} else if (OB_FAIL(stmt_map.set_refactored(key, new_stmt))) {
|
||||
LOG_WARN("failed to add parent child relation", K(ret));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < origin_stmt->get_table_size(); ++i) {
|
||||
TableItem *origin_table = origin_stmt->get_table_item(i);
|
||||
TableItem *new_table = new_stmt->get_table_item(i);
|
||||
ObDMLStmt *dummy = NULL;
|
||||
key = reinterpret_cast<uint64_t>(origin_table->ref_query_);
|
||||
if (OB_ISNULL(origin_table) || OB_ISNULL(new_table)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect null table item", K(ret));
|
||||
} else if (!origin_table->is_temp_table()) {
|
||||
//do nothing
|
||||
} else if (!new_table->is_temp_table()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpected table type", K(ret), KPC(origin_table), KPC(new_table));
|
||||
} else if (OB_FAIL(stmt_map.get_refactored(key, dummy))) {
|
||||
if (ret == OB_HASH_NOT_EXIST) {
|
||||
ret = get_stmt_map_after_copy(origin_table->ref_query_,
|
||||
new_table->ref_query_,
|
||||
stmt_map);
|
||||
}
|
||||
}
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < orgi_child_stmts.count(); i ++) {
|
||||
if (OB_FAIL(SMART_CALL(get_stmt_map_after_copy(orgi_child_stmts.at(i),
|
||||
new_child_stmts.at(i),
|
||||
stmt_map)))) {
|
||||
LOG_WARN("failed to get child stmt map", K(ret));
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTransformUtils::expand_temp_table(ObTransformerCtx *ctx, ObDMLStmt::TempTableInfo& table_info)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
ObSelectStmt *temp_table_query = NULL;
|
||||
if (OB_ISNULL(ctx) || OB_ISNULL(ctx->stmt_factory_) ||
|
||||
OB_UNLIKELY(table_info.table_items_.count() != table_info.upper_stmts_.count())) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect null ctx", K(ret));
|
||||
}
|
||||
for (int64_t j = 0; OB_SUCC(ret) && j < table_info.table_items_.count(); ++j) {
|
||||
TableItem *table = table_info.table_items_.at(j);
|
||||
ObDMLStmt *upper_stmt = table_info.upper_stmts_.at(j);
|
||||
if (OB_ISNULL(table) || OB_ISNULL(upper_stmt)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect null table item", K(ret));
|
||||
} else if (!table->is_temp_table()) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("expect temp table item", KPC(table), KPC(upper_stmt), K(ret));
|
||||
} else {
|
||||
ObSelectStmt *child_stmt = NULL;
|
||||
table->type_ = TableItem::GENERATED_TABLE;
|
||||
if (0 == j) {
|
||||
temp_table_query = table->ref_query_;
|
||||
if (OB_ISNULL(temp_table_query)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect null stmt", K(ret));
|
||||
} else if (OB_FAIL(temp_table_query->formalize_stmt(ctx->session_info_))) {
|
||||
LOG_WARN("failed to formalize stmt", K(ret));
|
||||
} else if (OB_FAIL(temp_table_query->formalize_stmt_expr_reference())) {
|
||||
LOG_WARN("failed to formalize stmt reference", K(ret));
|
||||
} else if (OB_FAIL(upper_stmt->formalize_stmt_expr_reference())) {
|
||||
LOG_WARN("failed to formalize stmt reference", K(ret));
|
||||
}
|
||||
} else if (OB_FAIL(ctx->stmt_factory_->create_stmt<ObSelectStmt>(child_stmt))) {
|
||||
LOG_WARN("failed to create stmt", K(ret));
|
||||
} else if (OB_ISNULL(child_stmt) || OB_ISNULL(temp_table_query)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("unexpect null stmt", K(ret));
|
||||
} else if (OB_FAIL(child_stmt->deep_copy(*ctx->stmt_factory_,
|
||||
*ctx->expr_factory_,
|
||||
*temp_table_query))) {
|
||||
LOG_WARN("failed to deep copy stmt", K(ret));
|
||||
} else if (OB_FAIL(child_stmt->formalize_stmt(ctx->session_info_))) {
|
||||
LOG_WARN("failed to formalize stmt", K(ret));
|
||||
} else if (OB_FAIL(child_stmt->formalize_stmt_expr_reference())) {
|
||||
LOG_WARN("failed to formalize stmt reference", K(ret));
|
||||
} else if (OB_FAIL(child_stmt->recursive_adjust_statement_id(ctx->allocator_,
|
||||
ctx->src_hash_val_,
|
||||
j))) {
|
||||
LOG_WARN("failed to recursive adjust statement id", K(ret));
|
||||
} else if (OB_FAIL(child_stmt->update_stmt_table_id(*temp_table_query))) {
|
||||
LOG_WARN("failed to update table id", K(ret));
|
||||
} else if (OB_FAIL(upper_stmt->formalize_stmt_expr_reference())) {
|
||||
LOG_WARN("failed to formalize stmt reference", K(ret));
|
||||
} else {
|
||||
table->ref_query_ = child_stmt;
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
ObSelectStmtPointer::ObSelectStmtPointer() : stmt_group_() {
|
||||
}
|
||||
|
||||
ObSelectStmtPointer::~ObSelectStmtPointer() {
|
||||
}
|
||||
|
||||
int ObSelectStmtPointer::get(ObSelectStmt *&stmt) const
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (stmt_group_.count() <= 0
|
||||
|| OB_ISNULL(stmt_group_.at(0))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("stmt pointer is not set", K(ret));
|
||||
} else {
|
||||
stmt = *(stmt_group_.at(0));
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSelectStmtPointer::set(ObSelectStmt *stmt)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
for (int64_t i = 0; OB_SUCC(ret) && i < stmt_group_.count(); ++i) {
|
||||
if (OB_ISNULL(stmt_group_.at(i))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("stmt pointer is null", K(ret));
|
||||
} else if (*stmt_group_.at(i) == stmt) {
|
||||
// not changed
|
||||
break;
|
||||
} else {
|
||||
*stmt_group_.at(i) = stmt;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObSelectStmtPointer::add_ref(ObSelectStmt **stmt)
|
||||
{
|
||||
return stmt_group_.push_back(stmt);
|
||||
}
|
||||
|
||||
int ObTransformUtils::check_stmt_contain_oversize_set_stmt(ObDMLStmt *stmt, bool &is_contain)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
is_contain = false;
|
||||
if (OB_ISNULL(stmt)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("stmt is null", K(ret));
|
||||
} else if (stmt->is_select_stmt()) {
|
||||
int64_t size = 0;
|
||||
if (OB_FAIL(static_cast<ObSelectStmt *>(stmt)->get_set_stmt_size(size))) {
|
||||
LOG_WARN("get set stmt size failed", K(ret));
|
||||
} else if (size > ObTransformUtils::MAX_SET_STMT_SIZE_OF_COSTED_BASED_RELUES) {
|
||||
is_contain = true;
|
||||
}
|
||||
}
|
||||
if (OB_SUCC(ret) && !is_contain) {
|
||||
ObSEArray<ObSelectStmt*, 4> child_stmts;
|
||||
if (OB_FAIL(stmt->get_child_stmts(child_stmts))) {
|
||||
LOG_WARN("failed to get child stmts", K(ret));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && !is_contain && i < child_stmts.count(); i++) {
|
||||
if (OB_FAIL(SMART_CALL(check_stmt_contain_oversize_set_stmt(child_stmts.at(i), is_contain)))) {
|
||||
LOG_WARN("get_stmt_trans_info failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTransformUtils::convert_preds_vector_to_scalar(ObTransformerCtx &ctx,
|
||||
ObRawExpr *expr,
|
||||
ObIArray<ObRawExpr*> &exprs,
|
||||
|
@ -116,6 +116,19 @@ public:
|
||||
ObArray<ObRawExpr *> filters_;
|
||||
};
|
||||
|
||||
class ObSelectStmtPointer {
|
||||
public:
|
||||
ObSelectStmtPointer();
|
||||
|
||||
virtual ~ObSelectStmtPointer();
|
||||
int get(ObSelectStmt *&stmt) const;
|
||||
int set(ObSelectStmt *stmt);
|
||||
int add_ref(ObSelectStmt **stmt);
|
||||
int64_t ref_count() const { return stmt_group_.count(); }
|
||||
TO_STRING_KV("", "");
|
||||
private:
|
||||
common::ObSEArray<ObSelectStmt **, 1> stmt_group_;
|
||||
};
|
||||
|
||||
class ObTransformUtils
|
||||
{
|
||||
@ -156,6 +169,7 @@ class ObTransformUtils
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(UniqueCheckHelper);
|
||||
};
|
||||
static const uint64_t MAX_SET_STMT_SIZE_OF_COSTED_BASED_RELUES = 5;
|
||||
|
||||
public:
|
||||
struct LazyJoinInfo {
|
||||
@ -515,11 +529,13 @@ public:
|
||||
*/
|
||||
static int find_expr(const ObIArray<const ObRawExpr *> &source,
|
||||
const ObRawExpr *target,
|
||||
bool &bret);
|
||||
bool &bret,
|
||||
ObExprEqualCheckContext *check_context = NULL);
|
||||
|
||||
static int find_expr(ObIArray<ObRawExpr *> &source,
|
||||
ObRawExpr *target,
|
||||
bool &bret);
|
||||
bool &bret,
|
||||
ObExprEqualCheckContext *check_context = NULL);
|
||||
|
||||
static int find_expr(const ObIArray<OrderItem> &source,
|
||||
const ObRawExpr *target,
|
||||
@ -1456,7 +1472,7 @@ public:
|
||||
|
||||
static int get_all_child_stmts(ObDMLStmt *stmt,
|
||||
ObIArray<ObSelectStmt*> &child_stmts,
|
||||
hash::ObHashMap<uint64_t, ObDMLStmt *> *parent_map = NULL);
|
||||
hash::ObHashMap<uint64_t, ObParentDMLStmt> *parent_map = NULL);
|
||||
|
||||
static int check_select_expr_is_const(ObSelectStmt *stmt, ObRawExpr *expr, bool &is_const);
|
||||
|
||||
@ -1592,13 +1608,6 @@ public:
|
||||
const ObDMLStmt &stmt,
|
||||
bool &is_target_table);
|
||||
|
||||
static int move_expr_into_view(ObRawExprFactory &expr_factory,
|
||||
ObDMLStmt &stmt,
|
||||
TableItem &view,
|
||||
ObIArray<ObRawExpr *> &exprs,
|
||||
ObIArray<ObRawExpr *> &new_exprs,
|
||||
ObIArray<ObQueryRefRawExpr *> *moved_query_refs = NULL);
|
||||
|
||||
static int get_generated_table_item(ObDMLStmt &parent_stmt,
|
||||
ObDMLStmt *child_stmt,
|
||||
TableItem *&table_item);
|
||||
@ -1773,6 +1782,15 @@ public:
|
||||
ObIArray<ObRawExpr *> &relation_exprs,
|
||||
ObIArray<ObRawExpr *> &common_exprs);
|
||||
static int check_is_index_part_key(ObTransformerCtx &ctx, ObDMLStmt &stmt, ObRawExpr *check_expr, bool &is_valid);
|
||||
|
||||
static int expand_temp_table(ObTransformerCtx *ctx, ObDMLStmt::TempTableInfo& table_info);
|
||||
|
||||
static int get_stmt_map_after_copy(ObDMLStmt *origin_stmt,
|
||||
ObDMLStmt *new_stmt,
|
||||
hash::ObHashMap<uint64_t, ObDMLStmt *> &stmt_map);
|
||||
|
||||
static int check_stmt_contain_oversize_set_stmt(ObDMLStmt *stmt, bool &is_contain);
|
||||
|
||||
static int convert_preds_vector_to_scalar(ObTransformerCtx &ctx,
|
||||
ObRawExpr *expr,
|
||||
ObIArray<ObRawExpr*> &exprs,
|
||||
|
@ -68,7 +68,7 @@ int ObTransformerImpl::transform(ObDMLStmt *&stmt)
|
||||
LOG_WARN("failed to do transform dblink write", K(ret));
|
||||
} else if (trans_happended) {
|
||||
//dml write query will be executed in remote, do not need transform
|
||||
} else if (OB_FAIL(SMART_CALL(get_stmt_trans_info(stmt)))) {
|
||||
} else if (OB_FAIL(SMART_CALL(get_stmt_trans_info(stmt, true)))) {
|
||||
LOG_WARN("get_stmt_trans_info failed", K(ret));
|
||||
} else if (OB_FAIL(do_transform_pre_precessing(stmt))) {
|
||||
LOG_WARN("failed to do transform pre_precessing", K(ret));
|
||||
@ -90,28 +90,24 @@ int ObTransformerImpl::transform(ObDMLStmt *&stmt)
|
||||
return ret;
|
||||
}
|
||||
|
||||
int ObTransformerImpl::get_stmt_trans_info(ObDMLStmt *stmt)
|
||||
int ObTransformerImpl::get_stmt_trans_info(ObDMLStmt *stmt, bool is_root)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
if (OB_ISNULL(stmt) || OB_ISNULL(ctx_)) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
LOG_WARN("stmt is null", K(ret));
|
||||
} else if (stmt->is_select_stmt()) {
|
||||
int64_t size = 0;
|
||||
if (OB_FAIL(static_cast<ObSelectStmt *>(stmt)->get_set_stmt_size(size))) {
|
||||
LOG_WARN("get set stmt size failed", K(ret));
|
||||
} else if (size > ObTransformerImpl::MAX_SET_STMT_SIZE_OF_COSTED_BASED_RELUES) {
|
||||
ctx_->is_set_stmt_oversize_ = true;
|
||||
}
|
||||
} else if (OB_FAIL(ObTransformUtils::check_stmt_contain_oversize_set_stmt(stmt, ctx_->is_set_stmt_oversize_))) {
|
||||
LOG_WARN("failed to check set stmt oversize");
|
||||
}
|
||||
if (OB_SUCC(ret) && !ctx_->is_set_stmt_oversize_) {
|
||||
ObSEArray<ObSelectStmt*, 4> child_stmts;
|
||||
if (OB_FAIL(stmt->get_child_stmts(child_stmts))) {
|
||||
LOG_WARN("failed to get child stmts", K(ret));
|
||||
if (OB_SUCC(ret) && !ctx_->is_set_stmt_oversize_ && is_root) {
|
||||
ObArray<ObDMLStmt::TempTableInfo> temp_table_infos;
|
||||
if (OB_FAIL(stmt->collect_temp_table_infos(temp_table_infos))) {
|
||||
LOG_WARN("failed to collect temp table infos", K(ret));
|
||||
}
|
||||
for (int64_t i = 0; OB_SUCC(ret) && !ctx_->is_set_stmt_oversize_&& i < child_stmts.count(); i++) {
|
||||
if (OB_FAIL(SMART_CALL(get_stmt_trans_info(child_stmts.at(i))))) {
|
||||
LOG_WARN("get_stmt_trans_info failed", K(ret));
|
||||
for (int64_t i = 0; OB_SUCC(ret) && !ctx_->is_set_stmt_oversize_ && i < temp_table_infos.count(); i++) {
|
||||
if (OB_FAIL(SMART_CALL(ObTransformUtils::check_stmt_contain_oversize_set_stmt(temp_table_infos.at(i).temp_table_query_,
|
||||
ctx_->is_set_stmt_oversize_)))) {
|
||||
LOG_WARN("check_contain_oversize_set_stmt failed", K(ret));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -42,7 +42,6 @@ class ObTransformerImpl
|
||||
{
|
||||
static const int64_t DEFAULT_ITERATION_COUNT = 10;
|
||||
static const int64_t MAX_RULE_COUNT = 64;
|
||||
static const uint64_t MAX_SET_STMT_SIZE_OF_COSTED_BASED_RELUES = 5;
|
||||
public:
|
||||
ObTransformerImpl(ObTransformerCtx *ctx)
|
||||
: ctx_(ctx),
|
||||
@ -147,7 +146,7 @@ public:
|
||||
private:
|
||||
|
||||
int collect_trans_stat(const ObTransformRule &rule);
|
||||
int get_stmt_trans_info(ObDMLStmt *stmt);
|
||||
int get_stmt_trans_info(ObDMLStmt *stmt, bool is_root);
|
||||
void print_trans_stat();
|
||||
|
||||
int finalize_exec_params(ObDMLStmt *stmt);
|
||||
|
@ -2433,22 +2433,22 @@ Query Plan
|
||||
==================================================================
|
||||
|ID|OPERATOR |NAME |EST.ROWS|EST.TIME(us)|
|
||||
------------------------------------------------------------------
|
||||
|0 |SORT | |11 |27 |
|
||||
|1 |└─SUBPLAN FILTER | |11 |26 |
|
||||
|0 |SORT | |11 |26 |
|
||||
|1 |└─SUBPLAN FILTER | |11 |25 |
|
||||
|2 | ├─TABLE FULL SCAN |x |11 |5 |
|
||||
|3 | └─LIMIT | |1 |21 |
|
||||
|4 | └─NESTED-LOOP JOIN CARTESIAN | |1 |21 |
|
||||
|5 | ├─SUBPLAN SCAN |VIEW1|1 |11 |
|
||||
|6 | │ └─LIMIT | |1 |11 |
|
||||
|7 | │ └─HASH JOIN | |1 |11 |
|
||||
|8 | │ ├─TABLE FULL SCAN |b |11 |5 |
|
||||
|9 | │ └─TABLE FULL SCAN |a |4 |4 |
|
||||
|10| └─MATERIAL | |1 |11 |
|
||||
|11| └─SUBPLAN SCAN |VIEW2|1 |11 |
|
||||
|12| └─LIMIT | |1 |11 |
|
||||
|13| └─HASH JOIN | |1 |11 |
|
||||
|14| ├─TABLE FULL SCAN |b |11 |5 |
|
||||
|15| └─TABLE FULL SCAN |a |3 |4 |
|
||||
|3 | └─LIMIT | |1 |20 |
|
||||
|4 | └─NESTED-LOOP JOIN CARTESIAN | |1 |20 |
|
||||
|5 | ├─SUBPLAN SCAN |VIEW1|1 |10 |
|
||||
|6 | │ └─LIMIT | |1 |10 |
|
||||
|7 | │ └─HASH JOIN | |1 |10 |
|
||||
|8 | │ ├─TABLE FULL SCAN |a |11 |5 |
|
||||
|9 | │ └─TABLE FULL SCAN |b |1 |4 |
|
||||
|10| └─MATERIAL | |1 |10 |
|
||||
|11| └─SUBPLAN SCAN |VIEW2|1 |10 |
|
||||
|12| └─LIMIT | |1 |10 |
|
||||
|13| └─HASH JOIN | |1 |10 |
|
||||
|14| ├─TABLE FULL SCAN |c |11 |5 |
|
||||
|15| └─TABLE FULL SCAN |d |1 |4 |
|
||||
==================================================================
|
||||
Outputs & filters:
|
||||
-------------------------------------
|
||||
@ -2469,30 +2469,30 @@ Outputs & filters:
|
||||
6 - output([1]), filter(nil), rowset=256
|
||||
limit(1), offset(nil)
|
||||
7 - output(nil), filter(nil), rowset=256
|
||||
equal_conds([a.c1 = b.d1]), other_conds([a.c1 = b.d1 OR a.c2 = b.d1])
|
||||
8 - output([b.d1]), filter(nil), rowset=256
|
||||
equal_conds([a.c1 = b.d1]), other_conds(nil)
|
||||
8 - output([a.c1]), filter(nil), rowset=256
|
||||
access([a.c1]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([a.__pk_increment]), range(MIN ; MAX)always true
|
||||
9 - output([b.d1]), filter(nil), rowset=256
|
||||
access([b.d1]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([b.__pk_increment]), range(MIN ; MAX)always true
|
||||
9 - output([a.c1], [a.c2]), filter(nil), rowset=256
|
||||
access([a.c1], [a.c2]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([a.__pk_increment]), range(MIN ; MAX)always true
|
||||
10 - output(nil), filter(nil), rowset=256
|
||||
11 - output(nil), filter(nil), rowset=256
|
||||
access(nil)
|
||||
12 - output([1]), filter(nil), rowset=256
|
||||
limit(1), offset(nil)
|
||||
13 - output(nil), filter(nil), rowset=256
|
||||
equal_conds([a.c2 = b.d1]), other_conds([a.c1 = b.d1 OR a.c2 = b.d1])
|
||||
14 - output([b.d1]), filter(nil), rowset=256
|
||||
access([b.d1]), partitions(p0)
|
||||
equal_conds([c.c2 = d.d1]), other_conds(nil)
|
||||
14 - output([c.c2]), filter(nil), rowset=256
|
||||
access([c.c2]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([b.__pk_increment]), range(MIN ; MAX)always true
|
||||
15 - output([a.c1], [a.c2]), filter(nil), rowset=256
|
||||
access([a.c1], [a.c2]), partitions(p0)
|
||||
range_key([c.__pk_increment]), range(MIN ; MAX)always true
|
||||
15 - output([d.d1]), filter(nil), rowset=256
|
||||
access([d.d1]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([a.__pk_increment]), range(MIN ; MAX)always true
|
||||
range_key([d.__pk_increment]), range(MIN ; MAX)always true
|
||||
select /*+ LEADING(@x2x (c d), (a b)) */
|
||||
* from nn1 x
|
||||
where exists (select /*+ qb_name(x2x) */ 1
|
||||
@ -2523,22 +2523,22 @@ Query Plan
|
||||
==================================================================
|
||||
|ID|OPERATOR |NAME |EST.ROWS|EST.TIME(us)|
|
||||
------------------------------------------------------------------
|
||||
|0 |SORT | |11 |27 |
|
||||
|1 |└─SUBPLAN FILTER | |11 |26 |
|
||||
|0 |SORT | |11 |26 |
|
||||
|1 |└─SUBPLAN FILTER | |11 |25 |
|
||||
|2 | ├─TABLE FULL SCAN |x |11 |5 |
|
||||
|3 | └─LIMIT | |1 |21 |
|
||||
|4 | └─NESTED-LOOP JOIN CARTESIAN | |1 |21 |
|
||||
|5 | ├─SUBPLAN SCAN |VIEW1|1 |11 |
|
||||
|6 | │ └─LIMIT | |1 |11 |
|
||||
|7 | │ └─HASH JOIN | |1 |11 |
|
||||
|8 | │ ├─TABLE FULL SCAN |b |11 |5 |
|
||||
|9 | │ └─TABLE FULL SCAN |a |4 |4 |
|
||||
|10| └─MATERIAL | |1 |11 |
|
||||
|11| └─SUBPLAN SCAN |VIEW2|1 |11 |
|
||||
|12| └─LIMIT | |1 |11 |
|
||||
|13| └─HASH JOIN | |1 |11 |
|
||||
|14| ├─TABLE FULL SCAN |b |11 |5 |
|
||||
|15| └─TABLE FULL SCAN |a |3 |4 |
|
||||
|3 | └─LIMIT | |1 |20 |
|
||||
|4 | └─NESTED-LOOP JOIN CARTESIAN | |1 |20 |
|
||||
|5 | ├─SUBPLAN SCAN |VIEW1|1 |10 |
|
||||
|6 | │ └─LIMIT | |1 |10 |
|
||||
|7 | │ └─HASH JOIN | |1 |10 |
|
||||
|8 | │ ├─TABLE FULL SCAN |a |11 |5 |
|
||||
|9 | │ └─TABLE FULL SCAN |b |1 |4 |
|
||||
|10| └─MATERIAL | |1 |10 |
|
||||
|11| └─SUBPLAN SCAN |VIEW2|1 |10 |
|
||||
|12| └─LIMIT | |1 |10 |
|
||||
|13| └─HASH JOIN | |1 |10 |
|
||||
|14| ├─TABLE FULL SCAN |c |11 |5 |
|
||||
|15| └─TABLE FULL SCAN |d |1 |4 |
|
||||
==================================================================
|
||||
Outputs & filters:
|
||||
-------------------------------------
|
||||
@ -2559,30 +2559,30 @@ Outputs & filters:
|
||||
6 - output([1]), filter(nil), rowset=256
|
||||
limit(1), offset(nil)
|
||||
7 - output(nil), filter(nil), rowset=256
|
||||
equal_conds([a.c1 = b.d1]), other_conds([a.c1 = b.d1 OR a.c2 = b.d1])
|
||||
8 - output([b.d1]), filter(nil), rowset=256
|
||||
equal_conds([a.c1 = b.d1]), other_conds(nil)
|
||||
8 - output([a.c1]), filter(nil), rowset=256
|
||||
access([a.c1]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([a.__pk_increment]), range(MIN ; MAX)always true
|
||||
9 - output([b.d1]), filter(nil), rowset=256
|
||||
access([b.d1]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([b.__pk_increment]), range(MIN ; MAX)always true
|
||||
9 - output([a.c1], [a.c2]), filter(nil), rowset=256
|
||||
access([a.c1], [a.c2]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([a.__pk_increment]), range(MIN ; MAX)always true
|
||||
10 - output(nil), filter(nil), rowset=256
|
||||
11 - output(nil), filter(nil), rowset=256
|
||||
access(nil)
|
||||
12 - output([1]), filter(nil), rowset=256
|
||||
limit(1), offset(nil)
|
||||
13 - output(nil), filter(nil), rowset=256
|
||||
equal_conds([a.c2 = b.d1]), other_conds([a.c1 = b.d1 OR a.c2 = b.d1])
|
||||
14 - output([b.d1]), filter(nil), rowset=256
|
||||
access([b.d1]), partitions(p0)
|
||||
equal_conds([c.c2 = d.d1]), other_conds(nil)
|
||||
14 - output([c.c2]), filter(nil), rowset=256
|
||||
access([c.c2]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([b.__pk_increment]), range(MIN ; MAX)always true
|
||||
15 - output([a.c1], [a.c2]), filter(nil), rowset=256
|
||||
access([a.c1], [a.c2]), partitions(p0)
|
||||
range_key([c.__pk_increment]), range(MIN ; MAX)always true
|
||||
15 - output([d.d1]), filter(nil), rowset=256
|
||||
access([d.d1]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([a.__pk_increment]), range(MIN ; MAX)always true
|
||||
range_key([d.__pk_increment]), range(MIN ; MAX)always true
|
||||
select /*+ LEADING(@x2x (c d), (a b)) */
|
||||
* from nn1 x
|
||||
where exists (select /*+ qb_name(x2x) LEADING(a c b d)*/ 1
|
||||
@ -2613,22 +2613,22 @@ Query Plan
|
||||
==================================================================
|
||||
|ID|OPERATOR |NAME |EST.ROWS|EST.TIME(us)|
|
||||
------------------------------------------------------------------
|
||||
|0 |SORT | |11 |27 |
|
||||
|1 |└─SUBPLAN FILTER | |11 |26 |
|
||||
|0 |SORT | |11 |26 |
|
||||
|1 |└─SUBPLAN FILTER | |11 |25 |
|
||||
|2 | ├─TABLE FULL SCAN |x |11 |5 |
|
||||
|3 | └─LIMIT | |1 |21 |
|
||||
|4 | └─NESTED-LOOP JOIN CARTESIAN | |1 |21 |
|
||||
|5 | ├─SUBPLAN SCAN |VIEW1|1 |11 |
|
||||
|6 | │ └─LIMIT | |1 |11 |
|
||||
|7 | │ └─HASH JOIN | |1 |11 |
|
||||
|8 | │ ├─TABLE FULL SCAN |b |11 |5 |
|
||||
|9 | │ └─TABLE FULL SCAN |a |4 |4 |
|
||||
|10| └─MATERIAL | |1 |11 |
|
||||
|11| └─SUBPLAN SCAN |VIEW2|1 |11 |
|
||||
|12| └─LIMIT | |1 |11 |
|
||||
|13| └─HASH JOIN | |1 |11 |
|
||||
|14| ├─TABLE FULL SCAN |b |11 |5 |
|
||||
|15| └─TABLE FULL SCAN |a |3 |4 |
|
||||
|3 | └─LIMIT | |1 |20 |
|
||||
|4 | └─NESTED-LOOP JOIN CARTESIAN | |1 |20 |
|
||||
|5 | ├─SUBPLAN SCAN |VIEW1|1 |10 |
|
||||
|6 | │ └─LIMIT | |1 |10 |
|
||||
|7 | │ └─HASH JOIN | |1 |10 |
|
||||
|8 | │ ├─TABLE FULL SCAN |a |11 |5 |
|
||||
|9 | │ └─TABLE FULL SCAN |b |1 |4 |
|
||||
|10| └─MATERIAL | |1 |10 |
|
||||
|11| └─SUBPLAN SCAN |VIEW2|1 |10 |
|
||||
|12| └─LIMIT | |1 |10 |
|
||||
|13| └─HASH JOIN | |1 |10 |
|
||||
|14| ├─TABLE FULL SCAN |c |11 |5 |
|
||||
|15| └─TABLE FULL SCAN |d |1 |4 |
|
||||
==================================================================
|
||||
Outputs & filters:
|
||||
-------------------------------------
|
||||
@ -2649,30 +2649,30 @@ Outputs & filters:
|
||||
6 - output([1]), filter(nil), rowset=256
|
||||
limit(1), offset(nil)
|
||||
7 - output(nil), filter(nil), rowset=256
|
||||
equal_conds([a.c1 = b.d1]), other_conds([a.c1 = b.d1 OR a.c2 = b.d1])
|
||||
8 - output([b.d1]), filter(nil), rowset=256
|
||||
equal_conds([a.c1 = b.d1]), other_conds(nil)
|
||||
8 - output([a.c1]), filter(nil), rowset=256
|
||||
access([a.c1]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([a.__pk_increment]), range(MIN ; MAX)always true
|
||||
9 - output([b.d1]), filter(nil), rowset=256
|
||||
access([b.d1]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([b.__pk_increment]), range(MIN ; MAX)always true
|
||||
9 - output([a.c1], [a.c2]), filter(nil), rowset=256
|
||||
access([a.c1], [a.c2]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([a.__pk_increment]), range(MIN ; MAX)always true
|
||||
10 - output(nil), filter(nil), rowset=256
|
||||
11 - output(nil), filter(nil), rowset=256
|
||||
access(nil)
|
||||
12 - output([1]), filter(nil), rowset=256
|
||||
limit(1), offset(nil)
|
||||
13 - output(nil), filter(nil), rowset=256
|
||||
equal_conds([a.c2 = b.d1]), other_conds([a.c1 = b.d1 OR a.c2 = b.d1])
|
||||
14 - output([b.d1]), filter(nil), rowset=256
|
||||
access([b.d1]), partitions(p0)
|
||||
equal_conds([c.c2 = d.d1]), other_conds(nil)
|
||||
14 - output([c.c2]), filter(nil), rowset=256
|
||||
access([c.c2]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([b.__pk_increment]), range(MIN ; MAX)always true
|
||||
15 - output([a.c1], [a.c2]), filter(nil), rowset=256
|
||||
access([a.c1], [a.c2]), partitions(p0)
|
||||
range_key([c.__pk_increment]), range(MIN ; MAX)always true
|
||||
15 - output([d.d1]), filter(nil), rowset=256
|
||||
access([d.d1]), partitions(p0)
|
||||
is_index_back=false, is_global_index=false,
|
||||
range_key([a.__pk_increment]), range(MIN ; MAX)always true
|
||||
range_key([d.__pk_increment]), range(MIN ; MAX)always true
|
||||
select /*+ LEADING(@x3x b d (a c)) */
|
||||
* from nn1 x
|
||||
where exists (select /*+ qb_name(x3x) */ 1
|
||||
|
Loading…
x
Reference in New Issue
Block a user