diff --git a/deps/oblib/src/common/object/ob_object.cpp b/deps/oblib/src/common/object/ob_object.cpp index 2aabbc397d..5f13214a16 100644 --- a/deps/oblib/src/common/object/ob_object.cpp +++ b/deps/oblib/src/common/object/ob_object.cpp @@ -2364,6 +2364,25 @@ int ObObj::convert_string_value_charset(ObCharsetType charset_type, ObIAllocator return ret; } +int ObObj::get_real_param_count(int64_t &count) const +{ + int ret = OB_SUCCESS; + count = 1; + if (ObExtendType == meta_.get_type()) { + const ObSqlArrayObj *array_obj = NULL; + if (OB_ISNULL(array_obj = reinterpret_cast(v_.ext_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected nullptr", K(ret), K(v_.ext_)); + } else if (array_obj->count_ < 0) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected group_idx", K(ret), K(array_obj->count_)); + } else { + count = array_obj->count_; + } + } + return ret; +} + //////////////////////////////////////////////////////////////// DEFINE_SERIALIZE(ObObj) { diff --git a/deps/oblib/src/common/object/ob_object.h b/deps/oblib/src/common/object/ob_object.h index e57df27667..ef3ce33254 100644 --- a/deps/oblib/src/common/object/ob_object.h +++ b/deps/oblib/src/common/object/ob_object.h @@ -1729,6 +1729,7 @@ public: OB_INLINE bool get_bool() const { return (0 != v_.int64_); } inline int64_t get_ext() const; + int get_real_param_count(int64_t &count) const; OB_INLINE int64_t get_unknown() const { return v_.unknown_; } OB_INLINE uint64_t get_bit() const { return v_.uint64_; } OB_INLINE uint64_t get_enum() const { return v_.uint64_; } diff --git a/deps/oblib/src/lib/random/ob_random.cpp b/deps/oblib/src/lib/random/ob_random.cpp index 6f939a8bad..2e3576c506 100644 --- a/deps/oblib/src/lib/random/ob_random.cpp +++ b/deps/oblib/src/lib/random/ob_random.cpp @@ -31,6 +31,14 @@ ObRandom::~ObRandom() { } +void ObRandom::seed(const uint64_t seed) +{ + seed_[0] = (uint16_t)seed; + seed_[1] = (uint16_t)(seed >> 16); + seed_[2] = (uint16_t)(seed >> 32); + seed48(seed_); +} + int64_t ObRandom::rand(const int64_t a, const int64_t b) { struct Wrapper { diff --git a/deps/oblib/src/lib/random/ob_random.h b/deps/oblib/src/lib/random/ob_random.h index 9f6460cdaa..870a99968b 100644 --- a/deps/oblib/src/lib/random/ob_random.h +++ b/deps/oblib/src/lib/random/ob_random.h @@ -27,6 +27,7 @@ public: ObRandom(); virtual ~ObRandom(); //get a random int64_t number in [min(a,b), max(a,b)] + void seed(const uint64_t seed); static int64_t rand(const int64_t a, const int64_t b); //get a random int64_t number int64_t get(); diff --git a/src/share/parameter/ob_parameter_seed.ipp b/src/share/parameter/ob_parameter_seed.ipp index 2dae16382f..2c53ea7951 100644 --- a/src/share/parameter/ob_parameter_seed.ipp +++ b/src/share/parameter/ob_parameter_seed.ipp @@ -1815,6 +1815,9 @@ DEF_TIME(_ha_rpc_timeout, OB_TENANT_PARAMETER, "0", "[0,120s]", DEF_BOOL(_enable_trace_tablet_leak, OB_TENANT_PARAMETER, "False", "enable t3m tablet leak checker. The default value is False", ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::STATIC_EFFECTIVE)); +DEF_INT(_inlist_rewrite_threshold, OB_TENANT_PARAMETER, "2147483647", "[1, 2147483647]" + "specifies transform how much const params in IN list to values table", + ObParameterAttr(Section::TENANT, Source::DEFAULT, EditLevel::DYNAMIC_EFFECTIVE)); // for set errsim module types, format like transfer;migration ERRSIM_DEF_STR_LIST(errsim_module_types, OB_TENANT_PARAMETER, "", diff --git a/src/sql/CMakeLists.txt b/src/sql/CMakeLists.txt index d7bee10148..f64414dde1 100644 --- a/src/sql/CMakeLists.txt +++ b/src/sql/CMakeLists.txt @@ -207,6 +207,7 @@ ob_set_subtarget(ob_sql engine_basic engine/basic/chunk_store/ob_chunk_block_compressor.cpp #engine/basic/chunk_store/ob_encoded_block_writer.cpp #engine/basic/chunk_store/ob_encoded_block_reader.cpp + engine/basic/ob_values_table_access_op.cpp ) ob_set_subtarget(ob_sql engine_cmd @@ -994,6 +995,7 @@ ob_set_subtarget(ob_sql optimizer optimizer/ob_log_stat_collector.cpp optimizer/ob_log_optimizer_stats_gathering.cpp optimizer/ob_dynamic_sampling.cpp + optimizer/ob_log_values_table_access.cpp ) ob_set_subtarget(ob_sql plan_cache @@ -1233,6 +1235,7 @@ ob_set_subtarget(ob_sql resolver_dml resolver/dml/ob_update_resolver.cpp resolver/dml/ob_update_stmt.cpp resolver/dml/ob_view_table_resolver.cpp + resolver/dml/ob_inlist_resolver.cpp ) ob_set_subtarget(ob_sql resolver_expr @@ -1304,6 +1307,7 @@ ob_set_subtarget(ob_sql rewrite rewrite/ob_transform_expr_pullup.cpp rewrite/ob_transformer_impl.cpp rewrite/ob_transform_dblink.cpp + rewrite/ob_transform_conditional_aggr_coalesce.cpp rewrite/ob_transform_mv_rewrite.cpp rewrite/ob_union_find.cpp rewrite/ob_transform_decorrelate.cpp diff --git a/src/sql/code_generator/ob_static_engine_cg.cpp b/src/sql/code_generator/ob_static_engine_cg.cpp index 6d2d2c1a07..384c675652 100644 --- a/src/sql/code_generator/ob_static_engine_cg.cpp +++ b/src/sql/code_generator/ob_static_engine_cg.cpp @@ -157,6 +157,8 @@ #ifdef OB_BUILD_TDE_SECURITY #include "share/ob_master_key_getter.h" #endif +#include "sql/optimizer/ob_log_values_table_access.h" +#include "sql/engine/basic/ob_values_table_access_op.h" namespace oceanbase { @@ -2553,6 +2555,81 @@ int ObStaticEngineCG::generate_spec(ObLogExprValues &op, return ret; } +int ObStaticEngineCG::generate_spec(ObLogValuesTableAccess &op, + ObValuesTableAccessSpec &spec, + const bool in_root_job) +{ + int ret = OB_SUCCESS; + UNUSED(in_root_job); + const ObValuesTableDef *table_def = NULL; + if (OB_ISNULL(table_def = op.get_values_table_def()) || + OB_UNLIKELY(op.get_output_exprs().empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected param", K(ret), KP(table_def), K(op.get_output_exprs())); + } else { + spec.access_type_ = table_def->access_type_; + spec.start_param_idx_ = table_def->start_param_idx_; + spec.end_param_idx_ = table_def->end_param_idx_; + ObIAllocator &allocator = phy_plan_->get_allocator(); + if (OB_FAIL(spec.column_exprs_.prepare_allocate(op.get_column_exprs().count()))) { + LOG_WARN("init fixed array failed", K(ret), K(op.get_column_exprs().count())); + } else if (OB_FAIL(spec.value_exprs_.prepare_allocate(table_def->access_exprs_.count()))) { + LOG_WARN("init fixed array failed", K(ret), K(table_def->access_exprs_.count())); + } else if (OB_FAIL(spec.obj_params_.prepare_allocate(table_def->access_objs_.count()))) { + LOG_WARN("init fixed array failed", K(ret), K(table_def->access_objs_.count())); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < op.get_column_exprs().count(); i++) { + ObColumnRefRawExpr *col_expr = op.get_column_exprs().at(i); + ObExpr *expr = NULL; + if (OB_ISNULL(col_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("raw_expr is null", K(ret), K(i), K(col_expr)); + } else if (OB_FAIL(mark_expr_self_produced(col_expr))) { + LOG_WARN("mark expr self produced failed", K(ret), KPC(col_expr)); + } else if (OB_FAIL(generate_rt_expr(*col_expr, expr))) { + LOG_WARN("fail to generate_rt_expr", K(ret), K(i), KPC(col_expr)); + } else if (OB_ISNULL(expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("value_info.expr_ is null", K(ret), K(i), KPC(expr)); + } else { + spec.column_exprs_.at(i) = expr; + } + } + for (int64_t i = 0; OB_SUCC(ret) && i < table_def->access_exprs_.count(); i++) { + ObRawExpr *raw_expr = table_def->access_exprs_.at(i); + ObExpr *expr = NULL; + if (OB_ISNULL(raw_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("raw_expr is null", K(ret), K(i), K(raw_expr)); + } else if (OB_FAIL(mark_expr_self_produced(raw_expr))) { + LOG_WARN("mark expr self produced failed", K(ret), KPC(raw_expr)); + } else if (OB_FAIL(generate_rt_expr(*raw_expr, expr))) { + LOG_WARN("fail to generate_rt_expr", K(ret), K(i), KPC(raw_expr)); + } else if (OB_ISNULL(expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("value_info.expr_ is null", K(ret), K(i), KPC(raw_expr)); + } else { + spec.value_exprs_.at(i) = expr; + } + } + for (int64_t i = 0; OB_SUCC(ret) && i < table_def->access_objs_.count(); i++) { + if (OB_FAIL(ob_write_obj(allocator, + table_def->access_objs_.at(i), + spec.obj_params_.at(i)))) { + LOG_WARN("failed to write obj", K(ret)); + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(mark_expr_self_produced(op.get_output_exprs()))) { + LOG_WARN("mark expr self produced failed", K(ret)); + } else { + spec.rows_ = table_def->row_cnt_; + } + } + } + return ret; +} + int ObStaticEngineCG::generate_spec(ObLogMerge &op, ObTableMergeSpec &spec, const bool in_root_job) @@ -8632,6 +8709,10 @@ int ObStaticEngineCG::get_phy_op_type(ObLogicalOperator &log_op, type = PHY_OPTIMIZER_STATS_GATHERING; break; } + case log_op_def::LOG_VALUES_TABLE_ACCESS: { + type = PHY_VALUES_TABLE_ACCESS; + break; + } default: ret = OB_ERR_UNEXPECTED; LOG_WARN("unknown logical operator", K(log_op.get_type()), K(lbt())); diff --git a/src/sql/code_generator/ob_static_engine_cg.h b/src/sql/code_generator/ob_static_engine_cg.h index 49847b51b7..5c6eaf8e2e 100644 --- a/src/sql/code_generator/ob_static_engine_cg.h +++ b/src/sql/code_generator/ob_static_engine_cg.h @@ -131,6 +131,9 @@ struct ObDASScanCtDef; struct InsertAllTableInfo; class ObHashDistinctVecSpec; class ObSortVecSpec; +class ObLogValuesTableAccess; +class ObValuesTableAccessSpec; + typedef common::ObList DASTableIdList; typedef common::ObSEArray, 1, common::ModulePageAllocator, true> RowParamMap; @@ -352,6 +355,7 @@ private: int set_partition_range_info(ObLogTableScan &op, ObTableScanSpec &spec); int generate_spec(ObLogExprValues &op, ObExprValuesSpec &spec, const bool in_root_job); + int generate_spec(ObLogValuesTableAccess &op, ObValuesTableAccessSpec &spec, const bool in_root_job); int generate_merge_with_das(ObLogMerge &op, ObTableMergeSpec &spec, const bool in_root_job); diff --git a/src/sql/engine/basic/ob_values_table_access_op.cpp b/src/sql/engine/basic/ob_values_table_access_op.cpp new file mode 100644 index 0000000000..b305f339b6 --- /dev/null +++ b/src/sql/engine/basic/ob_values_table_access_op.cpp @@ -0,0 +1,348 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_ENG + +#include "sql/engine/basic/ob_values_table_access_op.h" +#include "sql/engine/ob_exec_context.h" + +namespace oceanbase +{ +using namespace common; +namespace sql +{ +OB_SERIALIZE_MEMBER_INHERIT(ObValuesTableAccessSpec, + ObOpSpec, + access_type_, + column_exprs_, + value_exprs_, + start_param_idx_, + end_param_idx_, + obj_params_); + +ObValuesTableAccessOp::ObValuesTableAccessOp(ObExecContext &exec_ctx, + const ObOpSpec &spec, + ObOpInput *input) + : ObOperator(exec_ctx, spec, input), datum_caster_(), cm_(CM_NONE), + row_idx_(0), row_cnt_(0) {} + +int ObValuesTableAccessOp::inner_open() +{ + int ret = OB_SUCCESS; + row_idx_ = 0; + const int32_t result_flag = 0; + ObPhysicalPlanCtx *plan_ctx = GET_PHY_PLAN_CTX(ctx_); + if (OB_ISNULL(plan_ctx) || OB_ISNULL(ctx_.get_sql_ctx())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected NULL ptr", K(ret), KP(plan_ctx), KP(ctx_.get_sql_ctx())); + } else if (OB_UNLIKELY(MY_SPEC.column_exprs_.empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("should have one column at least", K(ret)); + } else if (OB_FAIL(datum_caster_.init(eval_ctx_.exec_ctx_))) { + LOG_WARN("fail to init datum_caster", K(ret)); + } else if (OB_FAIL(ObSQLUtils::get_default_cast_mode(false, 0, GET_MY_SESSION(ctx_), cm_))) { + LOG_WARN("fail to get_default_cast_mode", K(ret)); + } else { + cm_ = cm_ | CM_COLUMN_CONVERT; + if (OB_SUCC(ret)) { + if (ObValuesTableDef::FOLD_ACCESS_EXPR == MY_SPEC.access_type_) { + if (OB_UNLIKELY(MY_SPEC.start_param_idx_ < 0 || + MY_SPEC.start_param_idx_ > MY_SPEC.end_param_idx_ || + MY_SPEC.end_param_idx_ >= plan_ctx->get_param_store().count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected param", K(ret)); + } else { + const ObObj &first_column_node = plan_ctx->get_param_store().at(MY_SPEC.start_param_idx_); + if (OB_FAIL(first_column_node.get_real_param_count(row_cnt_))) { + LOG_WARN("failed to get row", K(ret)); + } + } + } else if (ObValuesTableDef::ACCESS_PARAM == MY_SPEC.access_type_) { + int64_t column_cnt = MY_SPEC.column_exprs_.count(); + row_cnt_ = (MY_SPEC.end_param_idx_ - MY_SPEC.start_param_idx_ + 1) / column_cnt; + } else if (ObValuesTableDef::ACCESS_OBJ == MY_SPEC.access_type_) { + int64_t column_cnt = MY_SPEC.column_exprs_.count(); + row_cnt_ = MY_SPEC.obj_params_.count() / column_cnt; + } else { + int64_t column_cnt = MY_SPEC.column_exprs_.count(); + row_cnt_ = MY_SPEC.value_exprs_.count() / column_cnt; + } + } + LOG_TRACE("values table access op info", K_(MY_SPEC.access_type), K_(row_cnt), + K_(MY_SPEC.column_exprs), K_(MY_SPEC.obj_params), K_(MY_SPEC.value_exprs)); + } + return ret; +} + +int ObValuesTableAccessOp::inner_rescan() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObOperator::inner_rescan())) { + LOG_WARN("failed to do inner rescan", K(ret)); + } else { + row_idx_ = 0; + } + return ret; +} + +int ObValuesTableAccessOp::switch_iterator() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObOperator::inner_switch_iterator())) { + LOG_WARN("failed to do inner rescan", K(ret)); + } else { + row_idx_ = 0; + } + return ret; +} + +int ObValuesTableAccessOp::inner_get_next_row() +{ + int ret = OB_SUCCESS; + if (OB_FAIL(try_check_status())) { + LOG_WARN("check physical plan status faild", K(ret)); + } else { + do { + clear_evaluated_flag(); + if (OB_FAIL(calc_next_row())) { + if(OB_ITER_END != ret) { + LOG_WARN("get next row from row store failed", K(ret)); + } + } else { + LOG_DEBUG("output row", "row", ROWEXPR2STR(eval_ctx_, MY_SPEC.output_)); + } + break; + } while (OB_SUCC(ret)); + } + return ret; +} + +void ObValuesTableAccessOp::update_src_meta(const ObObjMeta &src_obj_meta, + const ObAccuracy &src_obj_acc, + ObDatumMeta &src_meta) +{ + src_meta.type_ = src_obj_meta.get_type(); + src_meta.cs_type_ = src_obj_meta.get_collation_type(); + src_meta.scale_ = src_obj_acc.get_scale(); + src_meta.precision_ = src_obj_acc.get_precision(); +} + +int ObValuesTableAccessOp::get_real_src_obj_type(const int64_t row_idx, + ObExpr &src_expr, + ObDatumMeta &src_meta, + ObObjMeta &src_obj_meta) +{ + int ret = OB_SUCCESS; + if (T_QUESTIONMARK == src_expr.type_) { + int64_t param_idx = src_expr.extra_; + ObPhysicalPlanCtx *plan_ctx = GET_PHY_PLAN_CTX(ctx_); + const ObSqlArrayObj *array_obj = NULL; + if (OB_UNLIKELY(param_idx < 0 || param_idx >= plan_ctx->get_param_store().count())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid param idx", K(ret), K(param_idx)); + } else if (plan_ctx->get_param_store().at(param_idx).is_ext_sql_array()) { + // 如果是is_ext_sql_array的参数 + if (OB_ISNULL(array_obj = reinterpret_cast( + plan_ctx->get_param_store().at(param_idx).get_ext()))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected nullptr", K(ret), K(param_idx), K(plan_ctx->get_param_store())); + } else if (array_obj->count_ <= row_idx) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected row_idx", K(ret), K(array_obj->count_), K(row_idx), K(param_idx), + K(plan_ctx->get_param_store())); + } else { + src_obj_meta = array_obj->data_[row_idx].meta_; + const ObAccuracy &src_obj_acc = array_obj->data_[row_idx].get_accuracy(); + update_src_meta(src_obj_meta, src_obj_acc, src_meta); + } + } else if (src_expr.frame_idx_ < spec_.plan_->get_expr_frame_info().const_frame_.count() + + spec_.plan_->get_expr_frame_info().param_frame_.count()) { + src_obj_meta = plan_ctx->get_param_store().at(param_idx).meta_; + const ObAccuracy &src_obj_acc = plan_ctx->get_param_store().at(param_idx).get_accuracy(); + update_src_meta(src_obj_meta, src_obj_acc, src_meta); + } + } + return ret; +} + +int ObValuesTableAccessOp::calc_datum_from_expr(const int64_t col_idx, + const int64_t row_idx, + ObExpr *src_expr, + ObExpr *dst_expr) +{ + int ret = OB_SUCCESS; + ObDatumMeta src_meta = src_expr->datum_meta_; + ObObjMeta src_obj_meta = src_expr->obj_meta_; + ObDatum *datum = NULL; + bool need_adjust_decimal_int = false; + if (src_expr == dst_expr) { + /* actually should not happened */ + } else if (T_QUESTIONMARK == src_expr->type_ && + OB_FAIL(get_real_src_obj_type(row_idx, *src_expr, src_meta, src_obj_meta))) { + LOG_WARN("failed to get real obj type"); + } else { + need_adjust_decimal_int = src_meta.type_ == ObDecimalIntType && + dst_expr->datum_meta_.type_ == ObDecimalIntType && + ObDatumCast::need_scale_decimalint(src_meta.scale_, + src_meta.precision_, + dst_expr->datum_meta_.scale_, + dst_expr->datum_meta_.precision_); + } + if (OB_FAIL(ret)) { + } else if (src_meta.type_ == dst_expr->datum_meta_.type_ && + src_meta.cs_type_ == dst_expr->datum_meta_.cs_type_ && + !need_adjust_decimal_int) { + // when type and collation_type is same. src has the same type to dst; + if (OB_FAIL(src_expr->eval(eval_ctx_, datum))) { + LOG_WARN("failed to get src datum", K(ret)); + } else { + dst_expr->locate_datum_for_write(eval_ctx_) = *datum; + dst_expr->set_evaluated_projected(eval_ctx_); + } + } else { + if (OB_FAIL(ObCharset::check_valid_implicit_convert(src_meta.cs_type_, + dst_expr->datum_meta_.cs_type_))) { + LOG_WARN("failed to check valid implicit convert", K(ret)); + } else { + ObExpr real_src_expr = *src_expr; + real_src_expr.datum_meta_ = src_meta; + real_src_expr.obj_meta_ = src_obj_meta; + real_src_expr.obj_datum_map_ = ObDatum::get_obj_datum_map_type(src_meta.type_); + if (OB_FAIL(datum_caster_.to_type(dst_expr->datum_meta_, real_src_expr, cm_, datum))) { + LOG_WARN("fail to dynamic cast", K(ret), K(dst_expr->datum_meta_), K(real_src_expr), K(cm_)); + } else { + ObDatum &dst_datum = dst_expr->locate_datum_for_write(eval_ctx_); + if (ObObjDatumMapType::OBJ_DATUM_STRING == dst_expr->obj_datum_map_) { + ObExprStrResAlloc res_alloc(*dst_expr, eval_ctx_); + if (OB_FAIL(dst_datum.deep_copy(*datum, res_alloc))) { + LOG_WARN("fail to deep copy datum from cast res datum", K(ret), KP(datum)); + } + } else { + ObDataBuffer res_alloc(const_cast(dst_datum.ptr_), dst_expr->res_buf_len_); + if (OB_FAIL(dst_datum.deep_copy(*datum, res_alloc))) { + LOG_WARN("fail to deep copy datum from cast res datum", K(ret), KP(datum)); + } + } + dst_expr->set_evaluated_projected(eval_ctx_); + } + } + } + return ret; +} + +int ObValuesTableAccessOp::calc_datum_from_param(const ObObj &src_obj, ObExpr *dst_expr) +{ + int ret = OB_SUCCESS; + ObDatum &dst_datum = dst_expr->locate_datum_for_write(eval_ctx_); + const ObObjType &src_type = src_obj.get_type(); + const ObObjType &dst_type = dst_expr->obj_meta_.get_type(); + if (ob_is_decimal_int_tc(src_type) && ob_is_number_tc(dst_type)) { + ObObj dst_obj; + const ObDataTypeCastParams dtc_params = ObBasicSessionInfo::create_dtc_params(GET_MY_SESSION(ctx_)); + ObCastCtx cast_ctx(&eval_ctx_.exec_ctx_.get_allocator(), &dtc_params, cm_, dst_expr->obj_meta_.get_collation_type()); + cast_ctx.exec_ctx_ = &eval_ctx_.exec_ctx_; + if (OB_FAIL(ObObjCaster::to_type(dst_type, dst_expr->obj_meta_.get_collation_type(), cast_ctx, + src_obj, dst_obj))) { + LOG_WARN("failed to cast obj", K(ret), K(src_type), K(dst_type), K(src_obj), K(dst_obj)); + } else if (OB_FAIL(dst_datum.from_obj(dst_obj))) { + LOG_WARN("failed to from obj", K(ret)); + } else { + dst_expr->set_evaluated_projected(eval_ctx_); + } + } else if (OB_FAIL(dst_datum.from_obj(src_obj))) { + LOG_WARN("failed to from obj", K(ret)); + } else { + dst_expr->set_evaluated_projected(eval_ctx_); + } + return ret; +} + +OB_INLINE int ObValuesTableAccessOp::calc_next_row() +{ + int ret = OB_SUCCESS; + int64_t col_num = MY_SPEC.column_exprs_.count(); + int64_t col_idx = 0; + ObPhysicalPlanCtx *plan_ctx = GET_PHY_PLAN_CTX(ctx_); + if (row_idx_ == row_cnt_) { + ret = OB_ITER_END; + } else if (ObValuesTableDef::FOLD_ACCESS_EXPR == MY_SPEC.access_type_ && + OB_FAIL(plan_ctx->replace_batch_param_datum(row_idx_, + MY_SPEC.start_param_idx_, + MY_SPEC.end_param_idx_ - MY_SPEC.start_param_idx_ + 1))) { + LOG_WARN("replace batch param datum failed", K(ret), K(row_idx_)); + } else { + while (OB_SUCC(ret) && col_idx < col_num) { + ObExpr *col_expr = MY_SPEC.column_exprs_.at(col_idx); + switch (MY_SPEC.access_type_) + { + case ObValuesTableDef::ACCESS_EXPR : { + int64_t idx = row_idx_ * col_num + col_idx; + ObExpr *src_expr = MY_SPEC.value_exprs_.at(idx); + if (OB_FAIL(calc_datum_from_expr(col_idx, row_idx_, src_expr, col_expr))) { + LOG_WARN("failed to calc datum from expr", K(ret)); + } + break; + } + case ObValuesTableDef::FOLD_ACCESS_EXPR : { + ObExpr *src_expr = MY_SPEC.value_exprs_.at(col_idx); + if (OB_FAIL(calc_datum_from_expr(col_idx, row_idx_, src_expr, col_expr))) { + LOG_WARN("failed to calc datum from expr", K(ret)); + } + break; + } + case ObValuesTableDef::ACCESS_PARAM: { + int64_t param_idx = MY_SPEC.start_param_idx_ + col_idx + row_idx_ * col_num; + const ObObjParam ¶m = plan_ctx->get_param_store().at(param_idx); + if (OB_FAIL(calc_datum_from_param(param, col_expr))) { + LOG_WARN("failed to calc datum from expr", K(ret)); + } + break; + } + case ObValuesTableDef::ACCESS_OBJ: { + int64_t idx = row_idx_ * col_num + col_idx; + const ObObj ¶m = MY_SPEC.obj_params_.at(idx); + if (OB_FAIL(calc_datum_from_param(param, col_expr))) { + LOG_WARN("failed to calc datum from expr", K(ret)); + } + break; + } + }; + col_idx++; + } // while end + + for (int64_t i = 0; OB_SUCC(ret) && i < MY_SPEC.output_.count(); i++) { + ObExpr *output = MY_SPEC.output_.at(i); + if (i < col_num && output == MY_SPEC.column_exprs_.at(i)) { + /* need do nothing */ + } else { + ObDatum *dst_datum = NULL; + MY_SPEC.output_.at(i)->eval(eval_ctx_, dst_datum); + } + } + if (OB_SUCC(ret)) { + ++row_idx_; + } + } + return ret; +} + +int ObValuesTableAccessOp::inner_close() +{ + int ret = OB_SUCCESS; + row_idx_ = 0; + if (OB_FAIL(datum_caster_.destroy())) { + LOG_WARN("fail to destroy datum_caster", K(ret)); + } + return ret; +} +} // end namespace sql +} // end namespace oceanbase diff --git a/src/sql/engine/basic/ob_values_table_access_op.h b/src/sql/engine/basic/ob_values_table_access_op.h new file mode 100644 index 0000000000..28061a9498 --- /dev/null +++ b/src/sql/engine/basic/ob_values_table_access_op.h @@ -0,0 +1,79 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SQL_ENGINE_BASIC_OB_VALUES_TABLE_ACCESS_OP_ +#define OCEANBASE_SQL_ENGINE_BASIC_OB_VALUES_TABLE_ACCESS_OP_ + +#include "sql/engine/ob_operator.h" +#include "sql/engine/expr/ob_datum_cast.h" + +namespace oceanbase +{ +namespace sql +{ +class ObPhyOpSeriCtx; + +class ObValuesTableAccessSpec : public ObOpSpec +{ + OB_UNIS_VERSION_V(1); +public: + ObValuesTableAccessSpec(common::ObIAllocator &alloc, const ObPhyOperatorType type) + : ObOpSpec(alloc, type), access_type_(ObValuesTableDef::ACCESS_EXPR), column_exprs_(alloc), + value_exprs_(alloc), start_param_idx_(-1), end_param_idx_(-1), obj_params_(alloc) {} + INHERIT_TO_STRING_KV("op_spec", ObOpSpec, K_(access_type)); + ObValuesTableDef::TableAccessType access_type_; + common::ObFixedArray column_exprs_; + common::ObFixedArray value_exprs_; + int64_t start_param_idx_; + int64_t end_param_idx_; + common::ObFixedArray obj_params_; +private: + DISALLOW_COPY_AND_ASSIGN(ObValuesTableAccessSpec); +}; + +class ObValuesTableAccessOp : public ObOperator +{ +public: + ObValuesTableAccessOp(ObExecContext &exec_ctx, const ObOpSpec &spec, ObOpInput *input); + virtual int inner_open() override; + virtual int inner_rescan() override; + virtual int switch_iterator() override; + virtual int inner_get_next_row() override; + virtual int inner_close() override; + virtual void destroy() override { ObOperator::destroy(); } +private: + int calc_next_row(); + int calc_datum_from_expr(const int64_t col_idx, + const int64_t row_idx, + ObExpr *src_expr, + ObExpr *dst_expr); + int calc_datum_from_param(const ObObj &src_obj, ObExpr *dst_expr); + void update_src_meta(const ObObjMeta &src_obj_meta, + const ObAccuracy &src_obj_acc, + ObDatumMeta &src_meta); + int get_real_src_obj_type(const int64_t row_idx, + ObExpr &src_expr, + ObDatumMeta &src_meta, + ObObjMeta &src_obj_meta); +private: + DISALLOW_COPY_AND_ASSIGN(ObValuesTableAccessOp); + +private: + ObDatumCaster datum_caster_; + common::ObCastMode cm_; + int64_t row_idx_; + int64_t row_cnt_; +}; + +} // end namespace sql +} // end namespace oceanbase +#endif diff --git a/src/sql/engine/ob_operator_factory.cpp b/src/sql/engine/ob_operator_factory.cpp index 1df5ca41c2..dcf6d856ab 100644 --- a/src/sql/engine/ob_operator_factory.cpp +++ b/src/sql/engine/ob_operator_factory.cpp @@ -173,6 +173,8 @@ #include "sql/engine/basic/ob_temp_table_access_vec_op.h" #include "sql/engine/basic/ob_temp_table_transformation_vec_op.h" #include "sql/engine/sort/ob_sort_vec_op.h" +#include "sql/optimizer/ob_log_values_table_access.h" +#include "sql/engine/basic/ob_values_table_access_op.h" namespace oceanbase { diff --git a/src/sql/engine/ob_operator_reg.h b/src/sql/engine/ob_operator_reg.h index 368b55985a..7666cf80f4 100644 --- a/src/sql/engine/ob_operator_reg.h +++ b/src/sql/engine/ob_operator_reg.h @@ -382,6 +382,11 @@ class ObExprValuesSpec; class ObExprValuesOp; REGISTER_OPERATOR(ObLogExprValues, PHY_EXPR_VALUES, ObExprValuesSpec, ObExprValuesOp, NOINPUT); +class ObLogValuesTableAccess; +class ObValuesTableAccessSpec; +class ObValuesTableAccessOp; +REGISTER_OPERATOR(ObLogValuesTableAccess, PHY_VALUES_TABLE_ACCESS, ObValuesTableAccessSpec, ObValuesTableAccessOp, NOINPUT); + class ObLogDelete; class ObTableDeleteSpec; class ObTableDeleteOp; diff --git a/src/sql/engine/ob_physical_plan_ctx.cpp b/src/sql/engine/ob_physical_plan_ctx.cpp index fc8f1c9f34..e3c8b8b228 100644 --- a/src/sql/engine/ob_physical_plan_ctx.cpp +++ b/src/sql/engine/ob_physical_plan_ctx.cpp @@ -917,7 +917,7 @@ OB_DEF_DESERIALIZE(ObPhysicalPlanCtx) for (int64_t i = 0; OB_SUCC(ret) && i < real_param_cnt; ++i) { OB_UNIS_DECODE(param_idx); OB_UNIS_DECODE(param_obj); - ObObjParam tmp = param_obj; + ObObjParam tmp = param_obj; if (OB_UNLIKELY(param_idx < 0) || OB_UNLIKELY(param_idx >= param_cnt)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid param idx", K(param_idx), K(param_cnt)); @@ -934,7 +934,7 @@ OB_DEF_DESERIALIZE(ObPhysicalPlanCtx) //所以需要对param store的所有元素都执行一次深拷贝 for (int64_t i = 0; OB_SUCC(ret) && i < param_store_.count(); ++i) { const ObObjParam &objpara = param_store_.at(i); - ObObjParam tmp = objpara; + ObObjParam tmp = objpara; if (OB_FAIL(deep_copy_obj(allocator_, objpara, tmp))) { LOG_WARN("deep copy obj failed", K(ret)); } else { diff --git a/src/sql/engine/px/ob_granule_iterator_op.cpp b/src/sql/engine/px/ob_granule_iterator_op.cpp index 391a247589..df642263cd 100644 --- a/src/sql/engine/px/ob_granule_iterator_op.cpp +++ b/src/sql/engine/px/ob_granule_iterator_op.cpp @@ -991,25 +991,48 @@ int ObGranuleIteratorOp::set_dml_op(const ObTableModifySpec *dml_op) return OB_SUCCESS; } +// NOTE: this function is only used for the GI which only control one scan operator. +// Think about the following case, the GI attempt to control the right tsc op rather than +// the values table op(or maybe json table op), thus we need to traverse the OP tree to find the +// real consumer node(tsc op). +// PX GI +// | +// GBY +// | +// SORT +// | +// HJ +// / \ +// values tsc +// table +// int ObGranuleIteratorOp::get_gi_task_consumer_node(ObOperator *cur, - ObOperator *&child) const + ObOperator *&consumer) const { int ret = OB_SUCCESS; - ObOperator *first_child = NULL; - if (0 >= cur->get_child_cnt()) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("can't get the consumer node", K(ret), K(cur->get_child_cnt())); - } else if (OB_ISNULL(first_child = cur->get_child(0))) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("child is null", K(ret)); - } else if (PHY_TABLE_SCAN == first_child->get_spec().type_ || - PHY_BLOCK_SAMPLE_SCAN == first_child->get_spec().type_ || - PHY_ROW_SAMPLE_SCAN == first_child->get_spec().type_) { - child = first_child; - } else if (OB_FAIL(get_gi_task_consumer_node(first_child, child))) { - LOG_WARN("failed to get gi task consumer node", K(ret)); + int64_t child_cnt = cur->get_child_cnt(); + if (0 == child_cnt) { + if (PHY_TABLE_SCAN == cur->get_spec().type_ + || PHY_BLOCK_SAMPLE_SCAN == cur->get_spec().type_ + || PHY_ROW_SAMPLE_SCAN == cur->get_spec().type_) { + consumer = cur; + LOG_TRACE("find the gi_task consumer node", K(cur->get_spec().id_), + K(cur->get_spec().type_)); + } + } else { + ObOperator *child = nullptr; + for (int64_t i = 0; i < child_cnt && OB_SUCC(ret) && OB_ISNULL(consumer); ++i) { + if (OB_ISNULL(child = cur->get_child(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("child is null", K(ret)); + } else if (OB_FAIL(get_gi_task_consumer_node(child, consumer))) { + LOG_WARN("failed to get gi task consumer node", K(ret)); + } + } } - if (OB_SUCC(ret) && OB_ISNULL(child)) { + + // cur == this means only check the output stack + if (OB_SUCC(ret) && OB_ISNULL(consumer) && cur == this) { ret = OB_ERR_UNEXPECTED; LOG_WARN("can't find the tsc phy op", K(ret)); } diff --git a/src/sql/engine/px/ob_granule_iterator_op.h b/src/sql/engine/px/ob_granule_iterator_op.h index f1c03784d1..cd94491845 100644 --- a/src/sql/engine/px/ob_granule_iterator_op.h +++ b/src/sql/engine/px/ob_granule_iterator_op.h @@ -213,7 +213,7 @@ private: bool is_not_init() { return state_ == GI_UNINITIALIZED; } // 获得消费GI task的node: // 目前仅仅支持TSC或者Join - int get_gi_task_consumer_node(ObOperator *cur, ObOperator *&child) const; + int get_gi_task_consumer_node(ObOperator *cur, ObOperator *&consumer) const; // ---for nlj pkey int try_pruning_repart_partition( const ObGITaskSet &taskset, diff --git a/src/sql/ob_sql.cpp b/src/sql/ob_sql.cpp index 8aa77df7d2..4e14a4b3fb 100644 --- a/src/sql/ob_sql.cpp +++ b/src/sql/ob_sql.cpp @@ -3385,6 +3385,23 @@ int ObSql::generate_plan(ParseResult &parse_result, OPT_TRACE_TITLE("CURRENT SQL TEXT"); OPT_TRACE("sql_id =", ObString(strlen(sql_ctx.sql_id_), sql_ctx.sql_id_)); OPT_TRACE(ObString(parse_result.input_sql_len_, parse_result.input_sql_)); + + // hint seed injected random status rand + const ObQueryHint &query_hint = stmt->get_query_ctx()->get_query_hint(); + if (query_hint.has_outline_data() || session_info->is_inner()){ + // if there is outline data no error inject + } else { + int64_t tmp_ret = (OB_E(EventTable::EN_GENERATE_RANDOM_PLAN) OB_SUCCESS); + if (OB_SUCCESS == tmp_ret) { + // do nothing + } else { + time_t seed = OB_ERROR == tmp_ret ? time(NULL) : std::abs(tmp_ret); + stmt->get_query_ctx()->set_injected_random_status(true); + stmt->get_query_ctx()->set_random_plan_seed(seed); + LOG_TRACE("The random seed for plan gen is ", K(seed)); + } + } + if (OB_FAIL(ret)) { } else if (OB_FAIL(transform_stmt(&stmt->get_query_ctx()->sql_schema_guard_, opt_stat_mgr_, diff --git a/src/sql/ob_sql_context.h b/src/sql/ob_sql_context.h index 8307e5bc9e..defe3ee7ae 100644 --- a/src/sql/ob_sql_context.h +++ b/src/sql/ob_sql_context.h @@ -17,6 +17,7 @@ #include "lib/net/ob_addr.h" #include "lib/hash/ob_placement_hashset.h" #include "lib/container/ob_2d_array.h" +#include "lib/random/ob_random.h" #include "sql/optimizer/ob_table_partition_info.h" #include "sql/monitor/ob_exec_stat.h" #include "lib/hash_func/murmur_hash.h" @@ -693,7 +694,8 @@ public: root_stmt_(NULL), optimizer_features_enable_version_(0), udf_flag_(0), - has_dblink_(false) + has_dblink_(false), + injected_random_status_(false) { } TO_STRING_KV(N_PARAM_NUM, question_marks_count_, @@ -739,7 +741,6 @@ public: root_stmt_ = NULL; udf_flag_ = 0; optimizer_features_enable_version_ = 0; - has_dblink_ = false; } int64_t get_new_stmt_id() { return stmt_count_++; } @@ -770,6 +771,11 @@ public: void set_timezone_info(const common::ObTimeZoneInfo *tz_info) { tz_info_ = tz_info; } const common::ObTimeZoneInfo *get_timezone_info() const { return tz_info_; } int add_local_session_vars(ObIAllocator *alloc, const ObLocalSessionVar &local_session_var, int64_t &idx); + bool get_injected_random_status() const { return injected_random_status_; } + void set_injected_random_status(bool injected_random_status) { injected_random_status_ = injected_random_status; } + void set_random_plan_seed(uint64_t seed) {rand_gen_.seed(seed);} + + public: static const int64_t CALCULABLE_EXPR_NUM = 1; @@ -836,6 +842,8 @@ public: }; }; bool has_dblink_; + bool injected_random_status_; + ObRandom rand_gen_; }; } /* ns sql*/ } /* ns oceanbase */ diff --git a/src/sql/optimizer/ob_explain_log_plan.cpp b/src/sql/optimizer/ob_explain_log_plan.cpp index b1df728d1b..8a7ea40b39 100644 --- a/src/sql/optimizer/ob_explain_log_plan.cpp +++ b/src/sql/optimizer/ob_explain_log_plan.cpp @@ -132,7 +132,10 @@ int ObExplainLogPlan::check_explain_generate_plan_with_outline(ObLogPlan *real_p ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret), K(real_plan), K(explain_stmt), K(session_info), K(exec_ctx), K(sql_ctx)); } else if (session_info->is_inner() || sql_ctx->is_prepare_protocol_) { - /* do not check explain for inner sql (include query in PL) and prepare stmt */ + /* do not check explain for inner sql (include query in PL) */ + } else if (OB_UNLIKELY(get_optimizer_context().get_query_ctx() != NULL + && get_optimizer_context().get_query_ctx()->get_injected_random_status())) { + /* do nothing */ } else if (sql_ctx->multi_stmt_item_.is_part_of_multi_stmt() && sql_ctx->multi_stmt_item_.get_seq_num() > 0) { /* generate plan call by ObMPQuery::process_with_tmp_context use tmp context, do not check */ diff --git a/src/sql/optimizer/ob_join_order.cpp b/src/sql/optimizer/ob_join_order.cpp index fbeb27514d..625935e624 100644 --- a/src/sql/optimizer/ob_join_order.cpp +++ b/src/sql/optimizer/ob_join_order.cpp @@ -607,12 +607,14 @@ int ObJoinOrder::compute_base_table_path_ordering(AccessPath *path) int ret = OB_SUCCESS; bool is_left_prefix = false; bool is_right_prefix = false; + const ObDMLStmt *stmt = NULL; ObSEArray range_exprs; ObSEArray range_orders; path->is_local_order_ = false; path->is_range_order_ = false; if (OB_ISNULL(path) || OB_ISNULL(get_plan()) || OB_ISNULL(get_plan()->get_stmt()) || - OB_ISNULL(path->strong_sharding_) || OB_ISNULL(path->table_partition_info_)) { + OB_ISNULL(path->strong_sharding_) || OB_ISNULL(path->table_partition_info_) || + OB_ISNULL(stmt = get_plan()->get_stmt()) || OB_ISNULL(stmt->get_query_ctx())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(path), K(ret)); } else if (path->use_das_ && @@ -622,6 +624,12 @@ int ObJoinOrder::compute_base_table_path_ordering(AccessPath *path) path->is_local_order_ = false; } else if (get_plan()->get_optimizer_context().is_online_ddl()) { path->is_local_order_ = true; + } else if (((stmt->get_query_ctx()->optimizer_features_enable_version_ >= COMPAT_VERSION_4_2_3 && + stmt->get_query_ctx()->optimizer_features_enable_version_ < COMPAT_VERSION_4_3_0) || + stmt->get_query_ctx()->optimizer_features_enable_version_ >= COMPAT_VERSION_4_3_2) && + path->table_partition_info_->get_phy_tbl_location_info().get_phy_part_loc_info_list().count() == 1 && + !is_virtual_table(path->ref_table_id_)) { + path->is_range_order_ = true; } else if (OB_FAIL(path->table_partition_info_->get_not_insert_dml_part_sort_expr(*get_plan()->get_stmt(), &range_exprs))) { LOG_WARN("fail to get_not_insert_dml_part_sort_expr", K(ret)); @@ -1054,7 +1062,8 @@ int ObJoinOrder::get_query_range_info(const uint64_t table_id, : helper.filters_, range_info.get_expr_constraints(), table_id, - query_range))) { + query_range, + index_id))) { LOG_WARN("failed to extract query range", K(ret), K(index_id)); } else if (is_geo_index && OB_FAIL(extract_geo_preliminary_query_range(range_columns, is_oracle_inner_index_table @@ -3743,12 +3752,14 @@ int ObJoinOrder::extract_preliminary_query_range(const ObIArray &ran const ObIArray &predicates, ObIArray &expr_constraints, int64_t table_id, - ObQueryRange *&query_range) + ObQueryRange *&query_range, + int64_t index_id) { int ret = OB_SUCCESS; ObOptimizerContext *opt_ctx = NULL; const ParamStore *params = NULL; ObSQLSessionInfo *session_info = NULL; + int64_t index_prefix = -1; if (OB_ISNULL(get_plan()) || OB_ISNULL(opt_ctx = &get_plan()->get_optimizer_context()) || OB_ISNULL(allocator_) || @@ -3775,6 +3786,10 @@ int ObJoinOrder::extract_preliminary_query_range(const ObIArray &ran } else if (!enable_better_inlist && OB_FAIL(range_predicates.assign(predicates))) { LOG_WARN("failed to assign exprs", K(ret)); + } else if (OB_FAIL(get_plan()->get_log_plan_hint().get_index_prefix(table_id, + index_id, + index_prefix))) { + LOG_WARN("failed to get index prefix", K(table_id), K(index_id), K(index_prefix), K(ret)); } else { tmp_qr = new(tmp_ptr)ObQueryRange(*allocator_); const ObDataTypeCastParams dtc_params = @@ -3788,7 +3803,8 @@ int ObJoinOrder::extract_preliminary_query_range(const ObIArray &ran dtc_params, opt_ctx->get_exec_ctx(), &expr_constraints, params, false, true, - is_in_range_optimization_enabled))) { + is_in_range_optimization_enabled, + index_prefix))) { LOG_WARN("failed to preliminary extract query range", K(ret)); } } @@ -4928,9 +4944,16 @@ int ObJoinOrder::add_path(Path* path) ret = OB_INVALID_ARGUMENT; LOG_WARN("get unexpected null", K(path), K(ret)); } else { - bool should_add = true; DominateRelation plan_rel = DominateRelation::OBJ_UNCOMPARABLE; OPT_TRACE_TITLE("new candidate path:", path); + bool should_add = true; + + if (OB_UNLIKELY(get_plan()->get_optimizer_context().generate_random_plan())) { + ObQueryCtx* query_ctx = get_plan()->get_optimizer_context().get_query_ctx(); + bool random_flag = !OB_ISNULL(query_ctx) && (query_ctx->rand_gen_.get(0, 1) == 1); + should_add = interesting_paths_.empty() || random_flag; + } + /** * fake cte会生成两条path,一条local、一条match all * match fake cte路径只用来生成remote的计划 @@ -6139,10 +6162,9 @@ int TempTablePath::compute_sharding_info() int TempTablePath::compute_path_ordering() { int ret = OB_SUCCESS; - if (!ordering_.empty()) { - is_local_order_ = true; - is_range_order_ = false; - } + ordering_.reuse(); + is_local_order_ = false; + is_range_order_ = false; return ret; } @@ -6186,6 +6208,7 @@ int JoinPath::assign(const JoinPath &other, common::ObIAllocator *allocator) can_use_batch_nlj_ = other.can_use_batch_nlj_; is_naaj_ = other.is_naaj_; is_sna_ = other.is_sna_; + inherit_sharding_index_ = other.inherit_sharding_index_; if (OB_FAIL(Path::assign(other, allocator))) { LOG_WARN("failed to deep copy path", K(ret)); @@ -7615,6 +7638,7 @@ void JoinPath::reuse() contain_normal_nl_ = false; is_naaj_ = false; is_sna_ = false; + inherit_sharding_index_ = -1; } int JoinPath::compute_pipeline_info() @@ -15443,6 +15467,7 @@ int ValuesTablePath::assign(const ValuesTablePath &other, common::ObIAllocator * LOG_WARN("failed to assgin", K(ret)); } else { table_id_ = other.table_id_; + table_def_ = other.table_def_; } return ret; } @@ -15450,31 +15475,65 @@ int ValuesTablePath::assign(const ValuesTablePath &other, common::ObIAllocator * int ValuesTablePath::estimate_cost() { int ret = OB_SUCCESS; - if (OB_ISNULL(parent_) || OB_ISNULL(parent_->get_plan())) { + if (OB_ISNULL(parent_) || OB_ISNULL(parent_->get_plan()) || OB_ISNULL(table_def_)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("get unexpected null", K(parent_), K(ret)); } else { ObOptimizerContext &opt_ctx = parent_->get_plan()->get_optimizer_context(); - cost_ = ObOptEstCost::cost_get_rows(get_path_output_rows(), opt_ctx); + int64_t row_count = table_def_->row_cnt_; + cost_ = ObOptEstCost::cost_values_table(row_count, filter_, opt_ctx); op_cost_ = cost_; } return ret; } +int ValuesTablePath::estimate_row_count() +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(parent_) || OB_ISNULL(parent_->get_plan()) || OB_ISNULL(table_def_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("get unexpected null", K(parent_), K(ret)); + } else { + ObLogPlan *plan = parent_->get_plan(); + double selectivity = 1.0; + int64_t row_count = table_def_->row_cnt_; + ObOptimizerContext &opt_ctx = plan->get_optimizer_context(); + if (OB_FAIL(ObOptSelectivity::calculate_selectivity(plan->get_basic_table_metas(), + plan->get_selectivity_ctx(), + filter_, + selectivity, + plan->get_predicate_selectivities()))) { + LOG_WARN("failed to calc filter selectivities", K(ret)); + } else { + parent_->set_output_rows(row_count * selectivity); + if (OB_FAIL(ObOptSelectivity::update_table_meta_info(plan->get_basic_table_metas(), + plan->get_update_table_metas(), + plan->get_selectivity_ctx(), + table_id_, + parent_->get_output_rows(), + parent_->get_restrict_infos(), + plan->get_predicate_selectivities()))) { + LOG_WARN("failed to update table meta info", K(ret)); + } + } + } + return ret; +} + int ObJoinOrder::generate_values_table_paths() { int ret = OB_SUCCESS; ValuesTablePath *values_path = NULL; const ObDMLStmt *stmt = NULL; TableItem *table_item = NULL; + ObValuesTableDef *values_table = NULL; if (OB_ISNULL(get_plan()) || OB_ISNULL(stmt = get_plan()->get_stmt()) || OB_ISNULL(allocator_)) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("get unexpected null", K(get_plan()), K(stmt), K(allocator_), K(ret)); + LOG_WARN("get unexpected null", KP(get_plan()), KP(stmt), KP(allocator_), K(ret)); } else if (OB_ISNULL(table_item = stmt->get_table_item_by_id(table_id_)) || - OB_UNLIKELY(!table_item->is_values_table() || - stmt->get_column_size(table_id_) == 0 || - table_item->table_values_.empty() || - table_item->table_values_.count() % stmt->get_column_size(table_id_) != 0)) { + OB_UNLIKELY(!table_item->is_values_table()) || + OB_ISNULL(values_table = table_item->values_table_def_) || + OB_UNLIKELY(values_table->column_cnt_ <= 0)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(table_id_), KPC(stmt), K(ret)); } else if (OB_ISNULL(values_path = reinterpret_cast( @@ -15485,33 +15544,40 @@ int ObJoinOrder::generate_values_table_paths() values_path = new(values_path) ValuesTablePath(); values_path->table_id_ = table_id_; values_path->parent_ = this; + values_path->table_def_ = values_table; ObSEArray nl_params; - output_rows_ = table_item->table_values_.count() / stmt->get_column_size(table_id_); values_path->strong_sharding_ = get_plan()->get_optimizer_context().get_match_all_sharding(); if (OB_FAIL(values_path->set_parallel_and_server_info_for_match_all())) { LOG_WARN("failed set parallel and server info for match all", K(ret)); } else if (OB_FAIL(append(values_path->filter_, get_restrict_infos()))) { LOG_WARN("failed to append filter", K(ret)); + } else if (OB_FAIL(get_plan()->get_basic_table_metas().add_values_table_meta_info(stmt, + table_id_, get_plan()->get_selectivity_ctx(), values_table))) { + LOG_WARN("failed to add values table meta info", K(ret)); } else if (OB_FAIL(ObOptEstCost::estimate_width_for_table(get_plan()->get_basic_table_metas(), get_plan()->get_selectivity_ctx(), stmt->get_column_items(), table_id_, output_row_size_))) { LOG_WARN("estimate width of row failed", K(table_id_), K(ret)); - } else if (OB_FAIL(param_values_table_expr(table_item->table_values_, + } else if (OB_FAIL(values_path->estimate_row_count())) { + LOG_WARN("failed to estimate row count", K(ret)); + } else if (OB_FAIL(values_path->estimate_cost())) { + LOG_WARN("failed to estimate cost", K(ret)); + } else if (OB_FAIL(param_values_table_expr(values_table->access_exprs_, nl_params, values_path->subquery_exprs_))) { LOG_WARN("failed to extract param for values table expr", K(ret)); } else if (OB_FAIL(values_path->nl_params_.assign(nl_params))) { LOG_WARN("failed to assign nl params", K(ret)); - } else if (OB_FAIL(values_path->estimate_cost())) { - LOG_WARN("failed to estimate cost", K(ret)); } else if (OB_FAIL(values_path->compute_pipeline_info())) { LOG_WARN("failed to compute pipelined path", K(ret)); } else if (OB_FAIL(add_path(values_path))) { LOG_WARN("failed to add path", K(ret)); } else { /*do nothing*/ } } + LOG_TRACE("after allocate values path", K(output_row_size_), K(output_rows_), + K(values_path->filter_), K(values_path->subquery_exprs_)); return ret; } diff --git a/src/sql/optimizer/ob_join_order.h b/src/sql/optimizer/ob_join_order.h index 7a3bc1505e..ccc3889594 100644 --- a/src/sql/optimizer/ob_join_order.h +++ b/src/sql/optimizer/ob_join_order.h @@ -1095,10 +1095,12 @@ struct EstimateCostInfo { public: ValuesTablePath() : Path(NULL), - table_id_(OB_INVALID_ID) {} + table_id_(OB_INVALID_ID), + table_def_(NULL) {} virtual ~ValuesTablePath() { } int assign(const ValuesTablePath &other, common::ObIAllocator *allocator); virtual int estimate_cost() override; + virtual int estimate_row_count(); virtual int get_name_internal(char *buf, const int64_t buf_len, int64_t &pos) const { int ret = OB_SUCCESS; @@ -1109,6 +1111,7 @@ struct EstimateCostInfo { } public: uint64_t table_id_; + ObValuesTableDef *table_def_; private: DISALLOW_COPY_AND_ASSIGN(ValuesTablePath); }; @@ -1555,7 +1558,8 @@ struct NullAwareAntiJoinInfo { const common::ObIArray &predicates, ObIArray &expr_constraints, int64_t table_id, - ObQueryRange* &range); + ObQueryRange* &range, + int64_t index_id); int check_enable_better_inlist(int64_t table_id, bool &enable); diff --git a/src/sql/optimizer/ob_log_del_upd.cpp b/src/sql/optimizer/ob_log_del_upd.cpp index 06fff796e1..9cf6fd07bf 100644 --- a/src/sql/optimizer/ob_log_del_upd.cpp +++ b/src/sql/optimizer/ob_log_del_upd.cpp @@ -1669,3 +1669,11 @@ int ObLogDelUpd::is_my_fixed_expr(const ObRawExpr *expr, bool &is_fixed) } return ret; } + +int ObLogDelUpd::check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index) +{ + int ret = OB_SUCCESS; + used = false; + inherit_child_ordering_index = -1; + return ret; +} \ No newline at end of file diff --git a/src/sql/optimizer/ob_log_del_upd.h b/src/sql/optimizer/ob_log_del_upd.h index fcee73fb86..fa7fb34628 100644 --- a/src/sql/optimizer/ob_log_del_upd.h +++ b/src/sql/optimizer/ob_log_del_upd.h @@ -336,6 +336,7 @@ public: ObRawExprReplacer &replacer, const ObIArray &index_dml_infos); virtual int is_my_fixed_expr(const ObRawExpr *expr, bool &is_fixed) override; + virtual int check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index)override; protected: virtual int generate_rowid_expr_for_trigger() = 0; virtual int generate_part_id_expr_for_foreign_key(ObIArray &all_exprs) = 0; diff --git a/src/sql/optimizer/ob_log_distinct.cpp b/src/sql/optimizer/ob_log_distinct.cpp index c3b072e5ea..e9680a929e 100644 --- a/src/sql/optimizer/ob_log_distinct.cpp +++ b/src/sql/optimizer/ob_log_distinct.cpp @@ -383,5 +383,17 @@ int ObLogDistinct::get_card_without_filter(double &card) return ret; } +int ObLogDistinct::check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index) +{ + int ret = OB_SUCCESS; + used = true; + inherit_child_ordering_index = first_child; + if (HASH_AGGREGATE == get_algo()) { + inherit_child_ordering_index = -1; + used = false; + } + return ret; +} + } } diff --git a/src/sql/optimizer/ob_log_distinct.h b/src/sql/optimizer/ob_log_distinct.h index 71030fdfb7..28ddd3f8ee 100644 --- a/src/sql/optimizer/ob_log_distinct.h +++ b/src/sql/optimizer/ob_log_distinct.h @@ -75,6 +75,7 @@ public: inline bool is_partition_ig() const { return is_partition_gi_; } inline void set_is_partition_gi(bool v) { is_partition_gi_ = v; } virtual int get_card_without_filter(double &card) override; + virtual int check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index)override; private: common::ObSEArray distinct_exprs_; diff --git a/src/sql/optimizer/ob_log_exchange.cpp b/src/sql/optimizer/ob_log_exchange.cpp index 536827b3dc..72473e196b 100644 --- a/src/sql/optimizer/ob_log_exchange.cpp +++ b/src/sql/optimizer/ob_log_exchange.cpp @@ -468,8 +468,9 @@ int ObLogExchange::do_re_est_cost(EstimateCostInfo ¶m, double &card, double double child_cost = child->get_cost(); const int64_t parallel = param.need_parallel_; param.need_parallel_ = ObGlobalHint::UNSET_PARALLEL; - if (is_block_op()) { - param.need_row_count_ = -1; //reset need row count + //forbit limit re estimate cost + if (is_producer() || is_block_op()) { + param.need_row_count_ = -1; } if (OB_FAIL(SMART_CALL(child->re_est_cost(param, child_card, child_cost)))) { LOG_WARN("failed to re est exchange cost", K(ret)); @@ -1096,6 +1097,17 @@ int ObLogExchange::close_px_resource_analyze(CLOSE_PX_RESOURCE_ANALYZE_DECLARE_A } LOG_TRACE("[PxResAnaly] px coord close_px_resource_analyze", K(get_op_id()), KPC(px_info_), K(append_map), K(cur_parallel_thread_count), K(cur_parallel_group_count)); + } + return ret; +} + +int ObLogExchange::check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index) +{ + int ret = OB_SUCCESS; + used = true; + inherit_child_ordering_index = first_child; + if (is_producer() || !is_merge_sort()) { + used = false; } return ret; } \ No newline at end of file diff --git a/src/sql/optimizer/ob_log_exchange.h b/src/sql/optimizer/ob_log_exchange.h index 09b7d98065..c0b44f77f6 100644 --- a/src/sql/optimizer/ob_log_exchange.h +++ b/src/sql/optimizer/ob_log_exchange.h @@ -211,6 +211,7 @@ private: int check_expr_is_need(const ObRawExpr *expr, const common::ObIArray &drop_expr_idxs, bool &is_need); + virtual int check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index)override; private: virtual int inner_replace_op_exprs(ObRawExprReplacer &replacer) override; diff --git a/src/sql/optimizer/ob_log_expr_values.cpp b/src/sql/optimizer/ob_log_expr_values.cpp index 42a56499db..2c63d32d81 100644 --- a/src/sql/optimizer/ob_log_expr_values.cpp +++ b/src/sql/optimizer/ob_log_expr_values.cpp @@ -212,16 +212,24 @@ int ObLogExprValues::est_cost() int ObLogExprValues::do_re_est_cost(EstimateCostInfo ¶m, double &card, double &op_cost, double &cost) { int ret = OB_SUCCESS; - if (OB_ISNULL(get_plan()) || - OB_ISNULL(get_stmt())) { + if (OB_ISNULL(get_plan()) || OB_ISNULL(get_stmt())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); - } else if (get_stmt()->is_insert_stmt() || is_values_table_) { + } else if (get_stmt()->is_insert_stmt()) { ObOptimizerContext &opt_ctx = get_plan()->get_optimizer_context(); - card = get_stmt()->is_insert_stmt() ? static_cast(get_stmt())->get_insert_row_count() : - get_values_row_count(); + card = static_cast(get_stmt())->get_insert_row_count(); op_cost = ObOptEstCost::cost_get_rows(get_card(), opt_ctx); cost = op_cost; + } else if (is_values_table_) { + if (OB_ISNULL(table_def_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else { + ObOptimizerContext &opt_ctx = get_plan()->get_optimizer_context(); + card = get_card(); + op_cost = ObOptEstCost::cost_filter_rows(table_def_->row_cnt_, filter_exprs_, opt_ctx); + cost = op_cost; + } } else { ObOptimizerContext &opt_ctx = get_plan()->get_optimizer_context(); card = 1.0; @@ -249,9 +257,18 @@ int ObLogExprValues::compute_one_row_info() if (OB_ISNULL(get_stmt())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); - } else if (!get_stmt()->is_insert_stmt()) { - is_at_most_one_row_ = get_values_row_count() <= 1; - } else { /*do nothing*/ } + } else if (get_stmt()->is_insert_stmt()) { + /* do nothing */ + } else if (is_values_table_) { + if (OB_ISNULL(table_def_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else { + is_at_most_one_row_ = table_def_->row_cnt_ <= 1; + } + } else { + is_at_most_one_row_ = true; + } return ret; } @@ -320,10 +337,26 @@ int ObLogExprValues::allocate_expr_post(ObAllocExprContext &ctx) LOG_WARN("failed to construct sequence values", K(ret)); } else if (OB_FAIL(mark_probably_local_exprs())) { LOG_WARN("failed to mark local exprs", K(ret)); - } else if (is_values_table_) { /* defence code */ - if (OB_UNLIKELY(value_desc_.count() != get_output_exprs().count())) { + } else if (is_values_table_) { + // defence code for 4_2_1 values table + if (OB_UNLIKELY(output_exprs_.count() != value_desc_.count())) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("defence code, EXPRESSION request output_exprs equals to value_desc", K(ret)); + LOG_WARN("values table should output is same as value_desc", K(ret), K(output_exprs_), K(value_desc_)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < output_exprs_.count(); i++) { + ObSEArray column_exprs; + if (OB_FAIL(ObRawExprUtils::extract_column_exprs(output_exprs_.at(i), column_exprs))) { + LOG_WARN("failed to extract column expr", K(ret)); + } else if (OB_UNLIKELY(column_exprs.count() >= 2)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("values table should output is same as value_desc", K(ret)); + } else if (column_exprs.empty()) { + /* do nothing */ + } else if (OB_UNLIKELY(value_desc_.at(i) != column_exprs.at(0))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("values table should output is same as value_desc", K(ret)); + } + } } } diff --git a/src/sql/optimizer/ob_log_expr_values.h b/src/sql/optimizer/ob_log_expr_values.h index aebddd7326..b7c29e2e14 100644 --- a/src/sql/optimizer/ob_log_expr_values.h +++ b/src/sql/optimizer/ob_log_expr_values.h @@ -25,7 +25,9 @@ class ObLogExprValues : public ObLogicalOperator : ObLogicalOperator(plan), err_log_define_(), is_values_table_(false), - table_name_() + table_name_(), + table_id_(common::OB_INVALID_ID), + table_def_(NULL) {} virtual ~ObLogExprValues() {} @@ -77,12 +79,14 @@ class ObLogExprValues : public ObLogicalOperator ObSqlPlanItem &plan_item) override; int get_array_param_group_id(int64_t &group_id, bool &find); void set_is_values_table(bool is_values_table) { is_values_table_ = is_values_table; } - int64_t get_values_row_count() const { - return is_values_table_ && value_desc_.count() > 0 ? value_exprs_.count() / value_desc_.count() : 1; - } + inline bool is_values_table() const { return is_values_table_; } inline common::ObString &get_table_name() { return table_name_; } inline const common::ObString &get_table_name() const { return table_name_; } inline void set_table_name(const common::ObString &table_name) { table_name_ = table_name; } + inline uint64_t get_table_id() const { return table_id_; } + inline void set_table_id(const uint64_t table_id) { table_id_ = table_id; } + inline const ObValuesTableDef *get_values_table_def() { return table_def_; } + inline void set_values_table_def(ObValuesTableDef *table_def) { table_def_ = table_def; } private: int construct_array_binding_values(); int construct_sequence_values(); @@ -94,6 +98,8 @@ class ObLogExprValues : public ObLogicalOperator //for values table bool is_values_table_; common::ObString table_name_; + uint64_t table_id_; //table id or alias table id + ObValuesTableDef *table_def_; DISALLOW_COPY_AND_ASSIGN(ObLogExprValues); }; diff --git a/src/sql/optimizer/ob_log_granule_iterator.cpp b/src/sql/optimizer/ob_log_granule_iterator.cpp index 4a0c9ed41c..7323132f8f 100644 --- a/src/sql/optimizer/ob_log_granule_iterator.cpp +++ b/src/sql/optimizer/ob_log_granule_iterator.cpp @@ -142,16 +142,11 @@ int ObLogGranuleIterator::compute_op_ordering() int ObLogGranuleIterator::set_range_order() { int ret = OB_SUCCESS; + const ObDMLStmt *stmt = NULL; common::ObIArray &op_ordering = get_op_ordering(); - if (!op_ordering.empty()) { - // Suppose (range) partition order is asc, so first order is same partition order - bool is_asc_order = is_ascending_direction(op_ordering.at(0).order_type_); - if (is_asc_order) { - add_flag(GI_ASC_ORDER); - } else { - add_flag(GI_DESC_ORDER); - } - LOG_TRACE("partition order", K(is_asc_order), K(gi_attri_flag_), K(ret)); + if (OB_ISNULL(stmt = get_plan()->get_stmt()) || OB_ISNULL(stmt->get_query_ctx())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null param", K(ret)); } else if (affinitize()) { ObLogicalOperator *child = get_child(first_child); if (OB_ISNULL(child)) { @@ -169,6 +164,23 @@ int ObLogGranuleIterator::set_range_order() } LOG_TRACE("affinitize partition order", K(is_asc_order), K(gi_attri_flag_), K(ret)); } + } else if (!op_ordering.empty()) { + // Suppose (range) partition order is asc, so first order is same partition order + bool is_asc_order = is_ascending_direction(op_ordering.at(0).order_type_); + bool used = true; + if (((stmt->get_query_ctx()->optimizer_features_enable_version_ >= COMPAT_VERSION_4_2_3 && + stmt->get_query_ctx()->optimizer_features_enable_version_ < COMPAT_VERSION_4_3_0) || + stmt->get_query_ctx()->optimizer_features_enable_version_ >= COMPAT_VERSION_4_3_2) && + OB_FAIL(check_op_orderding_used_by_parent(used))) { + LOG_WARN("failed to check op ordering used by parent", K(ret)); + } else if (!used) { + //do nothing + } else if (is_asc_order) { + add_flag(GI_ASC_ORDER); + } else { + add_flag(GI_DESC_ORDER); + } + LOG_TRACE("partition order", K(is_asc_order), K(gi_attri_flag_), K(ret)); } return ret; } diff --git a/src/sql/optimizer/ob_log_group_by.cpp b/src/sql/optimizer/ob_log_group_by.cpp index c071feb6aa..c31b1dd3d9 100644 --- a/src/sql/optimizer/ob_log_group_by.cpp +++ b/src/sql/optimizer/ob_log_group_by.cpp @@ -902,4 +902,19 @@ int ObLogGroupBy::get_card_without_filter(double &card) int ret = OB_SUCCESS; card = get_distinct_card(); return ret; +} + +int ObLogGroupBy::check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index) +{ + int ret = OB_SUCCESS; + used = true; + inherit_child_ordering_index = first_child; + if (HASH_AGGREGATE == get_algo()) { + inherit_child_ordering_index = -1; + used = false; + } else if (get_group_by_exprs().empty() && + get_rollup_exprs().empty()) { + used = false; + } + return ret; } \ No newline at end of file diff --git a/src/sql/optimizer/ob_log_group_by.h b/src/sql/optimizer/ob_log_group_by.h index a04c46cba9..61462d2d17 100644 --- a/src/sql/optimizer/ob_log_group_by.h +++ b/src/sql/optimizer/ob_log_group_by.h @@ -242,6 +242,7 @@ private: virtual int compute_one_row_info() override; virtual int print_outline_data(PlanText &plan_text) override; virtual int print_used_hint(PlanText &plan_text) override; + virtual int check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index)override; private: common::ObSEArray group_exprs_; common::ObSEArray rollup_exprs_; diff --git a/src/sql/optimizer/ob_log_join.cpp b/src/sql/optimizer/ob_log_join.cpp index d71f672855..b6da9c7d50 100644 --- a/src/sql/optimizer/ob_log_join.cpp +++ b/src/sql/optimizer/ob_log_join.cpp @@ -715,7 +715,7 @@ bool ObLogJoin::is_scan_operator(log_op_def::ObLogOpType type) { return LOG_TABLE_SCAN == type || LOG_SUBPLAN_SCAN == type || LOG_FUNCTION_TABLE == type || LOG_UNPIVOT == type || - LOG_TEMP_TABLE_ACCESS == type || LOG_JSON_TABLE == type; + LOG_TEMP_TABLE_ACCESS == type || LOG_JSON_TABLE == type || LOG_VALUES_TABLE_ACCESS == type; } int ObLogJoin::append_used_join_hint(ObIArray &used_hints) @@ -1509,3 +1509,25 @@ int ObLogJoin::get_card_without_filter(double &card) } return ret; } + +int ObLogJoin::check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index) +{ + int ret = OB_SUCCESS; + used = true; + inherit_child_ordering_index = first_child; + if (HASH_JOIN == get_join_algo()) { + inherit_child_ordering_index = -1; + used = false; + } else if (NESTED_LOOP_JOIN == get_join_algo()) { + used = false; + if (CONNECT_BY_JOIN == get_join_type()) { + inherit_child_ordering_index = -1; + } else { + inherit_child_ordering_index = first_child; + } + } else if (MERGE_JOIN == get_join_algo()) { + used = true; + inherit_child_ordering_index = first_child; + } + return ret; +} \ No newline at end of file diff --git a/src/sql/optimizer/ob_log_join.h b/src/sql/optimizer/ob_log_join.h index 3de8818fa4..65e156fbf5 100644 --- a/src/sql/optimizer/ob_log_join.h +++ b/src/sql/optimizer/ob_log_join.h @@ -222,7 +222,7 @@ namespace sql int print_join_tables_in_hint(const ObDMLStmt &stmt, PlanText &plan_text, const ObRelIds &table_set); - + virtual int check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index)override; private: // all join predicates common::ObSEArray join_conditions_; //equal join condition, for merge-join diff --git a/src/sql/optimizer/ob_log_operator_factory.cpp b/src/sql/optimizer/ob_log_operator_factory.cpp index 4d873da656..59bf3dddc5 100644 --- a/src/sql/optimizer/ob_log_operator_factory.cpp +++ b/src/sql/optimizer/ob_log_operator_factory.cpp @@ -52,6 +52,7 @@ #include "ob_del_upd_log_plan.h" #include "ob_log_link_dml.h" #include "ob_log_optimizer_stats_gathering.h" +#include "ob_log_values_table_access.h" using namespace oceanbase; using namespace oceanbase::sql; using namespace oceanbase::sql::log_op_def; @@ -358,6 +359,13 @@ ObLogicalOperator *ObLogOperatorFactory::allocate(ObLogPlan &plan, ObLogOpType t } break; } + case LOG_VALUES_TABLE_ACCESS: { + ptr = allocator_.alloc(sizeof(ObLogValuesTableAccess)); + if (NULL != ptr) { + ret_op = new (ptr) ObLogValuesTableAccess(plan); + } else { /* do nothing */ } + break; + } default: { break; } diff --git a/src/sql/optimizer/ob_log_plan.cpp b/src/sql/optimizer/ob_log_plan.cpp index 86e407c432..35d11bc119 100644 --- a/src/sql/optimizer/ob_log_plan.cpp +++ b/src/sql/optimizer/ob_log_plan.cpp @@ -77,6 +77,7 @@ #ifdef OB_BUILD_SPM #include "sql/spm/ob_spm_define.h" #endif +#include "sql/optimizer/ob_log_values_table_access.h" using namespace oceanbase; using namespace sql; @@ -6381,14 +6382,25 @@ int ObLogPlan::get_minimal_cost_candidate(const ObIArray &candida CandidatePlan &candidate) { int ret = OB_SUCCESS; - for (int64_t i = 0; OB_SUCC(ret) && i < candidates.count(); i++) { - if (OB_ISNULL(candidates.at(i).plan_tree_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("get unexpected null", K(ret)); - } else if (NULL == candidate.plan_tree_ || - candidates.at(i).plan_tree_->get_cost() < candidate.plan_tree_->get_cost()) { - candidate = candidates.at(i); - } else { /*do nothing*/ } + if (OB_UNLIKELY(get_optimizer_context().generate_random_plan())) { + ObQueryCtx* query_ctx; + if (OB_ISNULL(query_ctx = get_optimizer_context().get_query_ctx())) { + // ignore ret + LOG_WARN("unexpected null value", K(query_ctx)); + candidate = candidates.at(0); + } else { + candidate = candidates.at(query_ctx->rand_gen_.get(0, candidates.count() - 1)); + } + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < candidates.count(); i++) { + if (OB_ISNULL(candidates.at(i).plan_tree_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (NULL == candidate.plan_tree_ || + candidates.at(i).plan_tree_->get_cost() < candidate.plan_tree_->get_cost()) { + candidate = candidates.at(i); + } else { /*do nothing*/ } + } } return ret; } @@ -6538,6 +6550,7 @@ int ObLogPlan::create_order_by_plan(ObLogicalOperator *&top, bool need_sort = false; int64_t prefix_pos = 0; ObExchangeInfo exch_info; + bool is_at_most_one_row = top->get_is_at_most_one_row(); exch_info.dist_method_ = (NULL != top && top->is_single()) ? ObPQDistributeMethod::NONE : ObPQDistributeMethod::LOCAL; if (OB_ISNULL(top)) { @@ -6562,7 +6575,7 @@ int ObLogPlan::create_order_by_plan(ObLogicalOperator *&top, topn_expr, is_fetch_with_ties))) { LOG_WARN("failed to allocate sort as top", K(ret)); - } else { /*do nothing*/ } + } return ret; } @@ -9425,11 +9438,26 @@ int ObLogPlan::prune_and_keep_best_plans(ObIArray &candidate_plan int ret = OB_SUCCESS; ObSEArray best_plans; OPT_TRACE_TITLE("prune and keep best plans"); - for (int64_t i = 0; OB_SUCC(ret) && i < candidate_plans.count(); i++) { - CandidatePlan &candidate_plan = candidate_plans.at(i); - if (OB_FAIL(add_candidate_plan(best_plans, candidate_plan))) { - LOG_WARN("failed to add candidate plan", K(ret)); - } else { /*do nothing*/ } + if (OB_UNLIKELY(get_optimizer_context().generate_random_plan())) { + ObQueryCtx* query_ctx = get_optimizer_context().get_query_ctx(); + for (int64_t i = 0; OB_SUCC(ret) && i < candidate_plans.count(); i++) { + bool random_flag = !OB_ISNULL(query_ctx) && query_ctx->rand_gen_.get(0, 1) == 1; + if (random_flag && OB_FAIL(best_plans.push_back(candidate_plans.at(i)))) { + LOG_WARN("failed to push back random candi plan", K(ret)); + } + } + if (OB_SUCC(ret) && best_plans.empty() && !candidate_plans.empty()) { + if (OB_FAIL(best_plans.push_back(candidate_plans.at(0)))) { + LOG_WARN("failed to push back random candi plan", K(ret)); + } + } + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < candidate_plans.count(); i++) { + CandidatePlan &candidate_plan = candidate_plans.at(i); + if (OB_FAIL(add_candidate_plan(best_plans, candidate_plan))) { + LOG_WARN("failed to add candidate plan", K(ret)); + } else { /*do nothing*/ } + } } if (OB_SUCC(ret)) { if (OB_FAIL(init_candidate_plans(best_plans))) { @@ -13476,12 +13504,42 @@ int ObLogPlan::allocate_values_table_path(ValuesTablePath *values_table_path, ObLogicalOperator *&out_access_path_op) { int ret = OB_SUCCESS; - ObLogExprValues *values_op = NULL; - const TableItem *table_item = NULL; + + if ((GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_2_2_0 && + GET_MIN_CLUSTER_VERSION() < CLUSTER_VERSION_4_3_0_0) || + GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_3_2_0) { + ObLogValuesTableAccess *values_op = NULL; + if (OB_FAIL(do_alloc_values_table_path(values_table_path, values_op))) { + LOG_WARN("failed to allocate values table access op", K(ret)); + } else { + out_access_path_op = values_op; + } + } else { + ObLogExprValues *values_op = NULL; + if (OB_FAIL(do_alloc_values_table_path(values_table_path, values_op))) { + LOG_WARN("failed to allocate values table access op", K(ret)); + } else { + out_access_path_op = values_op; + } + } + return ret; +} + +int ObLogPlan::do_alloc_values_table_path(ValuesTablePath *values_table_path, + ObLogExprValues *&values_op) +{ + int ret = OB_SUCCESS; + TableItem *table_item = NULL; + ObValuesTableDef *table_def = NULL; if (OB_ISNULL(values_table_path) || OB_ISNULL(get_stmt()) || OB_ISNULL(table_item = get_stmt()->get_table_item_by_id(values_table_path->table_id_))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(values_table_path), K(get_stmt()), K(ret)); + } else if (OB_UNLIKELY(!table_item->is_values_table()) || + OB_ISNULL(table_def = values_table_path->table_def_) || + OB_UNLIKELY(0 == table_def->column_cnt_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed. get unexpect param", K(ret), K(*table_item), KP(table_def)); } else if (OB_ISNULL(values_op = static_cast(get_log_op_factory(). allocate(*this, LOG_EXPR_VALUES)))) { ret = OB_ALLOCATE_MEMORY_FAILED; @@ -13489,8 +13547,10 @@ int ObLogPlan::allocate_values_table_path(ValuesTablePath *values_table_path, } else { values_op->set_table_name(table_item->get_table_name()); values_op->set_is_values_table(true); + values_op->set_table_id(values_table_path->table_id_); + values_op->set_values_table_def(table_def); ObSEArray values_desc; - if (OB_FAIL(values_op->add_values_expr(table_item->table_values_))) { + if (OB_FAIL(values_op->add_values_expr(table_def->access_exprs_))) { LOG_WARN("failed to add values expr", K(ret)); } else if (OB_FAIL(get_stmt()->get_column_exprs(values_table_path->table_id_, values_desc))) { LOG_WARN("failed to get column exprs"); @@ -13502,8 +13562,47 @@ int ObLogPlan::allocate_values_table_path(ValuesTablePath *values_table_path, LOG_WARN("failed to compute propery", K(ret)); } else if (OB_FAIL(values_op->pick_out_startup_filters())) { LOG_WARN("failed to pick out startup filters", K(ret)); - } else { - out_access_path_op = values_op; + } + } + return ret; +} + +int ObLogPlan::do_alloc_values_table_path(ValuesTablePath *values_table_path, + ObLogValuesTableAccess *&values_op) +{ + int ret = OB_SUCCESS; + TableItem *table_item = NULL; + ObValuesTableDef *table_def = NULL; + if (OB_ISNULL(values_table_path) || OB_ISNULL(get_stmt()) || + OB_ISNULL(table_item = get_stmt()->get_table_item_by_id(values_table_path->table_id_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(values_table_path), K(get_stmt()), K(ret)); + } else if (OB_UNLIKELY(!table_item->is_values_table()) || + OB_ISNULL(table_def = values_table_path->table_def_) || + OB_UNLIKELY(0 == table_def->column_cnt_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed. get unexpect param", K(ret), K(*table_item), KP(table_def)); + } else if (OB_ISNULL(values_op = static_cast(get_log_op_factory(). + allocate(*this, LOG_VALUES_TABLE_ACCESS)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to allocate values op", K(ret)); + } else { + values_op->set_table_name(table_item->get_table_name()); + values_op->set_table_id(values_table_path->table_id_); + values_op->set_values_table_def(table_def); + values_op->set_values_path(values_table_path); + ObSEArray column_exprs; + if (OB_FAIL(get_stmt()->get_column_exprs(values_table_path->table_id_, column_exprs))) { + LOG_WARN("failed to get column exprs"); + } else if (OB_UNLIKELY(column_exprs.count() != table_def->column_cnt_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("not allow to do project pruning now", K(ret)); + } else if (OB_FAIL(values_op->get_column_exprs().assign(column_exprs))) { + LOG_WARN("failed to add values desc", K(ret)); + } else if (OB_FAIL(append(values_op->get_filter_exprs(), values_table_path->filter_))) { + LOG_WARN("failed to append expr", K(ret)); + } else if (OB_FAIL(values_op->compute_property(values_table_path))) { + LOG_WARN("failed to compute propery", K(ret)); } } return ret; @@ -13937,4 +14036,4 @@ int ObLogPlan::init_lateral_table_depend_info(const ObIArray &table_ LOG_TRACE("succeed to init function table depend info", K(table_depend_infos_)); } return ret; -} \ No newline at end of file +} diff --git a/src/sql/optimizer/ob_log_plan.h b/src/sql/optimizer/ob_log_plan.h index 2c4718b23d..ae40c206bc 100644 --- a/src/sql/optimizer/ob_log_plan.h +++ b/src/sql/optimizer/ob_log_plan.h @@ -1716,6 +1716,10 @@ public: bool has_added_win_dist() const { return outline_print_flags_ & ADDED_WIN_DIST_HINT; } void set_added_win_dist() { outline_print_flags_ |= ADDED_WIN_DIST_HINT; } const common::ObIArray &get_onetime_query_refs() const { return onetime_query_refs_; } + int do_alloc_values_table_path(ValuesTablePath *values_table_path, + ObLogExprValues *&out_access_path_op); + int do_alloc_values_table_path(ValuesTablePath *values_table_path, + ObLogValuesTableAccess *&out_access_path_op); private: static const int64_t IDP_PATHNUM_THRESHOLD = 5000; protected: // member variable diff --git a/src/sql/optimizer/ob_log_set.cpp b/src/sql/optimizer/ob_log_set.cpp index dc6c583209..a93f6cc88a 100644 --- a/src/sql/optimizer/ob_log_set.cpp +++ b/src/sql/optimizer/ob_log_set.cpp @@ -910,4 +910,21 @@ int ObLogSet::get_card_without_filter(double &card) } } return ret; +} + +int ObLogSet::check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index) +{ + int ret = OB_SUCCESS; + used = true; + inherit_child_ordering_index = first_child; + if (HASH_SET == get_algo()) { + inherit_child_ordering_index = -1; + used = false; + } else if (!is_set_distinct()) { + used = false; + } + if (is_recursive_union()) { + inherit_child_ordering_index = -1; + } + return ret; } \ No newline at end of file diff --git a/src/sql/optimizer/ob_log_set.h b/src/sql/optimizer/ob_log_set.h index e119ab08dd..813d4541e9 100644 --- a/src/sql/optimizer/ob_log_set.h +++ b/src/sql/optimizer/ob_log_set.h @@ -125,6 +125,7 @@ public: int add_child_ndv(double ndv) { return child_ndv_.push_back(ndv); } virtual int get_card_without_filter(double &card) override; int append_child_fd_item_set(ObFdItemSet &all_fd_item_set, const ObFdItemSet &child_fd_item_set); + virtual int check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index)override; private: bool is_distinct_; bool is_recursive_union_; diff --git a/src/sql/optimizer/ob_log_sort.cpp b/src/sql/optimizer/ob_log_sort.cpp index 02802ed8b6..e4d504cce3 100644 --- a/src/sql/optimizer/ob_log_sort.cpp +++ b/src/sql/optimizer/ob_log_sort.cpp @@ -535,3 +535,14 @@ int ObLogSort::is_my_fixed_expr(const ObRawExpr *expr, bool &is_fixed) } return ret; } + +int ObLogSort::check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index) +{ + int ret = OB_SUCCESS; + used = true; + inherit_child_ordering_index = -1; + if (!is_prefix_sort() && !is_local_merge_sort()) { + used = false; + } + return ret; +} diff --git a/src/sql/optimizer/ob_log_sort.h b/src/sql/optimizer/ob_log_sort.h index 665070c369..1ac93021ff 100644 --- a/src/sql/optimizer/ob_log_sort.h +++ b/src/sql/optimizer/ob_log_sort.h @@ -102,6 +102,7 @@ namespace sql virtual int compute_op_ordering() override; virtual int get_plan_item_info(PlanText &plan_text, ObSqlPlanItem &plan_item) override; + virtual int check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index)override; protected: virtual int inner_replace_op_exprs(ObRawExprReplacer &replacer); int est_sort_key_width(); diff --git a/src/sql/optimizer/ob_log_subplan_filter.cpp b/src/sql/optimizer/ob_log_subplan_filter.cpp index ede8ccd846..4763261877 100644 --- a/src/sql/optimizer/ob_log_subplan_filter.cpp +++ b/src/sql/optimizer/ob_log_subplan_filter.cpp @@ -319,22 +319,6 @@ int ObLogSubPlanFilter::allocate_granule_post(AllocGIContext &ctx) int ret = OB_SUCCESS; if (OB_FAIL(pw_allocate_granule_post(ctx))) { LOG_WARN("failed to do pw allocate granule post", K(ret)); - } else if (DistAlgo::DIST_NONE_ALL == dist_algo_) { - ObLogicalOperator *op = NULL; - bool cnt_pd_range_cond = false; - if (OB_ISNULL(get_child(first_child))) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("invalid child", K(ret)); - } else if (OB_FAIL(get_child(first_child)->find_first_recursive(log_op_def::LOG_GRANULE_ITERATOR, op))) { - LOG_WARN("find granule iterator in right failed", K(ret)); - } else if (NULL == op) { - // granule iterator not found, do nothing - } else if (OB_FAIL(ObOptimizerUtil::check_pushdown_range_cond(get_child(first_child), cnt_pd_range_cond))) { - LOG_WARN("failed to check any push down range cond", K(ret)); - } else if (cnt_pd_range_cond) { - static_cast(op)->add_flag(GI_NLJ_PARAM_DOWN); - static_cast(op)->add_flag(GI_FORCE_PARTITION_GRANULE); - } } else if (DistAlgo::DIST_PARTITION_NONE == dist_algo_) { ObLogicalOperator *op = NULL; ObLogicalOperator *child = NULL; diff --git a/src/sql/optimizer/ob_log_table_scan.cpp b/src/sql/optimizer/ob_log_table_scan.cpp index 45cb422037..25e4711eff 100644 --- a/src/sql/optimizer/ob_log_table_scan.cpp +++ b/src/sql/optimizer/ob_log_table_scan.cpp @@ -119,38 +119,54 @@ int ObLogTableScan::do_re_est_cost(EstimateCostInfo ¶m, double &card, double double limit_percent = -1.0; int64_t limit_count = -1; int64_t offset_count = 0; - ObOptimizerContext *opt_ctx = NULL; + const ObDMLStmt *stmt = NULL; if (OB_ISNULL(access_path_)) { // table scan create from CteTablePath card = get_card(); op_cost = get_op_cost(); cost = get_cost(); - } else if (OB_ISNULL(get_plan()) || OB_ISNULL(opt_ctx = &get_plan()->get_optimizer_context()) - || OB_ISNULL(est_cost_info_) || OB_UNLIKELY(1 > param.need_parallel_)) { + } else if (OB_ISNULL(get_plan()) || OB_ISNULL(est_cost_info_) || + OB_ISNULL(stmt = get_plan()->get_stmt()) || OB_ISNULL(stmt->get_query_ctx()) || + OB_UNLIKELY(1 > param.need_parallel_)) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("get unexpected params", K(ret), K(opt_ctx), K(est_cost_info_), K(param)); + LOG_WARN("get unexpected params", K(ret), K(est_cost_info_), K(param)); } else if (OB_FAIL(get_limit_offset_value(NULL, limit_count_expr_, limit_offset_expr_, limit_percent, limit_count, offset_count))) { LOG_WARN("failed to get limit offset value", K(ret)); } else { card = get_card(); - double limit_count_double = static_cast(limit_count); - double offset_count_double = static_cast(offset_count); - param.need_row_count_ = 0 > param.need_row_count_ ? card : param.need_row_count_; + int64_t part_count = est_cost_info_->index_meta_info_.index_part_count_; if (0 <= limit_count) { - param.need_row_count_ = std::min(param.need_row_count_, limit_count_double * param.need_parallel_); + if (!use_das()) { + limit_count *= part_count; + offset_count *= part_count; + } + double need_row_count = limit_count + offset_count; + need_row_count = std::min(need_row_count, card); + if (param.need_row_count_ < 0) { + param.need_row_count_ = need_row_count; + } else { + param.need_row_count_ = std::min(param.need_row_count_, need_row_count); + } + } + if (((stmt->get_query_ctx()->optimizer_features_enable_version_ >= COMPAT_VERSION_4_2_3 && + stmt->get_query_ctx()->optimizer_features_enable_version_ < COMPAT_VERSION_4_3_0) || + stmt->get_query_ctx()->optimizer_features_enable_version_ >= COMPAT_VERSION_4_3_2) && + range_conds_.empty() && + (!est_cost_info_->postfix_filters_.empty() || + !est_cost_info_->table_filters_.empty() || + !est_cost_info_->ss_postfix_range_filters_.empty())) { + //full scan with table filters + param.need_row_count_ = -1; } - param.need_row_count_ = std::min(param.need_row_count_, card); - param.need_row_count_ += offset_count_double; if (OB_FAIL(AccessPath::re_estimate_cost(param, - *est_cost_info_, - sample_info_, - *opt_ctx, - card, - op_cost))) { + *est_cost_info_, + sample_info_, + get_plan()->get_optimizer_context(), + card, + op_cost))) { LOG_WARN("failed to re estimate cost", K(ret)); } else { cost = op_cost; - card = std::min(param.need_row_count_ - offset_count_double, card); } } return ret; @@ -1881,20 +1897,8 @@ int ObLogTableScan::print_outline_data(PlanText &plan_text) TableItem *table_item = NULL; ObString qb_name; const ObString *index_name = NULL; + int64_t index_prefix = -1; ObItemType index_type = T_INDEX_HINT; - if (is_skip_scan()) { - index_type = T_INDEX_SS_HINT; - if (ref_table_id_ == index_table_id_) { - index_name = &ObIndexHint::PRIMARY_KEY; - } else { - index_name = &get_index_name(); - } - } else if (ref_table_id_ == index_table_id_) { - index_type = T_FULL_HINT; - } else { - index_type = T_INDEX_HINT; - index_name = &get_index_name(); - } const ObDMLStmt *stmt = NULL; if (OB_ISNULL(get_plan()) || OB_ISNULL(stmt = get_plan()->get_stmt())) { ret = OB_ERR_UNEXPECTED; @@ -1913,6 +1917,29 @@ int ObLogTableScan::print_outline_data(PlanText &plan_text) LOG_WARN("failed to print table parallel hint", K(ret)); } } + if (OB_FAIL(ret)) { + } else if (is_skip_scan()) { + index_type = T_INDEX_SS_HINT; + if (ref_table_id_ == index_table_id_) { + index_name = &ObIndexHint::PRIMARY_KEY; + } else { + index_name = &get_index_name(); + } + } else if (OB_FAIL(get_plan()->get_log_plan_hint().get_index_prefix(table_id_, + index_table_id_, + index_prefix))) { + LOG_WARN("failed to get index prefix", K(table_id_), K(index_table_id_), K(index_prefix), K(ret)); + } else if (ref_table_id_ == index_table_id_ && index_prefix < 0) { + index_type = T_FULL_HINT; + index_name = &ObIndexHint::PRIMARY_KEY; + } else { + index_type = T_INDEX_HINT; + if (ref_table_id_ == index_table_id_) { + index_name = &ObIndexHint::PRIMARY_KEY; + } else { + index_name = &get_index_name(); + } + } if (OB_FAIL(ret)) { } else if (need_late_materialization() && @@ -1930,6 +1957,7 @@ int ObLogTableScan::print_outline_data(PlanText &plan_text) ObIndexHint index_hint(index_type); index_hint.set_qb_name(qb_name); index_hint.get_table().set_table(*table_item); + index_hint.get_index_prefix() = index_prefix; if (NULL != index_name) { index_hint.get_index_name().assign_ptr(index_name->ptr(), index_name->length()); } @@ -2033,12 +2061,35 @@ int ObLogTableScan::set_limit_offset(ObRawExpr *limit, ObRawExpr *offset) limit_offset_expr_ = offset; EstimateCostInfo param; param.need_parallel_ = get_parallel(); - if (OB_FAIL(do_re_est_cost(param, card, op_cost, cost))) { - LOG_WARN("failed to re est cost error", K(ret)); + + double limit_percent = -1.0; + int64_t limit_count = -1; + int64_t offset_count = 0; + double index_back_cost = 0.0; + if (OB_ISNULL(est_cost_info_)) { + //fake cte path + } else if (OB_FAIL(get_limit_offset_value(NULL, limit_count_expr_, limit_offset_expr_, + limit_percent, limit_count, offset_count))) { + LOG_WARN("failed to get limit offset value", K(ret)); } else { - set_op_cost(op_cost); - set_cost(cost); - set_card(card); + int64_t part_count = est_cost_info_->index_meta_info_.index_part_count_; + if (0 <= limit_count) { + if (!use_das()) { + limit_count *= part_count; + offset_count *= part_count; + } + param.need_row_count_ = limit_count + offset_count; + param.need_row_count_ = std::min(param.need_row_count_, get_card()); + } + if (OB_FAIL(do_re_est_cost(param, card, op_cost, cost))) { + LOG_WARN("failed to re est cost error", K(ret)); + } else { + card = std::min(card, static_cast(limit_count)); + set_op_cost(op_cost); + set_cost(cost); + set_card(card); + LOG_TRACE("push limit into table scan", K(param), K(limit_count), K(part_count), K(card)); + } } return ret; } diff --git a/src/sql/optimizer/ob_log_values_table_access.cpp b/src/sql/optimizer/ob_log_values_table_access.cpp new file mode 100644 index 0000000000..b0e2761df0 --- /dev/null +++ b/src/sql/optimizer/ob_log_values_table_access.cpp @@ -0,0 +1,240 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_OPT + +#include "sql/optimizer/ob_log_values_table_access.h" +#include "sql/optimizer/ob_opt_est_cost.h" +#include "sql/optimizer/ob_log_plan.h" + +using namespace oceanbase::common; + +namespace oceanbase +{ +namespace sql +{ +/** + * Print log info with expressions + */ +#define EXPLAIN_PRINT_VALUES_TABLE(access, type) \ + { \ + if (OB_FAIL(BUF_PRINTF(#access"("))) { \ + LOG_WARN("fail to print to buf", K(ret)); \ + } else { \ + int64_t N = access.count(); \ + if (N == 0) { \ + if (OB_FAIL(BUF_PRINTF("nil"))) { \ + LOG_WARN("fail to print to buf", K(ret)); \ + } \ + } else { \ + for (int64_t i = 0; OB_SUCC(ret) && i < N; i++) { \ + if (OB_ISNULL(access.at(i))) { \ + ret = OB_ERR_UNEXPECTED; \ + } else if (OB_FAIL(BUF_PRINTF("["))) { \ + LOG_WARN("fail to print to buf", K(ret)); \ + } else if (OB_FAIL(access.at(i)->get_name(buf, buf_len, pos, type))) { \ + } else { \ + if (i < N - 1) { \ + if (OB_FAIL(BUF_PRINTF("], "))) { \ + LOG_WARN("fail to print to buf", K(ret)); \ + } \ + } else if (OB_FAIL(BUF_PRINTF("]"))) { \ + LOG_WARN("fail to print to buf", K(ret)); \ + } \ + } \ + } \ + } \ + if (OB_SUCC(ret)) { \ + if (OB_FAIL(BUF_PRINTF(")"))) { \ + LOG_WARN("fail to print to buf", K(ret)); \ + } \ + } \ + } \ + } + +int ObLogValuesTableAccess::compute_equal_set() +{ + int ret = OB_SUCCESS; + set_output_equal_sets(&empty_expr_sets_); + return ret; +} + +int ObLogValuesTableAccess::compute_table_set() +{ + int ret = OB_SUCCESS; + set_table_set(&empty_table_set_); + return ret; +} + +int ObLogValuesTableAccess::est_cost() +{ + int ret = OB_SUCCESS; + double card = 0.0; + double op_cost = 0.0; + double cost = 0.0; + EstimateCostInfo param; + param.need_parallel_ = get_parallel(); + if (OB_FAIL(do_re_est_cost(param, card, op_cost, cost))) { + LOG_WARN("failed to get re est cost infos", K(ret)); + } else { + set_card(card); + set_op_cost(op_cost); + set_cost(cost); + } + return ret; +} + +int ObLogValuesTableAccess::do_re_est_cost(EstimateCostInfo ¶m, double &card, double &op_cost, double &cost) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(get_plan()) || OB_ISNULL(get_values_path()) || OB_ISNULL(table_def_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else { + ObOptimizerContext &opt_ctx = get_plan()->get_optimizer_context(); + double read_rows = table_def_->row_cnt_; + card = get_values_path()->get_path_output_rows(); + if (param.need_row_count_ >= 0 && param.need_row_count_ < card) { + read_rows = read_rows * param.need_row_count_ / card; + card = param.need_row_count_; + } + op_cost = ObOptEstCost::cost_values_table(read_rows, filter_exprs_, opt_ctx); + cost = op_cost; + } + return ret; +} + +int ObLogValuesTableAccess::get_op_exprs(ObIArray &all_exprs) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(table_def_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(ObLogicalOperator::get_op_exprs(all_exprs))) { + LOG_WARN("failed to get op exprs", K(ret)); + } else if (OB_FAIL(append(all_exprs, table_def_->access_exprs_))) { + LOG_WARN("failed to append exprs", K(ret)); + } else if (OB_FAIL(append(all_exprs, column_exprs_))) { + LOG_WARN("failed to append exprs", K(ret)); + } else { /*do nothing*/ } + return ret; +} + +int ObLogValuesTableAccess::allocate_expr_post(ObAllocExprContext &ctx) +{ + int ret = OB_SUCCESS; + for (int64_t i = 0; OB_SUCC(ret) && i < column_exprs_.count(); ++i) { + ObColumnRefRawExpr *value_col = column_exprs_.at(i); + if (OB_FAIL(mark_expr_produced(value_col, branch_id_, id_, ctx))) { + LOG_WARN("makr expr produced failed", K(ret)); + } else if (!is_plan_root() && OB_FAIL(output_exprs_.push_back(value_col))) { + LOG_WARN("failed to push back exprs", K(ret)); + } else { /*do nothing*/ } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(ObLogicalOperator::allocate_expr_post(ctx))) { + LOG_WARN("failed to allocate expr post", K(ret)); + } else if (get_output_exprs().empty() && OB_FAIL(allocate_dummy_output())) { + LOG_WARN("failed to allocate dummy output", K(ret)); + } else if (OB_FAIL(mark_probably_local_exprs())) { + LOG_WARN("failed to mark local exprs", K(ret)); + } else { /*do nothing*/ } + return ret; +} + +int ObLogValuesTableAccess::get_plan_item_info(PlanText &plan_text, + ObSqlPlanItem &plan_item) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObLogicalOperator::get_plan_item_info(plan_text, plan_item))) { + LOG_WARN("failed to get plan item info", K(ret)); + } else { + const ObIArray &access = get_column_exprs(); + BEGIN_BUF_PRINT; + EXPLAIN_PRINT_VALUES_TABLE(access, type); + END_BUF_PRINT(plan_item.special_predicates_, + plan_item.special_predicates_len_); + if (OB_SUCC(ret)) { + const ObString &name = get_table_name(); + BUF_PRINT_OB_STR(name.ptr(), + name.length(), + plan_item.object_alias_, + plan_item.object_alias_len_); + BUF_PRINT_STR("VALUES TABLE", plan_item.object_type_, plan_item.object_type_len_); + } + } + return ret; +} + +int ObLogValuesTableAccess::mark_probably_local_exprs() +{ + int ret = OB_SUCCESS; + FOREACH_CNT_X(e, table_def_->access_exprs_, OB_SUCC(ret)) { + CK(NULL != *e); + OZ((*e)->add_flag(IS_PROBABLY_LOCAL)); + } + + return ret; +} + +int ObLogValuesTableAccess::allocate_dummy_output() +{ + int ret = OB_SUCCESS; + ObConstRawExpr *dummy_expr = NULL; + if (OB_ISNULL(get_plan())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (is_oracle_mode() && ObRawExprUtils::build_const_number_expr( + get_plan()->get_optimizer_context().get_expr_factory(), + ObNumberType, + number::ObNumber::get_positive_one(), + dummy_expr)) { + LOG_WARN("failed to build const expr", K(ret)); + } else if (!is_oracle_mode() && ObRawExprUtils::build_const_int_expr( + get_plan()->get_optimizer_context().get_expr_factory(), + ObIntType, + 1, + dummy_expr)) { + LOG_WARN("failed to build const expr", K(ret)); + } else if (OB_ISNULL(dummy_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(dummy_expr->extract_info())) { + LOG_WARN("failed to extract info for dummy expr", K(ret)); + } else if (OB_FAIL(output_exprs_.push_back(dummy_expr))) { + LOG_WARN("failed to push back expr", K(ret)); + } else if (OB_FAIL(get_plan()->get_optimizer_context().get_all_exprs().append(dummy_expr))) { + LOG_WARN("failed to append exprs", K(ret)); + } else { /*do nothing*/ } + return ret; +} + +int ObLogValuesTableAccess::inner_replace_op_exprs(ObRawExprReplacer &replacer) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(table_def_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid argument", K(ret)); + } else if (OB_FAIL(replace_exprs_action(replacer, table_def_->access_exprs_))) { + LOG_WARN("failed to replace exprs", K(ret)); + } + return ret; +} + +int ObLogValuesTableAccess::is_my_fixed_expr(const ObRawExpr *expr, bool &is_fixed) +{ + is_fixed = ObOptimizerUtil::find_item(column_exprs_, expr); + return OB_SUCCESS; +} + +} // namespace sql +}// namespace oceanbase diff --git a/src/sql/optimizer/ob_log_values_table_access.h b/src/sql/optimizer/ob_log_values_table_access.h new file mode 100644 index 0000000000..a80c21188e --- /dev/null +++ b/src/sql/optimizer/ob_log_values_table_access.h @@ -0,0 +1,67 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef _OB_LOG_VALUES_TABLE_ACCESS_H +#define _OB_LOG_VALUES_TABLE_ACCESS_H +#include "sql/optimizer/ob_logical_operator.h" + +namespace oceanbase +{ +namespace sql +{ +class ObLogValuesTableAccess : public ObLogicalOperator + { + public: + ObLogValuesTableAccess(ObLogPlan &plan) + : ObLogicalOperator(plan), + table_def_(NULL), + table_name_(), + table_id_(common::OB_INVALID_ID), + values_path_(NULL) + {} + virtual ~ObLogValuesTableAccess() {} + virtual int est_cost() override; + virtual int do_re_est_cost(EstimateCostInfo ¶m, double &card, double &op_cost, double &cost); + virtual int compute_equal_set() override; + virtual int compute_table_set() override; + virtual int get_op_exprs(ObIArray &all_exprs) override; + virtual int is_my_fixed_expr(const ObRawExpr *expr, bool &is_fixed) override; + virtual int allocate_expr_post(ObAllocExprContext &ctx) override; + virtual int inner_replace_op_exprs(ObRawExprReplacer &replacer) override; + virtual int get_plan_item_info(PlanText &plan_text, ObSqlPlanItem &plan_item) override; + int mark_probably_local_exprs(); + int allocate_dummy_output(); + + const common::ObIArray &get_column_exprs() const { return column_exprs_; } + common::ObIArray &get_column_exprs() { return column_exprs_; } + inline const ObValuesTableDef *get_values_table_def() { return table_def_; } + inline void set_values_table_def(ObValuesTableDef *table_def) { table_def_ = table_def; } + inline common::ObString &get_table_name() { return table_name_; } + inline const common::ObString &get_table_name() const { return table_name_; } + inline void set_table_name(const common::ObString &table_name) { table_name_ = table_name; } + inline uint64_t get_table_id() const { return table_id_; } + inline void set_table_id(const uint64_t table_id) { table_id_ = table_id; } + inline void set_values_path(ValuesTablePath *values_path) { values_path_ = values_path; } + inline const ValuesTablePath *get_values_path() { return values_path_; } + private: + + private: + common::ObSEArray column_exprs_; + ObValuesTableDef *table_def_; + common::ObString table_name_; + uint64_t table_id_; + ValuesTablePath *values_path_; + DISALLOW_COPY_AND_ASSIGN(ObLogValuesTableAccess); + }; +} +} +#endif diff --git a/src/sql/optimizer/ob_log_window_function.cpp b/src/sql/optimizer/ob_log_window_function.cpp index de51ce24d5..d7ec3c86c9 100644 --- a/src/sql/optimizer/ob_log_window_function.cpp +++ b/src/sql/optimizer/ob_log_window_function.cpp @@ -614,3 +614,14 @@ int ObLogWindowFunction::is_my_fixed_expr(const ObRawExpr *expr, bool &is_fixed) is_fixed = ObOptimizerUtil::find_item(win_exprs_, expr); return OB_SUCCESS; } + +int ObLogWindowFunction::check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index) +{ + int ret = OB_SUCCESS; + used = true; + inherit_child_ordering_index = first_child; + if (get_sort_keys().empty()) { + used = false; + } + return ret; +} \ No newline at end of file diff --git a/src/sql/optimizer/ob_log_window_function.h b/src/sql/optimizer/ob_log_window_function.h index d856ab2d0b..73440717db 100644 --- a/src/sql/optimizer/ob_log_window_function.h +++ b/src/sql/optimizer/ob_log_window_function.h @@ -113,6 +113,7 @@ namespace sql int add_win_dist_options(const ObLogicalOperator *op, const ObIArray &all_win_funcs, ObWindowDistHint &win_dist_hint); + virtual int check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index)override; private: ObSEArray win_exprs_; diff --git a/src/sql/optimizer/ob_logical_operator.cpp b/src/sql/optimizer/ob_logical_operator.cpp index 826f9e8116..039cb8c9f2 100644 --- a/src/sql/optimizer/ob_logical_operator.cpp +++ b/src/sql/optimizer/ob_logical_operator.cpp @@ -3547,8 +3547,10 @@ int ObLogicalOperator::project_pruning_pre() int ret = OB_SUCCESS; // delete exprs who appeared in current op's output_exprs // but not used by it's parent's output_exprs_ - if (NULL != parent_ && !is_plan_root() && (LOG_EXPR_VALUES != type_) && - !(LOG_EXCHANGE == type_ && static_cast(this)->get_is_remote())) { + if (NULL != parent_ && !is_plan_root() && + LOG_EXPR_VALUES != type_ && + !(LOG_EXCHANGE == type_ && static_cast(this)->get_is_remote()) && + LOG_VALUES_TABLE_ACCESS != type_) { PPDeps deps; if (OB_FAIL(parent_->check_output_dependance(get_output_exprs(), deps))) { LOG_WARN("parent_->check_output_dep() fails", K(ret)); @@ -3594,8 +3596,7 @@ int ObLogicalOperator::project_pruning_pre() return ret; } -void ObLogicalOperator::do_project_pruning(ObIArray &exprs, - PPDeps &deps) +void ObLogicalOperator::do_project_pruning(ObIArray &exprs, PPDeps &deps) { int64_t i = 0; int64_t j = 0; @@ -4230,6 +4231,7 @@ int ObLogicalOperator::allocate_granule_nodes_above(AllocGIContext &ctx) int ret = OB_SUCCESS; bool partition_granule = false; bool has_temp_table_access = false; + const ObDMLStmt *stmt = NULL; // op granule iterator // | -> | // other op @@ -4237,7 +4239,8 @@ int ObLogicalOperator::allocate_granule_nodes_above(AllocGIContext &ctx) // other if (!ctx.alloc_gi_) { //do nothing - } else if (OB_ISNULL(get_plan()) || OB_ISNULL(get_sharding())) { + } else if (OB_ISNULL(get_plan()) || OB_ISNULL(get_sharding()) || + OB_ISNULL(stmt = get_plan()->get_stmt()) || OB_ISNULL(stmt->get_query_ctx())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("Get unexpected null", K(ret), K(get_plan()), K(get_sharding())); } else if (!get_plan()->get_optimizer_context().get_temp_table_infos().empty() && @@ -4269,6 +4272,17 @@ int ObLogicalOperator::allocate_granule_nodes_above(AllocGIContext &ctx) } else { ObLogGranuleIterator *gi_op = static_cast(log_op); if (NULL != get_parent()) { + //check topN sort + if (((stmt->get_query_ctx()->optimizer_features_enable_version_ >= COMPAT_VERSION_4_2_3 && + stmt->get_query_ctx()->optimizer_features_enable_version_ < COMPAT_VERSION_4_3_0) || + stmt->get_query_ctx()->optimizer_features_enable_version_ >= COMPAT_VERSION_4_3_2) && + LOG_SORT == get_parent()->get_type()) { + ObLogSort *parent = static_cast(get_parent()); + if (parent->is_local_merge_sort() && + NULL != parent->get_topn_expr()) { + gi_op->add_flag(GI_FORCE_PARTITION_GRANULE); + } + } bool found_child = false; for (int64_t i = 0; OB_SUCC(ret) && !found_child && i < get_parent()->get_num_of_child(); ++i) { if (get_parent()->get_child(i) == this) { @@ -6012,6 +6026,35 @@ int ObLogicalOperator::alloc_nodes_above(AllocOpContext& ctx, const uint64_t &fl return ret; } +int ObLogicalOperator::check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index) +{ + int ret = OB_SUCCESS; + if (get_num_of_child() > 0) { + if (LOG_GRANULE_ITERATOR == get_type() || + LOG_TEMP_TABLE_TRANSFORMATION == get_type() || + LOG_TEMP_TABLE_INSERT == get_type() || + LOG_SUBPLAN_SCAN == get_type() || + LOG_SUBPLAN_FILTER == get_type() || + LOG_MATERIAL == get_type() || + LOG_JOIN_FILTER == get_type() || + LOG_FOR_UPD == get_type() || + LOG_COUNT == get_type() || + LOG_LIMIT == get_type() || + LOG_STAT_COLLECTOR == get_type() || + LOG_OPTIMIZER_STATS_GATHERING == get_type() || + LOG_SELECT_INTO == get_type()) { + used = false; + } else { + used = true; + } + inherit_child_ordering_index = first_child; + } else { + used = false; + inherit_child_ordering_index = -1; + } + return ret; +} + int ObLogicalOperator::open_px_resource_analyze(OPEN_PX_RESOURCE_ANALYZE_DECLARE_ARG) { int ret = OB_SUCCESS; @@ -6199,3 +6242,42 @@ int ObLogicalOperator::find_max_px_resource_child(OPEN_PX_RESOURCE_ANALYZE_DECLA } return ret; } + + +int ObLogicalOperator::check_op_orderding_used_by_parent(bool &used) +{ + int ret = OB_SUCCESS; + used = true; + bool is_first_child = true; + bool inherit_child_ordering = true; + int64_t inherit_child_ordering_index = -1; + ObLogicalOperator *parent = get_parent(); + ObLogicalOperator *child = this; + while (OB_SUCC(ret) && NULL != parent) { + if (OB_FAIL(parent->check_use_child_ordering(used, inherit_child_ordering_index))) { + LOG_WARN("failed to check use child ordering", K(ret)); + } else { + inherit_child_ordering = child == parent->get_child(inherit_child_ordering_index); + if (!used && inherit_child_ordering && child->is_plan_root()) { + ObLogPlan *plan = child->get_plan(); + const ObDMLStmt *stmt = NULL; + if (OB_ISNULL(plan) || OB_ISNULL(stmt=plan->get_stmt())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpect null param", K(ret)); + } else if (0 == stmt->get_order_item_size()) { + //do nothing + } else { + used = true; + } + } + } + if (OB_FAIL(ret)) { + } else if (used || !inherit_child_ordering) { + break; + } else { + child = parent; + parent = parent->get_parent(); + } + } + return ret; +} diff --git a/src/sql/optimizer/ob_logical_operator.h b/src/sql/optimizer/ob_logical_operator.h index 37362c5f17..c6225074fa 100644 --- a/src/sql/optimizer/ob_logical_operator.h +++ b/src/sql/optimizer/ob_logical_operator.h @@ -1754,6 +1754,8 @@ protected: static int explain_print_partitions(const ObIArray &part_infos, const bool two_level, char *buf, int64_t &buf_len, int64_t &pos); + + int check_op_orderding_used_by_parent(bool &used); protected: void add_dist_flag(uint64_t &flags, DistAlgo method) const { @@ -1878,6 +1880,7 @@ private: int need_alloc_material_for_push_down_wf(ObLogicalOperator &curr_op, bool &need_alloc); int check_need_parallel_valid(int64_t need_parallel) const; virtual int get_card_without_filter(double &card); + virtual int check_use_child_ordering(bool &used, int64_t &inherit_child_ordering_index); private: ObLogicalOperator *parent_; // parent operator bool is_plan_root_; // plan root operator diff --git a/src/sql/optimizer/ob_opt_est_cost.cpp b/src/sql/optimizer/ob_opt_est_cost.cpp index 58279f3180..09878a45c4 100644 --- a/src/sql/optimizer/ob_opt_est_cost.cpp +++ b/src/sql/optimizer/ob_opt_est_cost.cpp @@ -765,4 +765,11 @@ double ObOptEstCost::calc_pred_cost_per_row(const ObRawExpr *expr, { GET_COST_MODEL(); return model->calc_pred_cost_per_row(expr, card, cost); +} + +double ObOptEstCost::cost_values_table(double rows, + ObIArray &filters, + const ObOptimizerContext &opt_ctx) +{ + return cost_get_rows(rows, opt_ctx) + cost_quals(rows, filters, opt_ctx); } \ No newline at end of file diff --git a/src/sql/optimizer/ob_opt_est_cost.h b/src/sql/optimizer/ob_opt_est_cost.h index 8c431a880c..c8e0f34bfa 100644 --- a/src/sql/optimizer/ob_opt_est_cost.h +++ b/src/sql/optimizer/ob_opt_est_cost.h @@ -246,6 +246,9 @@ public: double card, double &cost, const ObOptimizerContext &opt_ctx); + static double cost_values_table(double rows, + ObIArray &filters, + const ObOptimizerContext &opt_ctx); private: DISALLOW_COPY_AND_ASSIGN(ObOptEstCost); }; diff --git a/src/sql/optimizer/ob_opt_selectivity.cpp b/src/sql/optimizer/ob_opt_selectivity.cpp index 918d004386..7af4aaac8d 100644 --- a/src/sql/optimizer/ob_opt_selectivity.cpp +++ b/src/sql/optimizer/ob_opt_selectivity.cpp @@ -516,6 +516,47 @@ int OptTableMetas::add_generate_table_meta_info(const ObDMLStmt *parent_stmt, return ret; } +int OptTableMetas::add_values_table_meta_info(const ObDMLStmt *stmt, + const uint64_t table_id, + const OptSelectivityCtx &ctx, + ObValuesTableDef *table_def) +{ + int ret = OB_SUCCESS; + OptTableMeta *table_meta = NULL; + OptColumnMeta *column_meta = NULL; + ObSEArray column_items; + if (OB_ISNULL(stmt) || OB_ISNULL(table_def)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get null stmt", K(ret), KP(stmt), KP(table_def)); + } else if (OB_ISNULL(table_meta = table_metas_.alloc_place_holder())) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to allocate place holder for table meta", K(ret)); + } else if (OB_FAIL(stmt->get_column_items(table_id, column_items))) { + LOG_WARN("failed to get column items", K(ret)); + } else { + table_meta->set_table_id(table_id); + table_meta->set_ref_table_id(OB_INVALID_ID); + table_meta->set_rows(table_def->row_cnt_); + for (int64_t i = 0; OB_SUCC(ret) && i < column_items.count(); ++i) { + const ColumnItem &column_item = column_items.at(i); + int64_t idx = column_item.column_id_ - OB_APP_MIN_COLUMN_ID; + if (OB_UNLIKELY(idx >= table_def->column_ndvs_.count() || + idx >= table_def->column_nnvs_.count()) || + OB_ISNULL(column_meta = table_meta->get_column_metas().alloc_place_holder())) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to allocate place holder for column meta", K(ret)); + } else { + double avg_len = ObOptEstCost::get_estimate_width_from_type(column_item.expr_->get_result_type()); + column_meta->init(column_item.column_id_, + revise_ndv(table_def->column_ndvs_.at(idx)), + table_def->column_nnvs_.at(idx), + avg_len); + } + } + } + return ret; +} + int OptTableMetas::get_set_stmt_output_statistics(const ObSelectStmt &stmt, const OptTableMetas &child_table_metas, const int64_t idx, diff --git a/src/sql/optimizer/ob_opt_selectivity.h b/src/sql/optimizer/ob_opt_selectivity.h index 981b0d704b..f4af2ab53a 100644 --- a/src/sql/optimizer/ob_opt_selectivity.h +++ b/src/sql/optimizer/ob_opt_selectivity.h @@ -431,7 +431,10 @@ public: const OptTableMetas &child_table_metas, const OptSelectivityCtx &child_ctx, const double child_rows); - + int add_values_table_meta_info(const ObDMLStmt *stmt, + const uint64_t table_id, + const OptSelectivityCtx &ctx, + ObValuesTableDef *table_def); int get_set_stmt_output_statistics(const ObSelectStmt &stmt, const OptTableMetas &child_table_metas, const int64_t idx, diff --git a/src/sql/optimizer/ob_optimizer.cpp b/src/sql/optimizer/ob_optimizer.cpp index da4aa64da2..7c3f351db4 100644 --- a/src/sql/optimizer/ob_optimizer.cpp +++ b/src/sql/optimizer/ob_optimizer.cpp @@ -603,6 +603,10 @@ int ObOptimizer::extract_opt_ctx_basic_flags(const ObDMLStmt &stmt, ObSQLSession ctx_.set_merge_join_enabled(tenant_config->_optimizer_sortmerge_join_enabled); ctx_.set_nested_join_enabled(tenant_config->_nested_loop_join_enabled); } + if (!session.is_inner() && stmt.get_query_ctx()->get_injected_random_status()) { + ctx_.set_generate_random_plan(true); + } + //do nothing } return ret; } diff --git a/src/sql/optimizer/ob_optimizer_context.h b/src/sql/optimizer/ob_optimizer_context.h index 2753b32508..8e569419e3 100644 --- a/src/sql/optimizer/ob_optimizer_context.h +++ b/src/sql/optimizer/ob_optimizer_context.h @@ -233,7 +233,8 @@ ObOptimizerContext(ObSQLSessionInfo *session_info, optimizer_sortmerge_join_enabled_(true), nested_loop_join_enabled_(true), system_stat_(), - storage_estimation_enabled_(false) + storage_estimation_enabled_(false), + generate_random_plan_(false) { } inline common::ObOptStatManager *get_opt_stat_manager() { return opt_stat_manager_; } inline void set_opt_stat_manager(common::ObOptStatManager *sm) { opt_stat_manager_ = sm; } @@ -603,6 +604,8 @@ ObOptimizerContext(ObSQLSessionInfo *session_info, inline void set_nested_join_enabled(bool enabled) { nested_loop_join_enabled_ = enabled; } inline OptSystemStat& get_system_stat() { return system_stat_; } inline const OptSystemStat& get_system_stat() const { return system_stat_; } + inline bool generate_random_plan() const { return generate_random_plan_; } + inline void set_generate_random_plan(bool rand_plan) { generate_random_plan_ = rand_plan; } private: ObSQLSessionInfo *session_info_; @@ -687,6 +690,8 @@ private: bool nested_loop_join_enabled_; OptSystemStat system_stat_; bool storage_estimation_enabled_; + + bool generate_random_plan_; }; } } diff --git a/src/sql/parser/sql_parser_mysql_mode.l b/src/sql/parser/sql_parser_mysql_mode.l index 22cd79fb48..47f2a14650 100644 --- a/src/sql/parser/sql_parser_mysql_mode.l +++ b/src/sql/parser/sql_parser_mysql_mode.l @@ -1111,6 +1111,8 @@ Timestamp{whitespace}?\"[^\"]*\" { NO_SEMI_TO_INNER { return NO_SEMI_TO_INNER; } COALESCE_SQ { return COALESCE_SQ; } NO_COALESCE_SQ { return NO_COALESCE_SQ; } +WO_PULLUP { return WO_PULLUP; } +WITH_PULLUP { return WITH_PULLUP; } GBY_PUSHDOWN { return GBY_PUSHDOWN; } NO_GBY_PUSHDOWN { return NO_GBY_PUSHDOWN; } USE_HASH_DISTINCT { return USE_HASH_DISTINCT; } @@ -1160,6 +1162,8 @@ Timestamp{whitespace}?\"[^\"]*\" { NO_AGGR_FIRST_UNNEST { return NO_AGGR_FIRST_UNNEST; } JOIN_FIRST_UNNEST { return JOIN_FIRST_UNNEST; } NO_JOIN_FIRST_UNNEST { return NO_JOIN_FIRST_UNNEST; } +COALESCE_AGGR { return COALESCE_AGGR; } +NO_COALESCE_AGGR { return NO_COALESCE_AGGR; } OPT_PARAM { return OPT_PARAM; } OB_DDL_SCHEMA_VERSION { return OB_DDL_SCHEMA_VERSION; } DYNAMIC_SAMPLING { return DYNAMIC_SAMPLING; } diff --git a/src/sql/parser/sql_parser_mysql_mode.y b/src/sql/parser/sql_parser_mysql_mode.y index 2cbe37de3d..4e895142d2 100644 --- a/src/sql/parser/sql_parser_mysql_mode.y +++ b/src/sql/parser/sql_parser_mysql_mode.y @@ -177,6 +177,7 @@ PROJECT_PRUNE NO_PROJECT_PRUNE SIMPLIFY_SET NO_SIMPLIFY_SET OUTER_TO_INNER NO_OU COALESCE_SQ NO_COALESCE_SQ COUNT_TO_EXISTS NO_COUNT_TO_EXISTS LEFT_TO_ANTI NO_LEFT_TO_ANTI ELIMINATE_JOIN NO_ELIMINATE_JOIN PUSH_LIMIT NO_PUSH_LIMIT PULLUP_EXPR NO_PULLUP_EXPR WIN_MAGIC NO_WIN_MAGIC AGGR_FIRST_UNNEST NO_AGGR_FIRST_UNNEST JOIN_FIRST_UNNEST NO_JOIN_FIRST_UNNEST +COALESCE_AGGR NO_COALESCE_AGGR WITH_PULLUP WO_PULLUP MV_REWRITE NO_MV_REWRITE DECORRELATE NO_DECORRELATE // optimize hint @@ -424,11 +425,12 @@ END_P SET_VAR DELIMITER %type relation_factor_in_hint relation_factor_in_hint_list relation_factor_in_pq_hint opt_relation_factor_in_hint_list relation_factor_in_use_join_hint_list relation_factor_in_mv_hint_list opt_relation_factor_in_mv_hint_list %type relation_factor_in_leading_hint_list joined_table tbl_name table_subquery table_subquery_alias %type relation_factor_with_star relation_with_star_list opt_with_star -%type index_hint_type key_or_index index_hint_scope index_element index_list opt_index_list +%type index_hint_type key_or_index index_hint_scope index_element index_list opt_index_list opt_index_prefix %type add_key_or_index_opt add_key_or_index add_unique_key_opt add_unique_key add_constraint_uniq_key_opt add_constraint_uniq_key add_constraint_pri_key_opt add_constraint_pri_key add_primary_key_opt add_primary_key add_spatial_index_opt add_spatial_index %type index_hint_definition index_hint_list %type intnum_list %type qb_name_option qb_name_string qb_name_list multi_qb_name_list +%type coalesce_strategy_list %type join_condition inner_join_type opt_inner outer_join_type opt_outer natural_join_type except_full_outer_join_type opt_full_table_factor %type string_length_i opt_string_length_i opt_string_length_i_v2 opt_int_length_i opt_bit_length_i opt_datetime_fsp_i opt_unsigned_i opt_zerofill_i opt_year_i opt_time_func_fsp_i opt_cast_float_precision %type opt_float_precision opt_number_precision @@ -10888,6 +10890,18 @@ NO_REWRITE opt_qb_name { malloc_non_terminal_node($$, result->malloc_pool_, T_NO_DECORRELATE, 1, $2); } +| COALESCE_AGGR opt_qb_name +{ + malloc_non_terminal_node($$, result->malloc_pool_, T_COALESCE_AGGR, 1, $2); +} +| COALESCE_AGGR '(' qb_name_option coalesce_strategy_list ')' +{ + malloc_non_terminal_node($$, result->malloc_pool_, T_COALESCE_AGGR, 2, $3, $4); +} +| NO_COALESCE_AGGR opt_qb_name +{ + malloc_non_terminal_node($$, result->malloc_pool_, T_NO_COALESCE_AGGR, 1, $2); +} ; multi_qb_name_list: @@ -10919,9 +10933,9 @@ qb_name_list: ; optimize_hint: -INDEX_HINT '(' qb_name_option relation_factor_in_hint NAME_OB ')' +INDEX_HINT '(' qb_name_option relation_factor_in_hint NAME_OB opt_index_prefix ')' { - malloc_non_terminal_node($$, result->malloc_pool_, T_INDEX_HINT, 3, $3, $4, $5); + malloc_non_terminal_node($$, result->malloc_pool_, T_INDEX_HINT, 4, $3, $4, $5,$6); } | NO_INDEX_HINT '(' qb_name_option relation_factor_in_hint NAME_OB ')' { @@ -11150,6 +11164,18 @@ INDEX_HINT '(' qb_name_option relation_factor_in_hint NAME_OB ')' } ; +opt_index_prefix: +INTNUM +{ + $$=$1; +} +| NEG_SIGN INTNUM +{ + $2->value_ = -$2->value_; + $$ = $2; +} +| /*emtpy*/ {$$ = NULL;}; + win_dist_list: win_dist_desc { @@ -12845,6 +12871,32 @@ relation_factor_in_hint } ; +coalesce_strategy_list: +WO_PULLUP +{ + malloc_terminal_node($$, result->malloc_pool_, T_VARCHAR); + $$->str_value_ = "WO_PULLUP"; + $$->str_len_ = strlen("WO_PULLUP"); +} +| WITH_PULLUP +{ + malloc_terminal_node($$, result->malloc_pool_, T_VARCHAR); + $$->str_value_ = "WITH_PULLUP"; + $$->str_len_ = strlen("WITH_PULLUP"); +} +| WO_PULLUP WITH_PULLUP +{ + ParseNode *basic_node = NULL; + ParseNode *split_node = NULL; + malloc_terminal_node(basic_node, result->malloc_pool_, T_VARCHAR); + basic_node->str_value_ = "WO_PULLUP"; + basic_node->str_len_ = strlen("WO_PULLUP"); + malloc_terminal_node(split_node, result->malloc_pool_, T_VARCHAR); + split_node->str_value_ = "WITH_PULLUP"; + split_node->str_len_ = strlen("WITH_PULLUP"); + malloc_non_terminal_node($$, result->malloc_pool_, T_LINK_NODE, 2, basic_node, split_node); +} + intnum_list: INTNUM relation_sep_option intnum_list { diff --git a/src/sql/printer/ob_dml_stmt_printer.cpp b/src/sql/printer/ob_dml_stmt_printer.cpp index 52c0edc1ba..81a24df08d 100644 --- a/src/sql/printer/ob_dml_stmt_printer.cpp +++ b/src/sql/printer/ob_dml_stmt_printer.cpp @@ -287,7 +287,7 @@ int ObDMLStmtPrinter::print_table(const TableItem *table_item, } else if (is_stack_overflow) { ret = OB_SIZE_OVERFLOW; LOG_WARN("too deep recursive", K(ret), K(is_stack_overflow)); - } else if (OB_ISNULL(stmt_) || OB_ISNULL(buf_) || OB_ISNULL(pos_)) { + } else if (OB_ISNULL(stmt_) || OB_ISNULL(buf_) || OB_ISNULL(pos_) || OB_ISNULL(table_item)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("stmt_ is NULL or buf_ is NULL or pos_ is NULL", K(ret)); } else { @@ -482,30 +482,8 @@ int ObDMLStmtPrinter::print_table(const TableItem *table_item, break; } case TableItem::VALUES_TABLE: { - int64_t column_cnt = stmt_->get_column_size(table_item->table_id_); - const ObIArray &values = table_item->table_values_; - if (OB_UNLIKELY(column_cnt <= 0 || values.empty() || values.count() % column_cnt != 0)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("get unexpected error", K(ret), K(column_cnt), K(values)); - } else { - DATA_PRINTF("(VALUES "); - for (int64_t i = 0; OB_SUCC(ret) && i < values.count(); ++i) { - if (i % column_cnt == 0) { - if (i == 0) { - DATA_PRINTF("ROW("); - } else { - DATA_PRINTF("), ROW("); - } - } - if (OB_SUCC(ret)) { - OZ (expr_printer_.do_print(values.at(i), T_FROM_SCOPE)); - if (OB_SUCC(ret) && (i + 1) % column_cnt != 0) { - DATA_PRINTF(", "); - } - } - } - DATA_PRINTF("))"); - DATA_PRINTF(" %.*s", LEN_AND_PTR(table_item->alias_name_)); + if (OB_FAIL(print_values_table(*table_item, no_print_alias))) { + LOG_WARN("failed to print values table", K(ret)); } break; } @@ -520,6 +498,163 @@ int ObDMLStmtPrinter::print_table(const TableItem *table_item, return ret; } +int ObDMLStmtPrinter::print_values_table(const TableItem &table_item, bool no_print_alias) +{ + int ret = OB_SUCCESS; + ObValuesTableDef *table_def = table_item.values_table_def_; + if (print_params_.for_dblink_ || lib::is_oracle_mode()) { + if (OB_FAIL(print_values_table_to_union_all(table_item, no_print_alias))) { + LOG_WARN("failed to print values table for dblink", K(ret)); + } + } else if (OB_UNLIKELY(!table_item.is_values_table()) || OB_ISNULL(table_def)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("values table def should not be NULL", K(ret), KP(table_def)); + } else { + const int64_t column_cnt = table_def->column_cnt_; + const int64_t row_cnt = table_def->row_cnt_; + const ObIArray &values = table_def->access_exprs_; + if (OB_UNLIKELY(column_cnt <= 0 || values.empty() || values.count() % column_cnt != 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected error", K(ret), K(column_cnt), K(values)); + } else { + DATA_PRINTF("(VALUES "); + for (int64_t i = 0; OB_SUCC(ret) && i < values.count(); ++i) { + if (i % column_cnt == 0) { + if (i == 0) { + DATA_PRINTF("ROW("); // first row + } else { + DATA_PRINTF("), ROW("); // next row + } + } + if (OB_SUCC(ret)) { + OZ (expr_printer_.do_print(values.at(i), T_FROM_SCOPE)); + if (OB_SUCC(ret) && (i + 1) % column_cnt != 0) { + DATA_PRINTF(", "); + } + } + } + DATA_PRINTF("))"); + DATA_PRINTF(" %.*s", LEN_AND_PTR(table_item.alias_name_)); + } + } + return ret; +} + + +int ObDMLStmtPrinter::print_values_table_to_union_all(const TableItem &table_item, bool no_print_alias) +{ + int ret = OB_SUCCESS; + ObValuesTableDef *table_def = table_item.values_table_def_; + if (OB_UNLIKELY(!table_item.is_values_table()) || OB_ISNULL(table_def)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("values table def should not be NULL", K(ret), KP(table_def)); + } else { + int64_t column_cnt = table_def->column_cnt_; + int64_t row_cnt = table_def->row_cnt_; + if (OB_UNLIKELY(column_cnt <= 0) || OB_UNLIKELY(row_cnt <= 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected error", K(ret), K(column_cnt), K(row_cnt)); + } else if (ObValuesTableDef::ACCESS_EXPR == table_def->access_type_ || + ObValuesTableDef::FOLD_ACCESS_EXPR == table_def->access_type_) { + const ObIArray &values = table_def->access_exprs_; + if (OB_UNLIKELY(values.count() % column_cnt != 0 || values.empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("got unexpected param", K(ret)); + } + DATA_PRINTF("("); + for (int64_t i = 0; OB_SUCC(ret) && i < values.count(); ++i) { + if (i % column_cnt == 0) { + if (i == 0) { + DATA_PRINTF("SELECT "); + } else { + DATA_PRINTF(" FROM DUAL UNION ALL SELECT "); + } + } + if (OB_SUCC(ret)) { + OZ (expr_printer_.do_print(values.at(i), T_FROM_SCOPE)); + if (i < column_cnt) { + DATA_PRINTF(" AS \"column_%ld\"", i); + } + if ((i + 1) % column_cnt != 0) { + DATA_PRINTF(", "); + } + } + } + DATA_PRINTF(" FROM DUAL) \"%.*s\"", LEN_AND_PTR(table_item.alias_name_)); + } else if (ObValuesTableDef::ACCESS_PARAM == table_def->access_type_) { + const int64_t start_idx = table_def->start_param_idx_; + const int64_t end_idx = table_def->end_param_idx_; + DATA_PRINTF("("); + if (param_store_ == NULL) { + for (int64_t i = 0; OB_SUCC(ret) && i < row_cnt; i++) { + if (i == 0) { + DATA_PRINTF("SELECT "); + } else { + DATA_PRINTF(" FROM DUAL UNION ALL SELECT "); + } + for (int64_t j = 0; OB_SUCC(ret) && j < column_cnt; j++) { + OZ (ObLinkStmtParam::write(buf_, buf_len_, *pos_, start_idx + i * column_cnt + j, + table_def->column_types_.at(j).get_calc_type())); + if (i == 0) { + DATA_PRINTF(" AS \"column_%ld\"", j); + } + if (j + 1 != column_cnt) { + DATA_PRINTF(", "); + } + } + } + } else if (OB_ISNULL(param_store_) || + OB_UNLIKELY(start_idx < 0 || end_idx < start_idx) || + OB_UNLIKELY(start_idx >= param_store_->count() || end_idx >= param_store_->count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected error", K(ret), K(start_idx), K(end_idx)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < row_cnt; i++) { + if (i == 0) { + DATA_PRINTF("SELECT "); + } else { + DATA_PRINTF(" FROM DUAL UNION ALL SELECT "); + } + for (int64_t j = 0; OB_SUCC(ret) && j < column_cnt; j++) { + OZ (param_store_->at(start_idx + i * column_cnt + j).print_sql_literal(buf_, buf_len_, *pos_, print_params_)); + if (i == 0) { + DATA_PRINTF(" AS \"column_%ld\"", j); + } + if (j + 1 != column_cnt) { + DATA_PRINTF(", "); + } + } + } + } + DATA_PRINTF(" FROM DUAL) \"%.*s\"", LEN_AND_PTR(table_item.alias_name_)); + } else if (ObValuesTableDef::ACCESS_OBJ == table_def->access_type_) { + if (OB_UNLIKELY(table_def->access_objs_.count() != column_cnt * row_cnt)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("got unexpected param", K(ret)); + } + DATA_PRINTF("("); + for (int64_t i = 0; OB_SUCC(ret) && i < row_cnt; i++) { + if (i == 0) { + DATA_PRINTF("SELECT "); + } else { + DATA_PRINTF(" FROM DUAL UNION ALL SELECT "); + } + for (int64_t j = 0; OB_SUCC(ret) && j < column_cnt; j++) { + OZ (table_def->access_objs_.at(j + i * column_cnt).print_sql_literal(buf_, buf_len_, *pos_, print_params_)); + if (i == 0) { + DATA_PRINTF(" AS \"column_%ld\"", j); + } + if (j + 1 != column_cnt) { + DATA_PRINTF(", "); + } + } + } + DATA_PRINTF(" FROM DUAL) \"%.*s\"", LEN_AND_PTR(table_item.alias_name_)); + } + } + return ret; +} + int ObDMLStmtPrinter::print_json_return_type(int64_t value, ObDataType data_type) { int ret = OB_SUCCESS; diff --git a/src/sql/printer/ob_dml_stmt_printer.h b/src/sql/printer/ob_dml_stmt_printer.h index fc696a7e4f..c6ba47cfe5 100644 --- a/src/sql/printer/ob_dml_stmt_printer.h +++ b/src/sql/printer/ob_dml_stmt_printer.h @@ -110,6 +110,8 @@ public: int print_returning(); int print_json_table(const TableItem *table_item); int print_xml_table(const TableItem *table_item); + int print_values_table(const TableItem &table_item, bool no_print_alias); + int print_values_table_to_union_all(const TableItem &table_item, bool no_print_alias); int print_table(const TableItem *table_item, bool no_print_alias = false); int print_table_with_subquery(const TableItem *table_item); diff --git a/src/sql/resolver/cmd/ob_variable_set_resolver.cpp b/src/sql/resolver/cmd/ob_variable_set_resolver.cpp index da52b02500..8f892dade7 100644 --- a/src/sql/resolver/cmd/ob_variable_set_resolver.cpp +++ b/src/sql/resolver/cmd/ob_variable_set_resolver.cpp @@ -19,6 +19,7 @@ #include "sql/session/ob_sql_session_info.h" #include "sql/resolver/expr/ob_raw_expr_resolver_impl.h" #include "sql/resolver/cmd/ob_set_names_resolver.h" +#include "sql/resolver/dml/ob_inlist_resolver.h" namespace oceanbase { using namespace common; @@ -265,6 +266,7 @@ int ObVariableSetResolver::resolve_value_expr(ParseNode &val_node, ObRawExpr *&v ObArray sys_vars; ObArray op_exprs; ObSEArray user_var_exprs; + ObArray inlist_infos; ObSEArray match_exprs; ObCollationType collation_connection = CS_TYPE_INVALID; ObCharsetType character_set_connection = CHARSET_INVALID; @@ -291,11 +293,14 @@ int ObVariableSetResolver::resolve_value_expr(ParseNode &val_node, ObRawExpr *&v LOG_WARN("fail to get name case mode", K(ret)); } else if (OB_FAIL(expr_resolver.resolve(&val_node, value_expr, columns, sys_vars, sub_query_info, aggr_exprs, win_exprs, - udf_info, op_exprs, user_var_exprs, match_exprs))) { + udf_info, op_exprs, user_var_exprs, inlist_infos, match_exprs))) { LOG_WARN("resolve expr failed", K(ret)); } else if (udf_info.count() > 0) { ret = OB_ERR_UNEXPECTED; LOG_ERROR("UDFInfo should not found be here!!!", K(ret)); + } else if (inlist_infos.count() > 0) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("inlist_infos should not found be here!!!", K(ret)); } else if (OB_UNLIKELY(match_exprs.count() > 0)) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "fulltext search func"); diff --git a/src/sql/resolver/dml/ob_delete_resolver.cpp b/src/sql/resolver/dml/ob_delete_resolver.cpp index 9be93ec1ce..0defd42d38 100644 --- a/src/sql/resolver/dml/ob_delete_resolver.cpp +++ b/src/sql/resolver/dml/ob_delete_resolver.cpp @@ -109,14 +109,14 @@ int ObDeleteResolver::resolve(const ParseNode &parse_tree) } if (OB_SUCC(ret)) { - if (OB_FAIL(resolve_where_clause(parse_tree.children_[WHERE]))) { + if (OB_FAIL(resolve_hints(parse_tree.children_[HINT]))) { + LOG_WARN("resolve hints failed", K(ret)); + } else if (OB_FAIL(resolve_where_clause(parse_tree.children_[WHERE]))) { LOG_WARN("resolve delete where clause failed", K(ret)); } else if (OB_FAIL(resolve_order_clause(parse_tree.children_[ORDER_BY]))) { LOG_WARN("resolve delete order clause failed", K(ret)); } else if (OB_FAIL(resolve_limit_clause(parse_tree.children_[LIMIT], disable_limit_offset))) { LOG_WARN("resolve delete limit clause failed", K(ret)); - } else if (OB_FAIL(resolve_hints(parse_tree.children_[HINT]))) { - LOG_WARN("resolve hints failed", K(ret)); } else if (OB_FAIL(resolve_returning(parse_tree.children_[RETURNING]))) { LOG_WARN("resolve returning failed", K(ret)); } else if (is_oracle_mode() && NULL != parse_tree.children_[ERRORLOGGING] && diff --git a/src/sql/resolver/dml/ob_dml_resolver.cpp b/src/sql/resolver/dml/ob_dml_resolver.cpp index 519e63ea32..825a4bd7a9 100755 --- a/src/sql/resolver/dml/ob_dml_resolver.cpp +++ b/src/sql/resolver/dml/ob_dml_resolver.cpp @@ -62,6 +62,9 @@ #include "lib/udt/ob_udt_type.h" #include "sql/resolver/dml/ob_insert_resolver.h" #include "lib/xml/ob_path_parser.h" +#include "sql/resolver/dml/ob_inlist_resolver.h" +#include "sql/engine/aggregate/ob_aggregate_processor.h" +#include "sql/optimizer/ob_opt_selectivity.h" namespace oceanbase { @@ -1668,6 +1671,7 @@ int ObDMLResolver::resolve_sql_expr(const ParseNode &node, ObRawExpr *&expr, ObArray win_exprs; ObArray udf_info; ObArray op_exprs; + ObArray inlist_infos; ObCollationType collation_connection = CS_TYPE_INVALID; ObCharsetType character_set_connection = CHARSET_INVALID; ObSEArray match_exprs; @@ -1702,6 +1706,7 @@ int ObDMLResolver::resolve_sql_expr(const ParseNode &node, ObRawExpr *&expr, ctx.is_from_show_resolver_ = params_.is_from_show_resolver_; ctx.is_expanding_view_ = params_.is_expanding_view_; ctx.is_in_system_view_ = params_.is_in_sys_view_; + ctx.is_need_print_ = params_.is_from_create_view_ || params_.is_from_create_table_; ObRawExprResolverImpl expr_resolver(ctx); ObIArray &user_var_exprs = get_stmt()->get_user_vars(); bool is_multi_stmt = session_info_->get_cur_exec_ctx() != NULL && @@ -1719,21 +1724,23 @@ int ObDMLResolver::resolve_sql_expr(const ParseNode &node, ObRawExpr *&expr, } if (OB_SUCC(ret)) { OC( (expr_resolver.resolve)(&node, - expr, - *output_columns, - sys_vars, - sub_query_info, - aggr_exprs, - win_exprs, - udf_info, - op_exprs, - user_var_exprs, - match_exprs)); + expr, + *output_columns, + sys_vars, + sub_query_info, + aggr_exprs, + win_exprs, + udf_info, + op_exprs, + user_var_exprs, + inlist_infos, + match_exprs)); } if (OB_SUCC(ret)) { params_.prepare_param_count_ = ctx.prepare_param_count_; //prepare param count } + OZ(resolve_inlist_info(inlist_infos)); OC( (resolve_subquery_info)(sub_query_info)); if (OB_SUCC(ret)) { //are there any user variable assignments? @@ -6388,6 +6395,7 @@ int ObDMLResolver::resolve_partition_expr(const ParseNode &part_expr_node, ObRaw ObArray udf_info; ObArray op_exprs; ObSEArray user_var_exprs; + ObArray inlist_infos; ObSEArray match_exprs; ObCollationType collation_connection = CS_TYPE_INVALID; ObCharsetType character_set_connection = CHARSET_INVALID; @@ -6412,10 +6420,11 @@ int ObDMLResolver::resolve_partition_expr(const ParseNode &part_expr_node, ObRaw LOG_WARN("fail to get name case mode", K(ret)); } else if (OB_FAIL(expr_resolver.resolve(&part_expr_node, expr, columns, sys_vars, sub_query_info, aggr_exprs, win_exprs, udf_info, - op_exprs, user_var_exprs, match_exprs))) { + op_exprs, user_var_exprs, inlist_infos, match_exprs))) { LOG_WARN("resolve expr failed", K(ret)); } else if (sub_query_info.count() > 0 || sys_vars.count() > 0 || aggr_exprs.count() > 0 || - columns.count() <= 0 || udf_info.count() > 0 || op_exprs.count() > 0) { + columns.count() <= 0 || udf_info.count() > 0 || op_exprs.count() > 0 || + inlist_infos.count() > 0) { ret = OB_ERR_UNEXPECTED; //TODO Molly not allow type cast in part expr? LOG_WARN("part expr is invalid", K(sub_query_info.count()), K(sys_vars.count()), K(aggr_exprs.count()), K(columns.count()), K(udf_info.count())); @@ -7417,6 +7426,22 @@ int ObDMLResolver::check_stmt_order_by(const ObSelectStmt *stmt) return ret; } +int ObDMLResolver::resolve_inlist_info(ObIArray &inlist_infos) +{ + int ret = OB_SUCCESS; + if (inlist_infos.empty()) { + /* do nothing */ + } else { + ObInListResolver inlist_resolver(this); + for (int64_t i = 0; OB_SUCC(ret) && i < inlist_infos.count(); i++) { + if (OB_FAIL(inlist_resolver.resolve_inlist(inlist_infos.at(i)))) { + LOG_WARN("failed to resolve inlist"); + } + } + } + return ret; +} + int ObDMLResolver::resolve_subquery_info(const ObIArray &subquery_info) { int ret = OB_SUCCESS; @@ -14253,6 +14278,13 @@ int ObDMLResolver::resolve_transform_hint(const ParseNode &hint_node, } break; } + case T_COALESCE_AGGR: + case T_NO_COALESCE_AGGR: { + if (OB_FAIL(resolve_coalesce_aggr_hint(hint_node, trans_hint))) { + LOG_WARN("failed to resolve win magic hint", K(ret)); + } + break; + } case T_UNNEST: case T_NO_UNNEST: case T_PRED_DEDUCE: @@ -14476,6 +14508,7 @@ int ObDMLResolver::resolve_index_hint(const ParseNode &index_node, ObIndexHint *index_hint = NULL; ParseNode *table_node = NULL; ParseNode *index_name_node = NULL; + ParseNode *index_prefix_node = NULL; ObString qb_name; if (OB_UNLIKELY(2 > index_node.num_child_) || OB_ISNULL(table_node = index_node.children_[1])) { @@ -14493,16 +14526,31 @@ int ObDMLResolver::resolve_index_hint(const ParseNode &index_node, T_USE_COLUMN_STORE_HINT == index_hint->get_hint_type()) { index_hint->set_qb_name(qb_name); opt_hint = index_hint; - } else if (OB_UNLIKELY(3 != index_node.num_child_) || + } else if (OB_UNLIKELY(3 > index_node.num_child_) || OB_ISNULL(index_name_node = index_node.children_[2])) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected index hint", K(ret), K(index_node.type_), K(index_node.num_child_), K(index_name_node)); } else { + //T_NO_INDEX or T_INDEX_HINT index_hint->set_qb_name(qb_name); index_hint->get_index_name().assign_ptr(index_name_node->str_value_, static_cast(index_name_node->str_len_)); opt_hint = index_hint; + if (T_INDEX_HINT == index_hint->get_hint_type()) { + if (OB_UNLIKELY(4 != index_node.num_child_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected index hint", K(ret), K(index_node.type_), K(index_node.num_child_), + K(index_name_node)); + } else if (NULL == (index_prefix_node = index_node.children_[3])) { + index_hint->get_index_prefix() = -1; + } else if (OB_UNLIKELY(T_INT != index_prefix_node->type_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected index hint", K(ret), K(index_prefix_node->type_)); + } else { + index_hint->get_index_prefix() = index_prefix_node->value_; + } + } } return ret; } @@ -15165,6 +15213,43 @@ int ObDMLResolver::resolve_place_group_by_hint(const ParseNode &hint_node, return ret; } +int ObDMLResolver::resolve_coalesce_aggr_hint(const ParseNode &hint_node, + ObTransHint *&hint) +{ + int ret = OB_SUCCESS; + ObString qb_name; + ObCoalesceAggrHint *coalesce_aggr_hint = NULL; + if (OB_UNLIKELY(1 != hint_node.num_child_) && OB_UNLIKELY(2 != hint_node.num_child_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected num child", K(ret), K(hint_node.num_child_)); + } else if (OB_FAIL(ObQueryHint::create_hint(allocator_, hint_node.type_, coalesce_aggr_hint))) { + LOG_WARN("failed to create eliminate join hint", K(ret)); + } else if (OB_ISNULL(coalesce_aggr_hint)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(resolve_qb_name_node(hint_node.children_[0], qb_name))) { + LOG_WARN("failed to resolve qb name", K(ret)); + } else if (hint_node.num_child_ == 1) { + coalesce_aggr_hint->set_enable_trans_wo_pullup(true); + coalesce_aggr_hint->set_enable_trans_with_pullup(true); + } else if (hint_node.children_[1]->type_ == T_LINK_NODE && + hint_node.children_[1]->num_child_ == 2) { + coalesce_aggr_hint->set_enable_trans_wo_pullup(true); + coalesce_aggr_hint->set_enable_trans_with_pullup(true); + } else if (OB_ISNULL(hint_node.children_[1]->str_value_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (0 == STRNCASECMP(hint_node.children_[1]->str_value_,"WO_PULLUP", 9)) { + coalesce_aggr_hint->set_enable_trans_wo_pullup(true); + } else if (0 == STRNCASECMP(hint_node.children_[1]->str_value_,"WITH_PULLUP", 11)) { + coalesce_aggr_hint->set_enable_trans_with_pullup(true); + } + coalesce_aggr_hint->set_qb_name(qb_name); + hint = coalesce_aggr_hint; + LOG_DEBUG("show coalesce_aggr_hint hint", K(*coalesce_aggr_hint)); + return ret; +} + int ObDMLResolver::resolve_mv_rewrite_hint(const ParseNode &hint_node, ObTransHint *&hint) { @@ -16713,8 +16798,8 @@ int ObDMLResolver::resolve_values_table_item(const ParseNode &table_node, TableI } } if (OB_SUCC(ret)) { - int64_t column_cnt = 0; ObSEArray res_types; + void *buf = NULL; //common values table: values row(...), row(...),... /* 1.upper_insert_resolver_ != NULL && !is_mock @@ -16726,18 +16811,16 @@ int ObDMLResolver::resolve_values_table_item(const ParseNode &table_node, TableI ->insert into ....select * from (values table) as alias on ... is_mock use to resolve the generated values table */ - if ((upper_insert_resolver_ == NULL || is_mock) && - OB_FAIL(resolve_table_values_for_select(table_node, - new_table_item->table_values_, - res_types, - column_cnt))) { + if (OB_ISNULL(buf = allocator_->alloc(sizeof(ObValuesTableDef)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("faield to allocate memory table def buffer", K(ret)); + } else if (FALSE_IT(new_table_item->values_table_def_ = new (buf) ObValuesTableDef())) { + } else if ((upper_insert_resolver_ == NULL || is_mock) && + OB_FAIL(resolve_values_table_for_select(table_node, *new_table_item->values_table_def_))) { LOG_WARN("failed to resolve table values for select", K(ret)); //insert values table: insert into ....values row(...), row(...),... } else if (upper_insert_resolver_ != NULL && !is_mock && - OB_FAIL(resolve_table_values_for_insert(table_node, - new_table_item->table_values_, - res_types, - column_cnt))) { + OB_FAIL(resolve_values_table_for_insert(table_node, *new_table_item->values_table_def_))) { LOG_WARN("failed to resolve table values for insert", K(ret)); } else { new_table_item->table_id_ = generate_table_id(); @@ -16747,7 +16830,10 @@ int ObDMLResolver::resolve_values_table_item(const ParseNode &table_node, TableI new_table_item->is_view_table_ = false; if (OB_FAIL(dml_stmt->add_table_item(session_info_, new_table_item))) { LOG_WARN("add table item failed", K(ret)); - } else if (OB_FAIL(gen_values_table_column_items(column_cnt, res_types, *new_table_item))) { + } else if (OB_FAIL(gen_values_table_column_items( + new_table_item->values_table_def_->column_cnt_, + new_table_item->values_table_def_->column_types_, + *new_table_item))) { LOG_WARN("failed to gen values table column items", K(ret)); } else { table_item = new_table_item; @@ -16758,15 +16844,11 @@ int ObDMLResolver::resolve_values_table_item(const ParseNode &table_node, TableI return ret; } -int ObDMLResolver::resolve_table_values_for_select(const ParseNode &table_node, - ObIArray &table_values, - ObIArray &res_types, - int64_t &column_cnt) +int ObDMLResolver::resolve_values_table_for_select(const ParseNode &table_node, + ObValuesTableDef &table_def) { int ret = OB_SUCCESS; const ParseNode *values_node = NULL; - ObSEArray value_idxs; - column_cnt = 0; ObInsertStmt *insert_stmt = NULL; bool is_mock_for_row_alias = (upper_insert_resolver_!= NULL && upper_insert_resolver_->is_mock_for_row_alias()); @@ -16776,8 +16858,9 @@ int ObDMLResolver::resolve_table_values_for_select(const ParseNode &table_node, OB_ISNULL(params_.expr_factory_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret), K(values_node), K(table_node.type_), - K(table_node.num_child_), K(params_.expr_factory_)); + K(table_node.num_child_), KP(params_.expr_factory_)); } else { + int64_t column_cnt = 0; for (int64_t i = 0; OB_SUCC(ret) && i < values_node->num_child_; i++) { ParseNode *vector_node = values_node->children_[i]; if (OB_ISNULL(vector_node) || @@ -16863,45 +16946,47 @@ int ObDMLResolver::resolve_table_values_for_select(const ParseNode &table_node, } } if (OB_SUCC(ret)) { - if (OB_FAIL(append(table_values, cur_values_vector))) { + if (OB_FAIL(append(table_def.access_exprs_, cur_values_vector))) { LOG_WARN("failed to append", K(ret)); } else if (i == 0) { - if (OB_FAIL(append(res_types, cur_values_types))) { + if (OB_FAIL(append(table_def.column_types_, cur_values_types))) { LOG_WARN("failed to append", K(ret)); } else {/*do nothing*/} } else { - if (OB_FAIL(get_values_res_types(cur_values_types, res_types))) { + if (OB_FAIL(get_values_res_types(cur_values_types, table_def.column_types_))) { LOG_WARN("failed to get values res types", K(ret)); } } - LOG_TRACE("succeed to resolve table values", K(table_values), K(res_types), + LOG_TRACE("succeed to resolve table values", K(table_def.access_exprs_), K(table_def.column_types_), K(cur_values_vector), K(cur_values_types)); } } } } if (OB_SUCC(ret)) { - if (OB_FAIL(try_add_cast_to_values(res_types, table_values))) { + table_def.column_cnt_ = column_cnt; + if (OB_FAIL(try_add_cast_to_values(table_def.column_types_, table_def.access_exprs_))) { LOG_WARN("failed to try add cast to values", K(ret)); + } else if (OB_FAIL(compute_values_table_row_count(table_def))) { + LOG_WARN("failed to compute row"); + } else if (OB_FAIL(estimate_values_table_stats(table_def))) { + LOG_WARN("failed to estimate values table stats", K(ret)); } else { - LOG_TRACE("succeed to resolve table values", K(table_values), K(res_types)); + LOG_TRACE("succeed to resolve table values", K(table_def.access_exprs_), K(table_def.column_types_), K(table_def)); } } } return ret; } -int ObDMLResolver::resolve_table_values_for_insert(const ParseNode &table_node, - ObIArray &table_values, - ObIArray &res_types, - int64_t &column_cnt) +int ObDMLResolver::resolve_values_table_for_insert(const ParseNode &table_node, + ObValuesTableDef &table_def) { int ret = OB_SUCCESS; const ParseNode *values_node = NULL; ObSEArray value_idxs; ObInsertStmt *insert_stmt = NULL; bool is_all_default = false; - column_cnt = 0; if (OB_UNLIKELY(T_VALUES_TABLE_EXPRESSION != table_node.type_ || 1 != table_node.num_child_) || OB_ISNULL(table_node.children_) || OB_ISNULL(values_node = table_node.children_[0]) || OB_UNLIKELY(T_VALUE_LIST != values_node->type_) || @@ -16917,6 +17002,7 @@ int ObDMLResolver::resolve_table_values_for_insert(const ParseNode &table_node, LOG_WARN("failed to adjust values desc position", K(ret)); } else { bool is_all_default = false; + int64_t column_cnt = 0; for (int64_t i = 0; OB_SUCC(ret) && i < values_node->num_child_; i++) { ParseNode *vector_node = values_node->children_[i]; if (OB_ISNULL(vector_node) || @@ -17025,19 +17111,29 @@ int ObDMLResolver::resolve_table_values_for_insert(const ParseNode &table_node, } } if (OB_SUCC(ret)) { - if (OB_FAIL(append(table_values, cur_values_vector))) { + if (OB_FAIL(append(table_def.access_exprs_, cur_values_vector))) { LOG_WARN("failed to append", K(ret)); } else if (i == 0) { for (int64_t k = 0; OB_SUCC(ret) && k < cur_values_vector.count(); k++) { - if (OB_FAIL(res_types.push_back(cur_values_vector.at(k)->get_result_type()))) { + if (OB_FAIL(table_def.column_types_.push_back(cur_values_vector.at(k)->get_result_type()))) { LOG_WARN("failed to append", K(ret)); } } - LOG_TRACE("succeed to resolve one row", K(cur_values_vector), K(table_values)); + LOG_TRACE("succeed to resolve one row", K(cur_values_vector), K(table_def.access_exprs_)); } } } } + if (OB_SUCC(ret)) { + table_def.column_cnt_ = column_cnt; + if (OB_FAIL(compute_values_table_row_count(table_def))) { + LOG_WARN("failed to compute row"); + } else if (OB_FAIL(estimate_values_table_stats(table_def))) { + LOG_WARN("failed to estimate values table stats", K(ret)); + } else { + LOG_TRACE("success resolve table def", K(table_def)); + } + } } return ret; } @@ -17118,24 +17214,25 @@ int ObDMLResolver::gen_values_table_column_items(const int64_t column_cnt, TableItem &table_item) { int ret = OB_SUCCESS; + ObValuesTableDef *table_def = table_item.values_table_def_; if (OB_ISNULL(params_.expr_factory_) || OB_ISNULL(allocator_) || OB_ISNULL(get_stmt()) || - OB_UNLIKELY(column_cnt <= 0 || table_item.table_values_.empty() || - table_item.table_values_.count() % column_cnt != 0 || + OB_ISNULL(table_def) || + OB_UNLIKELY(column_cnt <= 0 || table_def->access_exprs_.empty() || + table_def->access_exprs_.count() % column_cnt != 0 || res_types.count() != column_cnt)) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("get unexpected error", K(column_cnt), K(params_.expr_factory_), - K(table_item.table_values_), K(ret)); + LOG_WARN("get unexpected error", K(column_cnt), K(params_.expr_factory_), K(ret)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < column_cnt; ++i) { ObColumnRefRawExpr *column_expr = NULL; if (OB_FAIL(params_.expr_factory_->create_raw_expr(T_REF_COLUMN, column_expr))) { LOG_WARN("create column ref raw expr failed", K(ret)); - } else if (OB_ISNULL(column_expr) || OB_ISNULL(table_item.table_values_.at(i))) { + } else if (OB_ISNULL(column_expr) || OB_ISNULL(table_def->access_exprs_.at(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN(("value desc is null")); } else { column_expr->set_result_type(res_types.at(i)); - column_expr->set_result_flag(table_item.table_values_.at(i)->get_result_flag()); + column_expr->set_result_flag(table_def->access_exprs_.at(i)->get_result_flag()); column_expr->set_ref_id(table_item.table_id_, i + OB_APP_MIN_COLUMN_ID); // compatible Mysql8.0, column name is column_0, column_1, ... ObSqlString tmp_col_name; @@ -17164,8 +17261,6 @@ int ObDMLResolver::gen_values_table_column_items(const int64_t column_cnt, column_item.column_name_ = column_expr->get_column_name(); if (OB_FAIL(get_stmt()->add_column_item(column_item))) { LOG_WARN("failed to add column item", K(ret)); - } else { - LOG_TRACE("succeed to gen table values desc", K(column_name), KPC(column_expr)); } } } @@ -17175,6 +17270,185 @@ int ObDMLResolver::gen_values_table_column_items(const int64_t column_cnt, return ret; } +int ObDMLResolver::compute_values_table_row_count(ObValuesTableDef &table_def) +{ + int ret = OB_SUCCESS; + ObRawExpr *expr1 = NULL; + ObRawExpr *expr2 = NULL; + bool is_dynamic = false; + int64_t row_cnt = 0; + int64_t expr_count = table_def.access_exprs_.count(); + if (OB_UNLIKELY(table_def.column_cnt_ == 0 || expr_count == 0) || + OB_ISNULL(expr1 = table_def.access_exprs_.at(0)) || + OB_ISNULL(expr2 = table_def.access_exprs_.at(expr_count - 1)) || + OB_ISNULL(session_info_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected NULL ptr", K(ret), KP(expr1), KP(expr2)); + } else if (T_QUESTIONMARK == expr1->get_expr_type() && expr1->is_static_scalar_const_expr() && + T_QUESTIONMARK == expr2->get_expr_type() && expr2->is_static_scalar_const_expr()) { + ObConstRawExpr *param_expr = static_cast(expr1); + int64_t param_idx = param_expr->get_value().get_unknown(); + bool is_ps_prepare = session_info_->is_varparams_sql_prepare(); + if (OB_ISNULL(params_.param_list_) || + OB_UNLIKELY(param_idx < 0 || param_idx >= params_.param_list_->count())) { + if (is_ps_prepare) { + LOG_TRACE("ps prepare param_store is empty"); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("param_idx is invalid", K(ret), K(param_idx)); + } + } else if (params_.param_list_->at(param_idx).is_ext_sql_array()) { + if (OB_FAIL(params_.param_list_->at(param_idx).get_real_param_count(row_cnt))) { + LOG_WARN("failed to get real param count", K(ret)); + } else { + is_dynamic = true; + table_def.access_type_ = ObValuesTableDef::FOLD_ACCESS_EXPR; + table_def.row_cnt_ = row_cnt; + table_def.start_param_idx_ = param_idx; + param_expr = static_cast(expr2); + table_def.end_param_idx_ = param_expr->get_value().get_unknown(); + } + } + } + if (OB_SUCC(ret) && !is_dynamic) { + table_def.row_cnt_ = expr_count / table_def.column_cnt_; + table_def.access_type_ = ObValuesTableDef::ACCESS_EXPR; + } + return ret; +} + +int ObDMLResolver::add_obj_to_llc_bitmap(const ObObj &obj, char *llc_bitmap, double &num_null) +{ + int ret = OB_SUCCESS; + uint64_t hash_value = 0; + if (OB_ISNULL(llc_bitmap)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected", K(ret), KP(llc_bitmap)); + } else if (obj.is_null()) { + num_null += 1.0; + } else { + if (obj.is_string_type()) { + hash_value = obj.varchar_hash(obj.get_collation_type(), hash_value); + } else if (OB_FAIL(obj.hash(hash_value, hash_value))) { + LOG_WARN("fail to do hash", K(ret), K(obj)); + } + if (OB_SUCC(ret)) { + if (OB_FAIL(ObAggregateProcessor::llc_add_value(hash_value, llc_bitmap, ObOptColumnStat::NUM_LLC_BUCKET))) { + LOG_WARN("fail to calc llc", K(ret)); + } + } + } + return ret; +} + +/* + row_cnt <= compute_ndv_thredhold(2000), do accurate estimate stats + row_cnt > compute_ndv_thredhold(2000), do accurate sampling 2000 rows record at head +*/ +int ObDMLResolver::estimate_values_table_stats(ObValuesTableDef &table_def) +{ + int ret = OB_SUCCESS; + const int64_t compute_ndv_thredhold = 2000; + char *llc_bitmap = NULL; + const int64_t llc_bitmap_size = ObOptColumnStat::NUM_LLC_BUCKET; + ObArenaAllocator alloc("ValuesTableStat"); + const ParamStore *param_store = params_.param_list_; + bool is_ps_prepare = false; + bool has_ps_param = false; + table_def.column_ndvs_.reset(); + table_def.column_nnvs_.reset(); + if (OB_ISNULL(session_info_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpect param", K(ret)); + } else { + is_ps_prepare = session_info_->is_varparams_sql_prepare(); + } + for (int64_t col_idx = 0; OB_SUCC(ret) && col_idx < table_def.column_cnt_; col_idx++) { + double ndv = table_def.row_cnt_; + double num_null = 0.0; + if (ObValuesTableDef::ACCESS_EXPR == table_def.access_type_) { + /* ndv = table_def.row_cnt_, num_null = 0.0 */ + } else if (OB_ISNULL(llc_bitmap = static_cast(alloc.alloc(llc_bitmap_size)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_ERROR("allocate memory for uncompressed data failed.", K(ret), K(llc_bitmap_size)); + } else { + MEMSET(llc_bitmap, 0, llc_bitmap_size); + if (ObValuesTableDef::FOLD_ACCESS_EXPR == table_def.access_type_) { + ObRawExpr *access_expr = NULL; + if (OB_UNLIKELY(col_idx >= table_def.access_exprs_.count()) || + OB_ISNULL(access_expr = table_def.access_exprs_.at(col_idx))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpect param", K(ret), K(col_idx), KP(access_expr)); + } else if (T_QUESTIONMARK == access_expr->get_expr_type()) { + ObConstRawExpr *param_expr = static_cast(access_expr); + int64_t param_idx = param_expr->get_value().get_unknown(); + if (OB_ISNULL(param_store) || + OB_UNLIKELY(param_idx < 0 || param_idx >= param_store->count())) { + if (is_ps_prepare) { + has_ps_param = true; + LOG_TRACE("ps prepare param_store is empty"); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("param_idx is invalid", K(ret), K(param_idx)); + } + } else if (param_store->at(param_idx).is_ext_sql_array()) { + const ObSqlArrayObj *array_obj = + reinterpret_cast(param_store->at(param_idx).get_ext()); + if (OB_ISNULL(array_obj)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected nullptr", K(ret), K(param_idx)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < array_obj->count_ && i < compute_ndv_thredhold; i++) { + const ObObjParam &obj_param = array_obj->data_[i]; + if (OB_FAIL(add_obj_to_llc_bitmap(obj_param, llc_bitmap, num_null))) { + LOG_WARN("failed to add obj to bitmap", K(ret)); + } + } + } + } + } + } else if (ObValuesTableDef::ACCESS_PARAM == table_def.access_type_) { + for (int64_t i = 0; OB_SUCC(ret) && i < table_def.row_cnt_ && i < compute_ndv_thredhold; i++) { + int64_t param_idx = table_def.start_param_idx_ + i * table_def.column_cnt_ + col_idx; + if (OB_UNLIKELY(param_idx < 0 || param_idx >= param_store->count())) { + if (is_ps_prepare) { + has_ps_param = true; + LOG_TRACE("ps prepare param_store is empty"); + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("param_idx is invalid", K(ret), K(param_idx)); + } + } else if (OB_FAIL(add_obj_to_llc_bitmap(param_store->at(param_idx), llc_bitmap, num_null))) { + LOG_WARN("failed to add obj to bitmap", K(ret)); + } + } + } else if (ObValuesTableDef::ACCESS_OBJ == table_def.access_type_) { + for (int64_t i = 0; OB_SUCC(ret) && i < table_def.row_cnt_ && i < compute_ndv_thredhold; i++) { + int64_t param_idx = i * table_def.column_cnt_ + col_idx; + if (OB_FAIL(add_obj_to_llc_bitmap(table_def.access_objs_.at(param_idx), llc_bitmap, num_null))) { + LOG_WARN("failed to add obj to bitmap", K(ret)); + } + } + } + if (OB_SUCC(ret) && !has_ps_param) { + ndv = MIN(ObGlobalNdvEval::get_ndv_from_llc(llc_bitmap), ndv); + ndv = table_def.row_cnt_ <= compute_ndv_thredhold ? ndv : + ObOptSelectivity::scale_distinct(table_def.row_cnt_, compute_ndv_thredhold, ndv); + } + } + if (OB_SUCC(ret)) { + if (OB_FAIL(table_def.column_ndvs_.push_back(ndv))) { + LOG_WARN("failed to push back", K(ret)); + } else if (OB_FAIL(table_def.column_nnvs_.push_back(num_null))) { + LOG_WARN("failed to push back", K(ret)); + } else { + LOG_TRACE("print stats", K(ndv), K(num_null)); + } + } + } + return ret; +} + int ObDMLResolver::refine_generate_table_column_name(const ParseNode &column_alias_node, ObSelectStmt &select_stmt) { diff --git a/src/sql/resolver/dml/ob_dml_resolver.h b/src/sql/resolver/dml/ob_dml_resolver.h index 0644ba9ed6..f63cf850b4 100644 --- a/src/sql/resolver/dml/ob_dml_resolver.h +++ b/src/sql/resolver/dml/ob_dml_resolver.h @@ -350,6 +350,7 @@ public: ObDMLStmt *get_stmt(); void set_upper_insert_resolver(ObInsertResolver *insert_resolver) { upper_insert_resolver_ = insert_resolver; } + int estimate_values_table_stats(ObValuesTableDef &table_def); protected: int generate_pl_data_type(ObRawExpr *expr, pl::ObPLDataType &pl_data_type); int resolve_into_variables(const ParseNode *node, @@ -470,6 +471,7 @@ protected: ObRawExpr *&doc_id_expr); int build_partid_expr(ObRawExpr *&expr, const uint64_t table_id); virtual int resolve_subquery_info(const common::ObIArray &subquery_info); + virtual int resolve_inlist_info(common::ObIArray &inlist_infos); virtual int resolve_aggr_exprs(ObRawExpr *&expr, common::ObIArray &aggr_exprs, const bool need_analyze = true); virtual int resolve_win_func_exprs(ObRawExpr *&expr, common::ObIArray &win_exprs); @@ -856,6 +858,9 @@ protected: const share::schema::ObTableSchema *table_schema, common::ObIArray &check_exprs, ObIArray *check_flags = NULL); + int gen_values_table_column_items(const int64_t column_cnt, + const ObIArray &res_types, + TableItem &table_item); int resolve_match_against_expr(ObMatchFunRawExpr &expr); private: int resolve_function_table_column_item_udf(const TableItem &table_item, @@ -943,6 +948,7 @@ private: int resolve_eliminate_join_hint(const ParseNode &hint_node, ObTransHint *&hint); int resolve_win_magic_hint(const ParseNode &hint_node, ObTransHint *&hint); int resolve_place_group_by_hint(const ParseNode &hint_node, ObTransHint *&hint); + int resolve_coalesce_aggr_hint(const ParseNode &hint_node, ObTransHint *&hint); int resolve_mv_rewrite_hint(const ParseNode &hint_node, ObTransHint *&hint); int resolve_tb_name_list(const ParseNode *tb_name_list_node, ObIArray> &tb_name_list); int resolve_alloc_ops(const ParseNode &alloc_op_node, ObIArray &alloc_op_hints); @@ -986,18 +992,10 @@ private: bool check_expr_has_colref(ObRawExpr *expr); int resolve_values_table_item(const ParseNode &table_node, TableItem *&table_item); - int resolve_table_values_for_select(const ParseNode &table_node, - ObIArray &table_values, - ObIArray &res_types, - int64_t &column_cnt); - int resolve_table_values_for_insert(const ParseNode &table_node, - ObIArray &table_values, - ObIArray &res_types, - int64_t &column_cnt); - - int gen_values_table_column_items(const int64_t column_cnt, - const ObIArray &res_types, - TableItem &table_item); + int resolve_values_table_for_select(const ParseNode &table_node, + ObValuesTableDef &table_values); + int resolve_values_table_for_insert(const ParseNode &table_node, + ObValuesTableDef &table_values); int get_values_res_types(const ObIArray &cur_values_types, ObIArray &res_types); int try_add_cast_to_values(const ObIArray &res_types, @@ -1023,6 +1021,8 @@ private: bool &is_true, ObIArray &constraints); int add_udt_dependency(const pl::ObUserDefinedType &udt_type); + int add_obj_to_llc_bitmap(const ObObj &obj, char *llc_bitmap, double &num_null); + int compute_values_table_row_count(ObValuesTableDef &table_def); protected: struct GenColumnExprInfo { GenColumnExprInfo(): diff --git a/src/sql/resolver/dml/ob_dml_stmt.cpp b/src/sql/resolver/dml/ob_dml_stmt.cpp index c2b6ab5826..a1669c3602 100644 --- a/src/sql/resolver/dml/ob_dml_stmt.cpp +++ b/src/sql/resolver/dml/ob_dml_stmt.cpp @@ -270,22 +270,20 @@ int TableItem::deep_copy(ObIRawExprCopier &expr_copier, ddl_schema_version_ = other.ddl_schema_version_; ddl_table_id_ = other.ddl_table_id_; ref_query_ = other.ref_query_; - if (is_json_table() && OB_FAIL(deep_copy_json_table_def(*other.json_table_def_, expr_copier, allocator))) { LOG_WARN("failed to deep copy json table define", K(ret)); - } else if (OB_FAIL(expr_copier.copy(other.flashback_query_expr_, - flashback_query_expr_))) { + } else if (OB_FAIL(expr_copier.copy(other.flashback_query_expr_, flashback_query_expr_))) { LOG_WARN("failed to deep copy raw expr", K(ret)); - } else if (OB_FAIL(expr_copier.copy(other.function_table_expr_, - function_table_expr_))) { + } else if (OB_FAIL(expr_copier.copy(other.function_table_expr_, function_table_expr_))) { LOG_WARN("failed to copy function table expr", K(ret)); } else if (OB_FAIL(part_ids_.assign(other.part_ids_))) { LOG_WARN("failed to assign part ids", K(ret)); } else if (OB_FAIL(part_names_.assign(other.part_names_))) { LOG_WARN("failed to assign part names", K(ret)); - } else if (OB_FAIL(expr_copier.copy(other.table_values_, table_values_))) { - LOG_WARN("failed to deep copy table values", K(ret)); + } else if (is_values_table() && + OB_FAIL(deep_copy_values_table_def(*other.values_table_def_, expr_copier, allocator))) { + LOG_WARN("failed to deep copy values table def", K(ret)); } else { exec_params_.reuse(); for (int64_t i = 0; OB_SUCC(ret) && i < other.exec_params_.count(); ++i) { @@ -579,9 +577,9 @@ int ObDMLStmt::deep_copy(ObStmtFactory &stmt_factory, } int deep_copy_stmt_tableItem(ObIAllocator &allocator, - ObIRawExprCopier &expr_copier, - const ObIArray &objs, - ObIArray &new_objs) + ObIRawExprCopier &expr_copier, + const ObIArray &objs, + ObIArray &new_objs) { int ret = OB_SUCCESS; for (int64_t i = 0; OB_SUCC(ret) && i < objs.count(); ++i) { @@ -888,9 +886,11 @@ int ObDMLStmt::iterate_stmt_expr(ObStmtExprVisitor &visitor) OB_FAIL(visitor.visit(table_items_.at(i)->json_table_def_->doc_expr_, SCOPE_FROM))) { LOG_WARN("failed to add json table doc expr", K(ret)); - } else if (OB_FAIL(visitor.visit(table_items_.at(i)->table_values_, - SCOPE_FROM))) { - LOG_WARN("failed to visit table values", K(ret)); + } else if (table_items_.at(i)->is_values_table() && + NULL != table_items_.at(i)->values_table_def_) { + if (OB_FAIL(visitor.visit(table_items_.at(i)->values_table_def_->access_exprs_, SCOPE_FROM))) { + LOG_WARN("failed to visit table values", K(ret)); + } } else if (table_items_.at(i)->is_lateral_table()) { TableItem *table_item = table_items_.at(i); for (int64_t j = 0; OB_SUCC(ret) && j < table_item->exec_params_.count(); ++j) { @@ -1890,6 +1890,7 @@ int ObDMLStmt::formalize_stmt_expr_reference(ObRawExprFactory *expr_factory, } else if (table_item->is_function_table() || table_item->is_json_table() || table_item->for_update_ || + table_item->is_values_table() || is_hierarchical_query()) { if (OB_FAIL(set_sharable_expr_reference(*column_item.expr_, ExplicitedRefType::REF_BY_NORMAL))) { LOG_WARN("failed to set sharable exprs reference", K(ret)); @@ -2764,6 +2765,48 @@ int ObDMLStmt::remove_table_item(const TableItem *ti) return ret; } +int ObDMLStmt::remove_table_item(const uint64_t tid, bool *remove_happened /* = NULL */) +{ + int ret = OB_SUCCESS; + bool happened = false; + for (int64_t i = 0; OB_SUCC(ret) && i < get_table_size(); ++i) { + TableItem *table_item = get_table_item(i); + if (OB_ISNULL(table_item)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("table item is NULL", K(ret)); + } else if (table_item->table_id_ != tid) { + // do nothing + } else if (OB_FAIL(table_items_.remove(i))) { + LOG_WARN("fail to remove table item", K(ret)); + } else { + happened = true; + break; + } + } + if (OB_SUCC(ret) && remove_happened != NULL) { + *remove_happened = happened; + } + return ret; +} + +int ObDMLStmt::remove_table_item(const ObIArray &tids, bool *remove_happened /* = NULL */) +{ + int ret = OB_SUCCESS; + bool happened = false; + for (int64_t i = 0; OB_SUCC(ret) && i < tids.count(); ++i) { + bool sub_hanppened = false; + if (OB_FAIL(remove_table_item(tids.at(i), &sub_hanppened))) { + LOG_WARN("fail to remove table item", K(ret), K(tids.at(i))); + } else { + happened |= sub_hanppened; + } + } + if (OB_SUCC(ret) && remove_happened != NULL) { + *remove_happened = happened; + } + return ret; +} + int ObDMLStmt::remove_table_info(const TableItem *table) { int ret = OB_SUCCESS; @@ -3189,6 +3232,79 @@ int ObDMLStmt::get_from_tables(common::ObIArray& from_tables) const return ret; } +int ObDMLStmt::get_from_tables(common::ObIArray& table_ids) const +{ + int ret = OB_SUCCESS; + for (int64_t i = 0; OB_SUCC(ret) && i < from_items_.count(); ++i) { + const FromItem &from_item = from_items_.at(i); + if (OB_FAIL(table_ids.push_back(from_item.table_id_))) { + LOG_WARN("failed to push back", K(ret)); + } + } + return ret; +} + +int ObDMLStmt::get_from_table(int64_t from_idx, TableItem* &from_table) const +{ + int ret = OB_SUCCESS; + from_table = NULL; + if (OB_UNLIKELY(from_idx < 0 || from_idx > from_items_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("from table idx invalid", K(ret), K(from_idx)); + } else { + const FromItem &from_item = from_items_.at(from_idx); + if (from_item.is_joined_ && + OB_ISNULL(from_table = get_joined_table(from_item.table_id_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (!from_item.is_joined_ && + OB_ISNULL(from_table = get_table_item_by_id(from_item.table_id_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } + } + return ret; +} + +int ObDMLStmt::get_from_item_rel_ids(int64_t from_idx, ObSqlBitSet<> &rel_ids) const +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(from_idx < 0 || from_idx > from_items_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("from table idx invalid", K(ret), K(from_idx)); + } else { + const FromItem &from_item = from_items_.at(from_idx); + TableItem *table_item = NULL; + if (from_item.is_joined_ && + OB_ISNULL(table_item = get_joined_table(from_item.table_id_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("table item is NULL", K(ret)); + } else if (!from_item.is_joined_ && + OB_ISNULL(table_item = get_table_item_by_id(from_item.table_id_))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("table item is NULL", K(ret)); + } else if (OB_FAIL(get_table_rel_ids(*table_item, rel_ids))) { + LOG_WARN("fail to get rel ids", K(ret)); + } + } + return ret; +} + +int ObDMLStmt::get_table_items(common::ObIArray& table_ids) const +{ + int ret = OB_SUCCESS; + for (int64_t i = 0; OB_SUCC(ret) && i < table_items_.count(); ++i) { + const TableItem *table_item; + if (OB_ISNULL(table_item = table_items_.at(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("table item is NULL", K(ret), K(i), K(table_items_.count())); + } else if (OB_FAIL(table_ids.push_back(table_item->table_id_))) { + LOG_WARN("failed to push back", K(ret)); + } + } + return ret; +} + ColumnItem *ObDMLStmt::get_column_item(uint64_t table_id, const ObString &col_name) { ColumnItem *item = NULL; @@ -3314,7 +3430,7 @@ int ObDMLStmt::remove_column_item(const ObIArray &column_exprs) int ObDMLStmt::remove_joined_table_item(const ObIArray &tables) { int ret = OB_SUCCESS; - for (int64_t i = 0; OB_SUCC(ret) && i < tables.count(); i++) { + for (int64_t i = 0; OB_SUCC(ret) && i < tables.count(); ++i) { ret = remove_joined_table_item(tables.at(i)); } return ret; @@ -3323,7 +3439,7 @@ int ObDMLStmt::remove_joined_table_item(const ObIArray &tables) int ObDMLStmt::remove_joined_table_item(const JoinedTable *joined_table) { int ret = OB_SUCCESS; - for (int64_t i = 0; OB_SUCC(ret) && i < joined_tables_.count(); i++) { + for (int64_t i = 0; OB_SUCC(ret) && i < joined_tables_.count(); ++i) { if (joined_table == joined_tables_.at(i)) { if (OB_FAIL(joined_tables_.remove(i))) { LOG_WARN("failed to remove joined table item", K(ret)); @@ -3334,6 +3450,26 @@ int ObDMLStmt::remove_joined_table_item(const JoinedTable *joined_table) return ret; } +int ObDMLStmt::remove_joined_table_item(uint64_t tid, bool *remove_happened/* = NULL*/) +{ + int ret = OB_SUCCESS; + if (NULL != remove_happened) { + *remove_happened = false; + } + for (int64_t i = 0; OB_SUCC(ret) && i < joined_tables_.count(); ++i) { + JoinedTable *table = joined_tables_.at(i); + if (table->table_id_ == tid) { + if (OB_FAIL(joined_tables_.remove(i))) { + LOG_WARN("failed to remove joined table item", K(ret)); + } else if (NULL != remove_happened) { + *remove_happened = true; + } + break; + } + } + return ret; +} + int ObDMLStmt::add_subquery_ref(ObQueryRefRawExpr *query_ref) { int ret = OB_SUCCESS; @@ -5174,6 +5310,60 @@ int ObJsonTableDef::assign(const ObJsonTableDef& src) return ret; } +int TableItem::deep_copy_values_table_def(const ObValuesTableDef& table_def, + ObIRawExprCopier &expr_copier, + ObIAllocator* allocator) +{ + int ret = OB_SUCCESS; + void* tmp = nullptr; + if (OB_ISNULL(allocator)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected param, invalid param.", K(ret)); + } else if (OB_ISNULL(tmp = allocator->alloc(sizeof(ObValuesTableDef)))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("allocate memory table define strunct failed.", K(ret)); + } else { + values_table_def_ = new (tmp) ObValuesTableDef(); + if (OB_FAIL(values_table_def_->deep_copy(table_def, expr_copier, allocator))) { + LOG_WARN("deep copy json table define failed.", K(ret)); + } + } + return ret; +} + +int ObValuesTableDef::deep_copy(const ObValuesTableDef &other, + ObIRawExprCopier &expr_copier, + ObIAllocator* allocator) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(allocator)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected param, invalid param.", K(ret)); + } else if (OB_FAIL(expr_copier.copy(other.access_exprs_, access_exprs_))) { + LOG_WARN("failed to copy semi condition exprs", K(ret)); + } else if (OB_FAIL(column_ndvs_.assign(other.column_ndvs_))) { + LOG_WARN("failed to assign", K(ret)); + } else if (OB_FAIL(column_nnvs_.assign(other.column_nnvs_))) { + LOG_WARN("failed to assign", K(ret)); + } else { + access_type_ = other.access_type_; + start_param_idx_ = other.start_param_idx_; + end_param_idx_ = other.end_param_idx_; + column_cnt_ = other.column_cnt_; + row_cnt_ = other.row_cnt_; + for (int64_t i = 0; OB_SUCC(ret) && i < other.access_objs_.count(); ++i) { + const ObObjParam &obj = other.access_objs_.at(i); + ObObjParam tmp_obj = obj; + if (OB_FAIL(ob_write_obj(*allocator, obj, tmp_obj))) { + LOG_WARN("failed to write obj", K(ret)); + } else if (OB_FAIL(access_objs_.push_back(tmp_obj))) { + LOG_WARN("failed to push back obj", K(ret)); + } + } + } + return ret; +} + int ObDMLStmt::get_match_expr_on_table(uint64_t table_id, ObMatchFunRawExpr *&match_expr) const { int ret = OB_SUCCESS; diff --git a/src/sql/resolver/dml/ob_dml_stmt.h b/src/sql/resolver/dml/ob_dml_stmt.h index 365990b039..51b2491e24 100644 --- a/src/sql/resolver/dml/ob_dml_stmt.h +++ b/src/sql/resolver/dml/ob_dml_stmt.h @@ -211,6 +211,32 @@ typedef struct ObJsonTableDef { common::ObSEArray namespace_arr_; } ObJsonTableDef; +struct ObValuesTableDef { + ObValuesTableDef() : start_param_idx_(-1), end_param_idx_(-1), column_cnt_(0), row_cnt_(0) , access_type_(ACCESS_EXPR) {} + enum TableAccessType { + ACCESS_EXPR = 0, // expr, one by one + FOLD_ACCESS_EXPR, // expr, one expr->ObSqlArray + ACCESS_PARAM, // by a continuous space of param_store + ACCESS_OBJ, // ObObj one by one, used for non-parametric + }; + int deep_copy(const ObValuesTableDef &other, + ObIRawExprCopier &expr_copier, + ObIAllocator* allocator); + common::ObArray access_exprs_; + int64_t start_param_idx_; + int64_t end_param_idx_; + common::ObArray access_objs_; + common::ObArray column_ndvs_; // column num distinct + common::ObArray column_nnvs_; // column num null + int64_t column_cnt_; + int64_t row_cnt_; + TableAccessType access_type_; + common::ObArray column_types_; + virtual TO_STRING_KV(K(column_cnt_), K(row_cnt_), K(access_exprs_), K(start_param_idx_), + K(end_param_idx_), K(access_objs_), K(column_ndvs_), K(column_nnvs_), + K(access_type_), K(column_types_)); +}; + struct TableItem { TableItem() @@ -240,6 +266,7 @@ struct TableItem ddl_table_id_ = common::OB_INVALID_ID; json_table_def_ = nullptr; table_type_ = MAX_TABLE_TYPE; + values_table_def_ = NULL; } virtual TO_STRING_KV(N_TID, table_id_, @@ -262,7 +289,7 @@ struct TableItem K_(is_view_table), K_(part_ids), K_(part_names), K_(cte_type), KPC_(function_table_expr), K_(flashback_query_type), KPC_(flashback_query_expr), K_(table_type), - K(table_values_), K_(exec_params), K_(mview_id), K_(need_expand_rt_mv)); + K_(exec_params), K_(mview_id), K_(need_expand_rt_mv)); enum TableType { @@ -347,7 +374,9 @@ struct TableItem ObJsonTableDef* get_json_table_def() { return json_table_def_; } int deep_copy_json_table_def(const ObJsonTableDef& jt_def, ObIRawExprCopier &expr_copier, ObIAllocator* allocator); - + int deep_copy_values_table_def(const ObValuesTableDef& table_def, + ObIRawExprCopier &expr_copier, + ObIAllocator* allocator); virtual bool has_for_update() const { return for_update_; } // if real table id, it is valid for all threads, // else if generated id, it is unique just during the thread session @@ -402,7 +431,7 @@ struct TableItem // json table ObJsonTableDef* json_table_def_; // values table - common::ObArray table_values_; + ObValuesTableDef *values_table_def_; }; struct ColumnItem @@ -747,6 +776,7 @@ public: int remove_from_item(uint64_t tid, bool *remove_happened = NULL); int remove_joined_table_item(const ObIArray &tables); int remove_joined_table_item(const JoinedTable *joined_table); + int remove_joined_table_item(uint64_t tid, bool *remove_happened = NULL); TableItem *create_table_item(common::ObIAllocator &allocator); int merge_from_items(const ObDMLStmt &stmt); @@ -914,6 +944,8 @@ public: TableItem *get_table_item(int64_t index) { return table_items_.at(index); } int remove_table_item(const TableItem *ti); int remove_table_item(const ObIArray &table_items); + int remove_table_item(const uint64_t tid, bool *remove_happened = NULL); + int remove_table_item(const ObIArray &tids, bool *remove_happened = NULL); int remove_table_info(const TableItem *table); int remove_table_info(const ObIArray &table_items); TableItem *get_table_item(const FromItem item); @@ -934,6 +966,8 @@ public: int get_from_tables(ObRelIds &table_set) const; int get_from_tables(ObSqlBitSet<> &table_set) const; int get_from_tables(common::ObIArray& from_tables) const; + int get_from_tables(common::ObIArray& table_ids) const; + int get_from_table(int64_t from_idx, TableItem* &from_table) const; int add_table_item(const ObSQLSessionInfo *session_info, TableItem *table_item); int add_table_item(const ObSQLSessionInfo *session_info, TableItem *table_item, bool &have_same_table_name); @@ -980,6 +1014,8 @@ public: int update_column_item_rel_id(); common::ObIArray &get_table_items() { return table_items_; } const common::ObIArray &get_table_items() const { return table_items_; } + int get_from_item_rel_ids(int64_t from_idx, ObSqlBitSet<> &rel_ids) const; + int get_table_items(common::ObIArray &table_ids) const; int get_CTE_table_items(ObIArray &cte_table_items) const; int get_all_CTE_table_items_recursive(ObIArray &cte_table_items) const; const common::ObIArray &get_nextval_sequence_ids() const { return nextval_sequence_ids_; } diff --git a/src/sql/resolver/dml/ob_hint.cpp b/src/sql/resolver/dml/ob_hint.cpp index 74fc36afc6..a9c4c1dd15 100644 --- a/src/sql/resolver/dml/ob_hint.cpp +++ b/src/sql/resolver/dml/ob_hint.cpp @@ -838,6 +838,10 @@ bool ObOptParamHint::is_param_val_valid(const OptParamType param_type, const ObO || 0 == val.get_varchar().case_compare("false")); break; } + case INLIST_REWRITE_THRESHOLD: { + is_valid = val.is_int() && (0 < val.get_int()); + break; + } default: LOG_TRACE("invalid opt param val", K(param_type), K(val)); break; @@ -988,6 +992,7 @@ ObItemType ObHint::get_hint_type(ObItemType type) case T_NO_AGGR_FIRST_UNNEST: return T_AGGR_FIRST_UNNEST; case T_NO_JOIN_FIRST_UNNEST: return T_JOIN_FIRST_UNNEST; case T_NO_DECORRELATE : return T_DECORRELATE; + case T_NO_COALESCE_AGGR: return T_COALESCE_AGGR; case T_MV_NO_REWRITE: return T_MV_REWRITE; // optimize hint @@ -1046,6 +1051,7 @@ const char* ObHint::get_hint_name(ObItemType type, bool is_enable_hint /* defaul case T_AGGR_FIRST_UNNEST: return is_enable_hint ? "AGGR_FIRST_UNNEST" : "NO_AGGR_FIRST_UNNEST"; case T_JOIN_FIRST_UNNEST: return is_enable_hint ? "JOIN_FIRST_UNNEST" : "NO_JOIN_FIRST_UNNEST"; case T_DECORRELATE : return is_enable_hint ? "DECORRELATE" : "NO_DECORRELATE"; + case T_COALESCE_AGGR: return is_enable_hint ? "COALESCE_AGGR" : "NO_COALESCE_AGGR"; case T_MV_REWRITE: return is_enable_hint ? "MV_REWRITE" : "NO_MV_REWRITE"; // optimize hint case T_INDEX_HINT: return "INDEX"; @@ -1154,6 +1160,7 @@ int ObHint::deep_copy_hint_contain_table(ObIAllocator *allocator, ObHint *&hint) case HINT_GROUPBY_PLACEMENT: DEEP_COPY_NORMAL_HINT(ObGroupByPlacementHint); break; case HINT_JOIN_FILTER: DEEP_COPY_NORMAL_HINT(ObJoinFilterHint); break; case HINT_WIN_MAGIC: DEEP_COPY_NORMAL_HINT(ObWinMagicHint); break; + case HINT_COALESCE_AGGR: DEEP_COPY_NORMAL_HINT(ObCoalesceAggrHint); break; default: { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected hint type to deep copy", K(ret), K(hint_class_)); @@ -1664,6 +1671,34 @@ bool ObGroupByPlacementHint::enable_groupby_placement(ObCollationType cs_type, return bret; } +int ObCoalesceAggrHint::assign(const ObCoalesceAggrHint &other) +{ + int ret = OB_SUCCESS; + if (OB_FAIL(ObTransHint::assign(other))) { + LOG_WARN("fail to assign hint", K(ret)); + } else { + enable_trans_wo_pullup_ = other.enable_trans_wo_pullup_; + enable_trans_with_pullup_ = other.enable_trans_with_pullup_; + } + return ret; +} + +int ObCoalesceAggrHint::print_hint_desc(PlanText &plan_text) const +{ + int ret = OB_SUCCESS; + char *buf = plan_text.buf_; + int64_t &buf_len = plan_text.buf_len_; + int64_t &pos = plan_text.pos_; + if (enable_trans_wo_pullup_ && enable_trans_with_pullup_ && OB_FAIL(BUF_PRINTF("WO_PULLUP WITH_PULLUP"))) { + LOG_WARN("failed to do BUF_PRINTF", K(ret)); + } else if (enable_trans_wo_pullup_ && !enable_trans_with_pullup_ && OB_FAIL(BUF_PRINTF("WO_PULLUP"))) { + LOG_WARN("failed to do BUF_PRINTF", K(ret)); + } else if (!enable_trans_wo_pullup_ && enable_trans_with_pullup_ && OB_FAIL(BUF_PRINTF("WITH_PULLUP"))) { + LOG_WARN("failed to do BUF_PRINTF", K(ret)); + } + return ret; +} + int ObWinMagicHint::assign(const ObWinMagicHint &other) { int ret = OB_SUCCESS; @@ -2008,6 +2043,7 @@ int ObIndexHint::assign(const ObIndexHint &other) { int ret = OB_SUCCESS; index_name_ = other.index_name_; + index_prefix_ = other.index_prefix_; if (OB_FAIL(table_.assign(other.table_))) { LOG_WARN("fail to assign table", K(ret)); } else if (OB_FAIL(ObOptHint::assign(other))) { @@ -2030,6 +2066,10 @@ int ObIndexHint::print_hint_desc(PlanText &plan_text) const /* do nothing */ } else if (OB_FAIL(BUF_PRINTF(" \"%.*s\"", index_name_.length(), index_name_.ptr()))) { LOG_WARN("fail to print index name", K(ret)); + } else if (T_INDEX_HINT != hint_type_ || index_prefix_ < 0) { + //do nothing + } else if (OB_FAIL(BUF_PRINTF(" %ld", index_prefix_))) { + LOG_WARN("fail to print index prefix", K(ret)); } return ret; } diff --git a/src/sql/resolver/dml/ob_hint.h b/src/sql/resolver/dml/ob_hint.h index 6e7f8c6324..e49bb61905 100644 --- a/src/sql/resolver/dml/ob_hint.h +++ b/src/sql/resolver/dml/ob_hint.h @@ -156,6 +156,7 @@ struct ObOptParamHint DEF(ENABLE_RICH_VECTOR_FORMAT,) \ DEF(_ENABLE_STORAGE_CARDINALITY_ESTIMATION,) \ DEF(PRESERVE_ORDER_FOR_PAGINATION,) \ + DEF(INLIST_REWRITE_THRESHOLD,) \ DECLARE_ENUM(OptParamType, opt_param, OPT_PARAM_TYPE_DEF, static); @@ -505,6 +506,7 @@ public: HINT_ELIMINATE_JOIN, HINT_GROUPBY_PLACEMENT, HINT_WIN_MAGIC, + HINT_COALESCE_AGGR, HINT_MV_REWRITE, // optimize hint below HINT_OPTIMIZE, // normal optimize hint @@ -587,6 +589,7 @@ public: bool is_table_dynamic_sampling_hint() const { return T_TABLE_DYNAMIC_SAMPLING == hint_type_; } bool is_pq_subquery_hint() const { return T_PQ_SUBQUERY == hint_type_; } bool is_decorrelate_hint() const { return T_DECORRELATE == hint_type_; } + bool is_coalesce_aggr_hint() const {return HINT_COALESCE_AGGR == hint_class_; } VIRTUAL_TO_STRING_KV("hint_type", get_type_name(hint_type_), K_(hint_class), K_(qb_name), @@ -778,6 +781,29 @@ public: private: common::ObSEArray table_list_; }; +class ObCoalesceAggrHint : public ObTransHint +{ +public: + ObCoalesceAggrHint(ObItemType hint_type) + : ObTransHint(hint_type), + enable_trans_wo_pullup_(false), + enable_trans_with_pullup_(false) + { + set_hint_class(HINT_COALESCE_AGGR); + } + int assign(const ObCoalesceAggrHint &other); + virtual ~ObCoalesceAggrHint() {} + virtual int print_hint_desc(PlanText &plan_text) const override; + void set_enable_trans_wo_pullup(bool flag) {enable_trans_wo_pullup_ = flag;} + void set_enable_trans_with_pullup(bool flag) {enable_trans_with_pullup_ = flag;} + bool enable_trans_wo_pullup() const {return enable_trans_wo_pullup_;}; + bool enable_trans_with_pullup() const {return enable_trans_with_pullup_;}; + INHERIT_TO_STRING_KV("ObHint", ObHint, K_(enable_trans_wo_pullup), K_(enable_trans_with_pullup)); + +private: + bool enable_trans_wo_pullup_; + bool enable_trans_with_pullup_; +}; class ObWinMagicHint : public ObTransHint { public: @@ -912,7 +938,8 @@ class ObIndexHint : public ObOptHint { public: ObIndexHint(ObItemType hint_type) - : ObOptHint(hint_type) + : ObOptHint(hint_type), + index_prefix_(-1) { set_hint_class(HINT_ACCESS_PATH); } @@ -927,14 +954,17 @@ public: const ObTableInHint &get_table() const { return table_; } ObString &get_index_name() { return index_name_; } const ObString &get_index_name() const { return index_name_; } + int64_t &get_index_prefix() { return index_prefix_; } + const int64_t &get_index_prefix() const { return index_prefix_; } bool is_use_index_hint() const { return T_NO_INDEX_HINT != get_hint_type(); } bool use_skip_scan() const { return T_INDEX_SS_HINT == get_hint_type(); } - INHERIT_TO_STRING_KV("ObHint", ObHint, K_(table), K_(index_name)); + INHERIT_TO_STRING_KV("ObHint", ObHint, K_(table), K_(index_name), K_(index_prefix)); private: ObTableInHint table_; common::ObString index_name_; + int64_t index_prefix_; }; class ObTableParallelHint : public ObOptHint diff --git a/src/sql/resolver/dml/ob_inlist_resolver.cpp b/src/sql/resolver/dml/ob_inlist_resolver.cpp new file mode 100644 index 0000000000..9580377fc5 --- /dev/null +++ b/src/sql/resolver/dml/ob_inlist_resolver.cpp @@ -0,0 +1,484 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_RESV + +#include "sql/resolver/dml/ob_inlist_resolver.h" +#include "sql/resolver/dml/ob_dml_resolver.h" +#include "sql/engine/expr/ob_expr_version.h" +#include "sql/rewrite/ob_transform_pre_process.h" + +namespace oceanbase +{ + +using namespace common; + +namespace sql +{ +int ObInListResolver::resolve_inlist(ObInListInfo &inlist_info) +{ + int ret = OB_SUCCESS; + const ParseNode *list_node = inlist_info.in_list_; + const int64_t column_cnt = inlist_info.column_cnt_; + const int64_t row_cnt = inlist_info.row_cnt_; + ObValuesTableDef *table_def = NULL; + if (OB_UNLIKELY(row_cnt <= 0 || column_cnt <= 0) || OB_ISNULL(cur_resolver_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("got NULL ptr", K(ret), KP(list_node), KP(cur_resolver_)); + } else { + ObResolverParams ¶ms = cur_resolver_->params_; + if (OB_FAIL(resolve_values_table_from_inlist(list_node, column_cnt, row_cnt, + inlist_info.is_question_mark_, params.param_list_, + params.session_info_, params.allocator_, + table_def))) { + LOG_WARN("failed to resolve values table from inlist", K(ret)); + } else if (OB_FAIL(resolve_subquery_from_values_table(params.stmt_factory_, + params.session_info_, + params.allocator_, + params.query_ctx_, + params.expr_factory_, + table_def, + params.is_prepare_protocol_ && params.is_prepare_stage_, + column_cnt, + inlist_info.in_list_expr_))) { + LOG_WARN("failed to alloc and init values stmt", K(ret)); + } + } + return ret; +} + +int ObInListResolver::resolve_values_table_from_inlist(const ParseNode *in_list, + const int64_t column_cnt, + const int64_t row_cnt, + const bool is_question_mark, + const ParamStore *param_store, + ObSQLSessionInfo *session_info, + ObIAllocator *allocator, + ObValuesTableDef *&table_def) +{ + int ret = OB_SUCCESS; + char *table_buf = NULL; + ObValuesTableDef::TableAccessType access_type = is_question_mark ? + ObValuesTableDef::ACCESS_PARAM : ObValuesTableDef::ACCESS_OBJ; + if (OB_ISNULL(allocator) || OB_ISNULL(session_info) || OB_ISNULL(in_list)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("got unexpected NULL ptr", K(ret)); + } else if (OB_UNLIKELY(in_list->num_child_ != row_cnt)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("got unexpected param", K(ret), K(row_cnt), K(in_list->num_child_)); + } else if (OB_ISNULL(table_buf = static_cast(allocator->alloc(sizeof(ObValuesTableDef))))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("sub_query or table_buf is null", K(ret), KP(table_buf)); + } else { + table_def = new (table_buf) ObValuesTableDef(); + table_def->column_cnt_ = column_cnt; + table_def->row_cnt_ = row_cnt; + table_def->access_type_ = access_type; + } + if (OB_FAIL(ret)) { + } else if (ObValuesTableDef::ACCESS_PARAM == access_type && + OB_FAIL(resolve_access_param_values_table(*in_list, column_cnt, row_cnt, param_store, + session_info, allocator, *table_def))) { + LOG_WARN("failed to resolve access param values table", K(ret)); + } else if (ObValuesTableDef::ACCESS_OBJ == access_type && + OB_FAIL(resolve_access_obj_values_table(*in_list, column_cnt, row_cnt, session_info, + allocator, *table_def))) { + LOG_WARN("failed to resolve access obj values table", K(ret)); + } else if (OB_FAIL(cur_resolver_->estimate_values_table_stats(*table_def))) { + LOG_WARN("failed to estimate values table stats", K(ret)); + } + return ret; +} + +int ObInListResolver::resolve_subquery_from_values_table(ObStmtFactory *stmt_factory, + ObSQLSessionInfo *session_info, + ObIAllocator *allocator, + ObQueryCtx *query_ctx, + ObRawExprFactory *expr_factory, + ObValuesTableDef *table_def, + const bool is_prepare_stmt, + const int64_t column_cnt, + ObQueryRefRawExpr *query_ref) +{ + int ret = OB_SUCCESS; + ObSelectStmt *subquery = NULL; + if (OB_ISNULL(allocator) || OB_ISNULL(session_info) || OB_ISNULL(stmt_factory) || + OB_ISNULL(query_ctx) || OB_ISNULL(expr_factory) || OB_ISNULL(query_ref) || + OB_ISNULL(table_def)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("got unexpected NULL ptr", K(ret)); + } else if (OB_FAIL(stmt_factory->create_stmt(subquery))) { + LOG_WARN("failed to create stmt", K(ret)); + } else if (OB_ISNULL(subquery)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("create stmt success, but stmt is null"); + } else { + subquery->set_query_ctx(query_ctx); + subquery->get_query_ctx()->set_is_prepare_stmt(is_prepare_stmt); + subquery->get_query_ctx()->set_timezone_info(get_timezone_info(session_info)); + subquery->get_query_ctx()->set_sql_stmt_coll_type(get_obj_print_params(session_info).cs_type_); + subquery->assign_distinct(); + query_ref->set_ref_stmt(subquery); + query_ref->set_is_set(true); + query_ref->set_output_column(column_cnt); + if (OB_FAIL(subquery->set_stmt_id())) { + LOG_WARN("fail to set stmt id", K(ret)); + } else if (OB_FAIL(ObResolverUtils::create_values_table_query(session_info, allocator, + expr_factory, query_ctx, subquery, + table_def))) { + LOG_WARN("failed to resolve values table query", K(ret)); + } else if (OB_FAIL(cur_resolver_->get_stmt()->add_subquery_ref(query_ref))) { + LOG_WARN("failed to add subquery reference", K(ret)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < column_cnt; i++) { + if (OB_FAIL(query_ref->add_column_type(table_def->column_types_.at(i)))) { + LOG_WARN("add column type to subquery ref expr failed", K(ret)); + } + } + } + } + return ret; +} + +int ObInListResolver::check_inlist_rewrite_enable(const ParseNode &in_list, + const ObItemType op_type, + const ObRawExpr &left_expr, + const ObStmtScope &scope, + const bool is_root_condition, + const bool is_need_print, + const bool is_prepare_protocol, + const bool is_in_pl, + const ObSQLSessionInfo *session_info, + const ParamStore *param_store, + const ObStmt *stmt, + ObIAllocator &alloc, + bool &is_question_mark, + bool &is_enable) +{ + int ret = OB_SUCCESS; + is_enable = false; + int64_t threshold = INT64_MAX; + uint64_t optimizer_features_enable_version = 0; + // 1. check basic requests + if (OB_ISNULL(session_info)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret), KP(session_info)); + } else if (T_WHERE_SCOPE != scope || !is_root_condition || T_OP_IN != op_type || + T_EXPR_LIST != in_list.type_ || is_need_print || is_prepare_protocol || + (NULL != stmt && stmt->is_select_stmt() && static_cast(stmt)->is_hierarchical_query())) { + LOG_TRACE("no need rewrite inlist", K(is_root_condition), K(scope), K(in_list.type_), + K(op_type), K(is_need_print), K(is_prepare_protocol)); + } else { + if (NULL == stmt) { + if (OB_FAIL(session_info->get_optimizer_features_enable_version(optimizer_features_enable_version))) { + LOG_WARN("failed to check ddl schema version", K(ret)); + } else { + threshold = session_info->get_inlist_rewrite_threshold(); + } + } else { + if (OB_ISNULL(stmt->get_query_ctx())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else { + threshold = session_info->get_inlist_rewrite_threshold(); + const ObGlobalHint &global_hint = stmt->get_query_ctx()->get_global_hint(); + if (OB_FAIL(global_hint.opt_params_.get_integer_opt_param( + ObOptParamHint::INLIST_REWRITE_THRESHOLD, threshold))) { + LOG_WARN("failed to get integer opt param", K(ret)); + } else if (global_hint.has_valid_opt_features_version()) { + optimizer_features_enable_version = global_hint.opt_features_version_; + } else if (OB_FAIL(session_info->get_optimizer_features_enable_version(optimizer_features_enable_version))) { + LOG_WARN("failed to check ddl schema version", K(ret)); + } + } + } + if (OB_SUCC(ret)) { + if (!ObTransformUtils::is_enable_values_table_rewrite(optimizer_features_enable_version)) { + LOG_TRACE("current optimizer version is less then COMPAT_VERSION_4_3_2"); + } else if (in_list.num_child_ < threshold) { + LOG_TRACE("check rewrite inlist threshold", K(threshold), K(in_list.num_child_)); + } else { + is_enable = true; + } + } + } + // 2. check same node type requests + if (OB_SUCC(ret) && is_enable) { + const int64_t row_cnt = in_list.num_child_; + const int64_t column_cnt = T_OP_ROW == left_expr.get_expr_type() ? left_expr.get_param_count() : + 1; + ObSEArray param_types; + ObCollationType connect_collation = CS_TYPE_INVALID; + ObCollationType nchar_collation = session_info->get_nls_collation_nation(); + int64_t server_collation = CS_TYPE_INVALID; + bool enable_decimal_int = false; + if (OB_UNLIKELY(row_cnt <= 0 || column_cnt <= 0) || OB_ISNULL(in_list.children_[0]) || + OB_UNLIKELY(column_cnt > 1 && in_list.children_[0]->num_child_ != column_cnt)) { + is_enable = false; /* delay return error code */ + } else if (OB_FAIL(session_info->get_collation_connection(connect_collation))) { + LOG_WARN("fail to get collation_connection", K(ret)); + } else if (OB_FAIL(ObSQLUtils::check_enable_decimalint(session_info, enable_decimal_int))) { + LOG_WARN("fail to check enable decimal int", K(ret)); + } else if (lib::is_oracle_mode() && OB_FAIL(session_info->get_sys_variable( + share::SYS_VAR_COLLATION_SERVER, server_collation))) { + LOG_WARN("get sys variables failed", K(ret)); + } else { + for (int64_t j = 0; OB_SUCC(ret) && is_enable && j < column_cnt; j++) { + const ParseNode *node = column_cnt == 1 ? in_list.children_[0] : + in_list.children_[0]->children_[j]; + DistinctObjMeta param_type; + // know this inlist is question mark or const + if (j == 0) { + if (OB_ISNULL(node)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected param", K(ret)); + } else { + is_question_mark = T_QUESTIONMARK == node->type_; + } + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(get_const_node_types(node, param_store, is_question_mark, + connect_collation, nchar_collation, static_cast(server_collation), enable_decimal_int, alloc, + param_type, is_enable))) { + LOG_WARN("failed to got const node types", K(ret)); + } else if (is_enable) { + if (ob_is_enum_or_set_type(param_type.obj_type_) || + is_lob_locator(param_type.obj_type_)) { + is_enable = false; + } else if (OB_FAIL(param_types.push_back(param_type))) { + LOG_WARN("failed to push back", K(ret)); + } + } + } + + for (int64_t i = 1; OB_SUCC(ret) && is_enable && i < row_cnt; i++) { + for (int64_t j = 0; OB_SUCC(ret) && is_enable && j < column_cnt; j++) { + if (OB_UNLIKELY(column_cnt > 1 && in_list.children_[i]->num_child_ != column_cnt)) { + is_enable = false; /* delay return error code */ + } else { + const ParseNode *node = column_cnt == 1 ? in_list.children_[i] : + in_list.children_[i]->children_[j]; + DistinctObjMeta param_type; + if (OB_FAIL(get_const_node_types(node, param_store, is_question_mark, + connect_collation, nchar_collation, static_cast(server_collation), enable_decimal_int, alloc, + param_type, is_enable))) { + LOG_WARN("failed to got const node types", K(ret)); + } else if (is_enable && param_type == param_types.at(j)) { + /*is same type*/ + } else { + is_enable = false; + } + } + } + } + } + } + return ret; +} + +int ObInListResolver::get_const_node_types(const ParseNode *node, + const ParamStore *param_store, + const bool is_question_mark, + const ObCollationType connect_collation, + const ObCollationType nchar_collation, + const ObCollationType server_collation, + const bool enable_decimal_int, + ObIAllocator &alloc, + DistinctObjMeta ¶m_type, + bool &is_const) +{ + int ret = OB_SUCCESS; + is_const = false; + if (OB_ISNULL(node)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid argument", K(ret)); + } else if (!(T_QUESTIONMARK == node->type_ && is_question_mark) && + !(IS_DATATYPE_OP(node->type_) && !is_question_mark)) { + /* is_const = false*/ + } else if (OB_FAIL(ObResolverUtils::fast_get_param_type(*node, param_store, connect_collation, + nchar_collation, server_collation, enable_decimal_int, alloc, + param_type.obj_type_, param_type.coll_type_, + param_type.coll_level_))) { + LOG_WARN("failed to fast get param type", K(ret)); + ret = OB_SUCCESS; + /* is_const = false*/ + } else { + is_const = true; + } + return ret; +} + +int ObInListResolver::resolve_access_param_values_table(const ParseNode &in_list, + const int64_t column_cnt, + const int64_t row_cnt, + const ParamStore *param_store, + ObSQLSessionInfo *session_info, + ObIAllocator *allocator, + ObValuesTableDef &table_def) +{ + int ret = OB_SUCCESS; + const ParseNode *row_node = NULL; + ObCollationType coll_type = CS_TYPE_INVALID; + ObLengthSemantics length_semantics = LS_DEFAULT; + if (OB_ISNULL(allocator) || OB_ISNULL(session_info) || OB_ISNULL(param_store)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("got unexpected NULL ptr", K(ret)); + } else if (OB_FAIL(session_info->get_collation_connection(coll_type))) { + LOG_WARN("fail to get collation_connection", K(ret)); + } else { + length_semantics = session_info->get_actual_nls_length_semantics(); + } + + for (int64_t i = 0; OB_SUCC(ret) && i < row_cnt; i++) { + row_node = in_list.children_[i]; + if (OB_ISNULL(row_node)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("got unexpected ptr", K(ret), KP(row_node), K(i)); + } + for (int64_t j = 0; OB_SUCC(ret) && j < column_cnt; j++) { + const ParseNode *element = column_cnt == 1 ? row_node : row_node->children_[j]; + const ObObjParam &obj_param = param_store->at(element->value_); + ObExprResType res_type; + res_type.set_meta(obj_param.get_param_meta()); + res_type.set_accuracy(obj_param.get_accuracy()); + res_type.set_result_flag(obj_param.get_result_flag()); + if (i == 0) { + if (OB_FAIL(table_def.column_types_.push_back(res_type))) { + LOG_WARN("failed to push back", K(ret)); + } + } else { + // is not same ObExprResType, than compute a new one + ObExprResType new_res_type; + ObExprVersion dummy_op(*allocator); + ObSEArray tmp_res_types; + if (OB_FAIL(tmp_res_types.push_back(table_def.column_types_.at(j)))) { + LOG_WARN("failed to push back res type", K(ret)); + } else if (OB_FAIL(tmp_res_types.push_back(res_type))) { + LOG_WARN("failed to push back res type", K(ret)); + } else if (OB_FAIL(dummy_op.aggregate_result_type_for_merge(new_res_type, + &tmp_res_types.at(0), 2, coll_type, lib::is_oracle_mode(), length_semantics, + session_info))) { + LOG_WARN("failed to aggregate result type for merge", K(ret)); + } else { + table_def.column_types_.at(j) = new_res_type; + } + } + } + } + if (OB_SUCC(ret)) { + table_def.start_param_idx_ = column_cnt == 1 ? in_list.children_[0]->value_ : + in_list.children_[0]->children_[0]->value_; + table_def.end_param_idx_ = column_cnt == 1 ? in_list.children_[row_cnt - 1]->value_ : + in_list.children_[row_cnt - 1]->children_[column_cnt - 1]->value_; + } + return ret; +} + +int ObInListResolver::resolve_access_obj_values_table(const ParseNode &in_list, + const int64_t column_cnt, + const int64_t row_cnt, + ObSQLSessionInfo *session_info, + ObIAllocator *allocator, + ObValuesTableDef &table_def) +{ + int ret = OB_SUCCESS; + const bool is_oracle_mode = lib::is_oracle_mode(); + const bool is_paramlize = false; + const ObTimeZoneInfo *timezone_info = NULL; + const bool is_from_pl = nullptr != cur_resolver_->params_.secondary_namespace_; + ObCollationType coll_type = CS_TYPE_INVALID; + ObLengthSemantics length_semantics = LS_DEFAULT; + int64_t server_collation = CS_TYPE_INVALID; + stmt::StmtType stmt_type = stmt::T_NONE; + ObExprInfo parents_expr_info; + ObString literal_prefix; + ObCollationType nchar_collation = CS_TYPE_INVALID; + const ParseNode *row_node = NULL; + bool enable_decimal_int = false; + ObCompatType compat_type = COMPAT_MYSQL57; + if (OB_ISNULL(allocator) || OB_ISNULL(session_info)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("got unexpected NULL ptr", K(ret)); + } else if (OB_FAIL(session_info->get_compatibility_control(compat_type))) { + LOG_WARN("failed to get compat type", K(ret)); + } else if (OB_FAIL(session_info->get_collation_connection(coll_type))) { + LOG_WARN("fail to get collation_connection", K(ret)); + } else if (is_oracle_mode && OB_FAIL(session_info->get_sys_variable( + share::SYS_VAR_COLLATION_SERVER, server_collation))) { + LOG_WARN("get sys variables failed", K(ret)); + } else if (OB_FAIL(ObSQLUtils::check_enable_decimalint(session_info, enable_decimal_int))) { + LOG_WARN("fail to check enable decimal int", K(ret)); + } else { + if (lib::is_oracle_mode() && cur_resolver_->params_.is_expanding_view_) { + // numeric constants should parsed with ObNumber in view expansion for oracle mode + enable_decimal_int = false; + } + length_semantics = session_info->get_actual_nls_length_semantics(); + timezone_info = session_info->get_timezone_info(); + stmt_type = is_oracle_mode ? session_info->get_stmt_type() : stmt::T_NONE; + nchar_collation = session_info->get_nls_collation_nation(); + } + + for (int64_t i = 0; OB_SUCC(ret) && i < row_cnt; i++) { + row_node = in_list.children_[i]; + if (OB_ISNULL(row_node)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("got unexpected ptr", K(ret), KP(row_node)); + } + for (int64_t j = 0; OB_SUCC(ret) && j < column_cnt; j++) { + const ParseNode *element = column_cnt == 1 ? row_node : row_node->children_[j]; + ObObjParam obj_param; + ObExprResType res_type; + if (OB_FAIL(ObResolverUtils::resolve_const(element, stmt_type, *allocator, coll_type, + nchar_collation, timezone_info, obj_param, is_paramlize, + literal_prefix, length_semantics, + static_cast(server_collation), + &parents_expr_info, + session_info->get_sql_mode(), + enable_decimal_int, + compat_type, + is_from_pl))) { + LOG_WARN("failed to resolve const", K(ret)); + } else if (OB_FAIL(table_def.access_objs_.push_back(obj_param))) { + LOG_WARN("failed to push back", K(ret)); + } else { + res_type.set_meta(obj_param.get_param_meta()); + res_type.set_accuracy(obj_param.get_accuracy()); + res_type.set_result_flag(obj_param.get_result_flag()); + if (i == 0) { + if (OB_FAIL(table_def.column_types_.push_back(res_type))) { + LOG_WARN("failed to push back", K(ret)); + } + } else { + ObExprResType new_res_type; + ObExprVersion dummy_op(*allocator); + ObSEArray tmp_res_types; + if (OB_FAIL(tmp_res_types.push_back(table_def.column_types_.at(j)))) { + LOG_WARN("failed to push back res type", K(ret)); + } else if (OB_FAIL(tmp_res_types.push_back(res_type))) { + LOG_WARN("failed to push back res type", K(ret)); + } else if (OB_FAIL(dummy_op.aggregate_result_type_for_merge(new_res_type, + &tmp_res_types.at(0), 2, coll_type, is_oracle_mode, length_semantics, + session_info))) { + LOG_WARN("failed to aggregate result type for merge", K(ret)); + } else { + table_def.column_types_.at(j) = new_res_type; + } + } + } + } + } + return ret; +} + +} // namespace sql +} // namespace oceanbase diff --git a/src/sql/resolver/dml/ob_inlist_resolver.h b/src/sql/resolver/dml/ob_inlist_resolver.h new file mode 100644 index 0000000000..d401acfaa0 --- /dev/null +++ b/src/sql/resolver/dml/ob_inlist_resolver.h @@ -0,0 +1,105 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef OCEANBASE_SRC_SQL_RESOLVER_DML_OB_INLIST_RESOLVER_H_ +#define OCEANBASE_SRC_SQL_RESOLVER_DML_OB_INLIST_RESOLVER_H_ +#include "sql/resolver/expr/ob_raw_expr.h" + +namespace oceanbase +{ +namespace sql +{ +class ObValuesTableDef; +class ObDMLResolver; +class ObQueryRefRawExpr; +struct DistinctObjMeta; + +struct ObInListInfo +{ + ObInListInfo() : in_list_(NULL), in_list_expr_(NULL), row_cnt_(0), column_cnt_(0), is_question_mark_(true) {} + TO_STRING_KV(K_(in_list_expr)); + const ParseNode *in_list_; + ObQueryRefRawExpr *in_list_expr_; + int64_t row_cnt_; + int64_t column_cnt_; + bool is_question_mark_; +}; + +class ObInListResolver +{ +public: + ObInListResolver(ObDMLResolver *cur_resolver) + : cur_resolver_(cur_resolver) {} + virtual ~ObInListResolver() {} + static int check_inlist_rewrite_enable(const ParseNode &in_list, + const ObItemType op_type, + const ObRawExpr &left_expr, + const ObStmtScope &scope, + const bool is_root_condition, + const bool is_need_print, + const bool is_prepare_protocol, + const bool is_in_pl, + const ObSQLSessionInfo *session_info, + const ParamStore *param_store, + const ObStmt *stmt, + common::ObIAllocator &alloc, + bool &is_question_mark, + bool &is_enable); + int resolve_inlist(ObInListInfo &inlist_infos); +private: + int resolve_values_table_from_inlist(const ParseNode *in_list, + const int64_t column_cnt, + const int64_t row_cnt, + const bool is_question_mark, + const ParamStore *param_store, + ObSQLSessionInfo *session_info, + ObIAllocator *allocator, + ObValuesTableDef *&table_def); + int resolve_subquery_from_values_table(ObStmtFactory *stmt_factory, + ObSQLSessionInfo *session_info, + ObIAllocator *allocator, + ObQueryCtx *query_ctx, + ObRawExprFactory *expr_factory, + ObValuesTableDef *table_def, + const bool is_prepare_stmt, + const int64_t column_cnt, + ObQueryRefRawExpr *query_ref); + static int get_const_node_types(const ParseNode *node, + const ParamStore *param_store, + const bool is_question_mark, + const ObCollationType connect_collation, + const ObCollationType nchar_collation, + const ObCollationType server_collation, + const bool enable_decimal_int, + ObIAllocator &alloc, + DistinctObjMeta ¶m_type, + bool &is_const); + int resolve_access_param_values_table(const ParseNode &in_list, + const int64_t column_cnt, + const int64_t row_cnt, + const ParamStore *param_store, + ObSQLSessionInfo *session_info, + ObIAllocator *allocator, + ObValuesTableDef &table_def); + int resolve_access_obj_values_table(const ParseNode &in_list, + const int64_t column_cnt, + const int64_t row_cnt, + ObSQLSessionInfo *session_info, + ObIAllocator *allocator, + ObValuesTableDef &table_def); +private: + ObDMLResolver *cur_resolver_; +}; + +} // namespace sql +} // namespace oceanbase +#endif /* OCEANBASE_SRC_SQL_RESOLVER_DML_OB_INLIST_RESOLVER_H_ */ diff --git a/src/sql/resolver/dml/ob_select_resolver.cpp b/src/sql/resolver/dml/ob_select_resolver.cpp index 8214ca1c57..8ba71c156e 100644 --- a/src/sql/resolver/dml/ob_select_resolver.cpp +++ b/src/sql/resolver/dml/ob_select_resolver.cpp @@ -1336,6 +1336,7 @@ int ObSelectResolver::resolve_normal_query(const ParseNode &parse_tree) if (OB_SUCC(ret) && (start_with != NULL || connect_by != NULL)) { select_stmt->set_hierarchical_query(true); } + OZ( resolve_hints(parse_tree.children_[PARSE_SELECT_HINTS]) ); /* resolve from clause */ OZ( resolve_from_clause(parse_tree.children_[PARSE_SELECT_FROM]) ); /* resolve start with clause */ @@ -1399,7 +1400,6 @@ int ObSelectResolver::resolve_normal_query(const ParseNode &parse_tree) has_top_limit_ = false; select_stmt->set_has_top_limit(NULL != parse_tree.children_[PARSE_SELECT_LIMIT]); } - OZ( resolve_hints(parse_tree.children_[PARSE_SELECT_HINTS]) ); //bug: //由于支持mysql模式下的name window,需要提前解析name window保存下来,然后再解析引用的win expr的表达式,当前实现 diff --git a/src/sql/resolver/dml/ob_sql_hint.cpp b/src/sql/resolver/dml/ob_sql_hint.cpp index 04177d5355..367db77000 100644 --- a/src/sql/resolver/dml/ob_sql_hint.cpp +++ b/src/sql/resolver/dml/ob_sql_hint.cpp @@ -1407,6 +1407,7 @@ void ObLogPlanHint::reset() table_hints_.reuse(); join_hints_.reuse(); normal_hints_.reuse(); + enable_index_prefix_ = false; } #ifndef OB_BUILD_SPM @@ -1426,6 +1427,7 @@ int ObLogPlanHint::init_log_plan_hint(ObSqlSchemaGuard &schema_guard, #ifdef OB_BUILD_SPM is_spm_evolution_ = is_spm_evolution; #endif + enable_index_prefix_ = (stmt.get_query_ctx()->optimizer_features_enable_version_ >= COMPAT_VERSION_4_2_3); const ObStmtHint &stmt_hint = stmt.get_stmt_hint(); if (OB_FAIL(join_order_.init_leading_info(stmt, query_hint, stmt_hint.get_normal_hint(T_LEADING)))) { LOG_WARN("failed to get leading hint info", K(ret)); @@ -1935,6 +1937,21 @@ DistAlgo ObLogPlanHint::get_valid_set_dist_algo(int64_t *random_none_idx /* defa return set_dist_algo; } +int ObLogPlanHint::get_index_prefix(const uint64_t table_id, + const uint64_t index_id, + int64_t &index_prefix) const +{ + int ret = OB_SUCCESS; + const LogTableHint *log_table_hint = NULL; + if (!enable_index_prefix_ || OB_ISNULL(log_table_hint = get_index_hint(table_id))) { + //do nothing + } else if (!log_table_hint->is_use_index_hint()) { + //do nothing + } else if (OB_FAIL(log_table_hint->get_index_prefix(index_id, index_prefix))) { + LOG_WARN("fail to get index prefix", K(ret)); + } + return ret; +} int ObLogPlanHint::check_valid_set_left_branch(const ObSelectStmt *select_stmt, bool &hint_valid, bool &need_swap) const @@ -2554,6 +2571,31 @@ int LogTableHint::allowed_skip_scan(const uint64_t index_id, bool &allowed) cons return ret; } +int LogTableHint::get_index_prefix(const uint64_t index_id, int64_t &index_prefix) const +{ + int ret = OB_SUCCESS; + index_prefix = -1; + if (!is_use_index_hint()) { + /* do nothing */ + } else if (OB_UNLIKELY(index_list_.count() != index_hints_.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected count", K(ret), K(index_list_.count()), K(index_hints_.count())); + } else { + bool find = false; + for (int64_t i = 0; OB_SUCC(ret) && !find && i < index_list_.count(); ++i) { + if (index_list_.at(i) != index_id) { + /* do nothing */ + } else if (OB_ISNULL(index_hints_.at(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret), K(i), K(index_hints_)); + } else if (T_INDEX_HINT == index_hints_.at(i)->get_hint_type()) { + index_prefix = index_hints_.at(i)->get_index_prefix(); + } + } + } + return ret; +} + int JoinFilterPushdownHintInfo::check_use_join_filter(const ObDMLStmt &stmt, const ObQueryHint &query_hint, uint64_t filter_table_id, diff --git a/src/sql/resolver/dml/ob_sql_hint.h b/src/sql/resolver/dml/ob_sql_hint.h index 0af19fc964..9b41aabdf8 100644 --- a/src/sql/resolver/dml/ob_sql_hint.h +++ b/src/sql/resolver/dml/ob_sql_hint.h @@ -316,6 +316,7 @@ struct LogTableHint const ObQueryHint &query_hint, const ObJoinFilterHint &hint); int allowed_skip_scan(const uint64_t index_id, bool &allowed) const; + int get_index_prefix(const uint64_t index_id, int64_t &index_prefix) const; TO_STRING_KV(K_(table), K_(index_list), K_(index_hints), K_(parallel_hint), K_(use_das_hint), @@ -473,6 +474,9 @@ struct ObLogPlanHint int get_valid_pq_subquery_hint(const ObIArray &sub_qb_names, const ObPQSubqueryHint *&explicit_hint, const ObPQSubqueryHint *&implicit_hint) const; + int get_index_prefix(const uint64_t table_id, + const uint64_t index_id, + int64_t &index_prefix) const; DistAlgo get_valid_pq_subquery_dist_algo(const ObIArray &sub_qb_names, const bool implicit_allowed) const; @@ -493,7 +497,7 @@ struct ObLogPlanHint TO_STRING_KV(K_(is_outline_data), K_(join_order), K_(table_hints), K_(join_hints), - K_(normal_hints)); + K_(normal_hints), K_(enable_index_prefix)); bool is_outline_data_; #ifdef OB_BUILD_SPM @@ -503,6 +507,7 @@ struct ObLogPlanHint common::ObSEArray table_hints_; common::ObSEArray join_hints_; common::ObSEArray normal_hints_; + bool enable_index_prefix_; }; } diff --git a/src/sql/resolver/dml/ob_update_resolver.cpp b/src/sql/resolver/dml/ob_update_resolver.cpp index 8cf568dc78..a5420f5c5f 100644 --- a/src/sql/resolver/dml/ob_update_resolver.cpp +++ b/src/sql/resolver/dml/ob_update_resolver.cpp @@ -154,6 +154,8 @@ int ObUpdateResolver::resolve(const ParseNode &parse_tree) LOG_WARN("failed to add remove const expr", K(ret)); } else if (OB_FAIL(resolve_update_constraints())) { LOG_WARN("failed to resolve check exprs", K(ret)); + } else if (OB_FAIL(resolve_hints(parse_tree.children_[HINT]))) { + LOG_WARN("resolve hints failed", K(ret)); } else if (OB_FAIL(resolve_where_clause(parse_tree.children_[WHERE]))) { LOG_WARN("resolve where clause failed", K(ret)); } else if (params_.is_batch_stmt_ && OB_FAIL(generate_batched_stmt_info())) { @@ -162,8 +164,6 @@ int ObUpdateResolver::resolve(const ParseNode &parse_tree) LOG_WARN("resolve order clause failed", K(ret)); } else if (OB_FAIL(resolve_limit_clause(parse_tree.children_[LIMIT], disable_limit_offset))) { LOG_WARN("resolve limit clause failed", K(ret)); - } else if (OB_FAIL(resolve_hints(parse_tree.children_[HINT]))) { - LOG_WARN("resolve hints failed", K(ret)); } else if (OB_FAIL(resolve_returning(parse_tree.children_[RETURNING]))) { LOG_WARN("resolve returning failed", K(ret)); } else if (OB_FAIL(try_expand_returning_exprs())) { diff --git a/src/sql/resolver/expr/ob_raw_expr.cpp b/src/sql/resolver/expr/ob_raw_expr.cpp index f6d3ac54ac..8cce4d726e 100644 --- a/src/sql/resolver/expr/ob_raw_expr.cpp +++ b/src/sql/resolver/expr/ob_raw_expr.cpp @@ -1993,6 +1993,8 @@ int ObColumnRefRawExpr::assign(const ObRawExpr &other) is_unique_key_column_ = tmp.is_unique_key_column_; is_mul_key_column_ = tmp.is_mul_key_column_; is_strict_json_column_ = tmp.is_strict_json_column_; + srs_id_ = tmp.srs_id_; + udt_set_id_ = tmp.udt_set_id_; } } return ret; diff --git a/src/sql/resolver/expr/ob_raw_expr.h b/src/sql/resolver/expr/ob_raw_expr.h index 63d77e81d7..397acf4233 100644 --- a/src/sql/resolver/expr/ob_raw_expr.h +++ b/src/sql/resolver/expr/ob_raw_expr.h @@ -68,6 +68,7 @@ class ObIRawExprCopier; class ObSelectStmt; class ObRTDatumArith; class ObLogicalOperator; +class ObInListInfo; extern ObRawExpr *USELESS_POINTER; // If is_stack_overflow is true, the printing will not continue @@ -1605,6 +1606,7 @@ class ObPseudoColumnRawExpr; class ObOpRawExpr; class ObWinFunRawExpr; class ObUserVarIdentRawExpr; +class ObDMLResolver; struct ObUDFInfo; class ObMatchFunRawExpr; template @@ -1633,6 +1635,7 @@ struct ObResolveContext udf_info_(NULL), op_exprs_(NULL), user_var_exprs_(nullptr), + inlist_infos_(NULL), is_extract_param_type_(true), param_list_(NULL), prepare_param_count_(0), @@ -1650,9 +1653,10 @@ struct ObResolveContext view_ref_id_(OB_INVALID_ID), is_variable_allowed_(true), is_expanding_view_(false), + is_need_print_(false), + is_from_show_resolver_(false), is_in_system_view_(false), - match_exprs_(NULL), - is_from_show_resolver_(false) + match_exprs_(NULL) { } @@ -1673,6 +1677,7 @@ struct ObResolveContext common::ObIArray *udf_info_; common::ObIArray *op_exprs_; common::ObIArray *user_var_exprs_; + common::ObIArray *inlist_infos_; //由于单测expr resolver中包含一些带?的表达式case, //所以为expr resolver ctx增添一个配置变量isextract_param_type //如果配置该参数为true,那么遇到?将为其填上真实的参数类型, @@ -1699,9 +1704,10 @@ struct ObResolveContext uint64_t view_ref_id_; bool is_variable_allowed_; bool is_expanding_view_; + bool is_need_print_; + bool is_from_show_resolver_; bool is_in_system_view_; common::ObIArray *match_exprs_; - bool is_from_show_resolver_; }; typedef ObResolveContext ObExprResolveContext; @@ -2077,7 +2083,7 @@ inline void ObRawExpr::unset_result_flag(uint32_t result_flag) inline int ObRawExpr::add_relation_id(int64_t rel_idx) { int ret = common::OB_SUCCESS; - if (rel_idx < 0) { + if (OB_UNLIKELY(rel_idx < 0)) { ret = common::OB_INVALID_ARGUMENT; } else { ret = rel_ids_.add_member(rel_idx); @@ -4720,8 +4726,8 @@ public: bool is_cte_query_type() const { return T_CTE_SEARCH_COLUMN == type_ || T_CTE_CYCLE_COLUMN == type_; } void set_cte_cycle_value(ObRawExpr *v, ObRawExpr *d_v) {cte_cycle_value_ = v; cte_cycle_default_value_ = d_v; }; void get_cte_cycle_value(ObRawExpr *&v, ObRawExpr *&d_v) {v = cte_cycle_value_; d_v = cte_cycle_default_value_; }; - void set_table_id(int64_t table_id) { table_id_ = table_id; } - int64_t get_table_id() const { return table_id_; } + void set_table_id(uint64_t table_id) { table_id_ = table_id; } + uint64_t get_table_id() const { return table_id_; } void set_table_name(const common::ObString &table_name) { table_name_ = table_name; } const common::ObString & get_table_name() const { return table_name_; } @@ -4734,7 +4740,7 @@ public: private: ObRawExpr *cte_cycle_value_; ObRawExpr *cte_cycle_default_value_; - int64_t table_id_; + uint64_t table_id_; common::ObString table_name_; DISALLOW_COPY_AND_ASSIGN(ObPseudoColumnRawExpr); }; diff --git a/src/sql/resolver/expr/ob_raw_expr_resolver.h b/src/sql/resolver/expr/ob_raw_expr_resolver.h index 1763876d5c..29d428b483 100644 --- a/src/sql/resolver/expr/ob_raw_expr_resolver.h +++ b/src/sql/resolver/expr/ob_raw_expr_resolver.h @@ -38,6 +38,7 @@ public: common::ObIArray &udf_exprs, common::ObIArray &op_exprs, common::ObIArray &user_var_exprs, + common::ObIArray &inlist_infos, common::ObIArray &match_exprs) = 0; private: // disallow copy diff --git a/src/sql/resolver/expr/ob_raw_expr_resolver_impl.cpp b/src/sql/resolver/expr/ob_raw_expr_resolver_impl.cpp index e370607a6a..300fdcdabe 100644 --- a/src/sql/resolver/expr/ob_raw_expr_resolver_impl.cpp +++ b/src/sql/resolver/expr/ob_raw_expr_resolver_impl.cpp @@ -35,6 +35,7 @@ #include "sql/resolver/dml/ob_del_upd_stmt.h" #include "deps/oblib/src/lib/json_type/ob_json_path.h" #include "share/resource_manager/ob_resource_manager.h" +#include "sql/resolver/dml/ob_inlist_resolver.h" namespace oceanbase { @@ -57,6 +58,7 @@ int ObRawExprResolverImpl::resolve(const ParseNode *node, ObIArray &udf_info, ObIArray &op_exprs, ObIArray &user_var_exprs, + ObIArray &inlist_infos, ObIArray &match_exprs) { ctx_.columns_ = &columns; @@ -67,8 +69,9 @@ int ObRawExprResolverImpl::resolve(const ParseNode *node, ctx_.win_exprs_ = &win_exprs; ctx_.udf_info_ = &udf_info; ctx_.user_var_exprs_ = &user_var_exprs; + ctx_.inlist_infos_ = &inlist_infos; ctx_.match_exprs_ = &match_exprs; - int ret = recursive_resolve(node, expr); + int ret = recursive_resolve(node, expr, true); if (OB_SUCC(ret)) { if (OB_FAIL(expr->extract_info())) { LOG_WARN("failed to extract info", K(ret), K(*expr)); @@ -252,12 +255,14 @@ int ObRawExprResolverImpl::try_negate_const(ObRawExpr *&expr, return ret; } -int ObRawExprResolverImpl::recursive_resolve(const ParseNode *node, ObRawExpr *&expr) +int ObRawExprResolverImpl::recursive_resolve(const ParseNode *node, ObRawExpr *&expr, bool is_root_expr) { - return SMART_CALL(do_recursive_resolve(node, expr)); + return SMART_CALL(do_recursive_resolve(node, expr, is_root_expr)); } -int ObRawExprResolverImpl::do_recursive_resolve(const ParseNode *node, ObRawExpr *&expr) +int ObRawExprResolverImpl::do_recursive_resolve(const ParseNode *node, + ObRawExpr *&expr, + bool is_root_expr) { int ret = OB_SUCCESS; bool is_stack_overflow = false; @@ -553,7 +558,8 @@ int ObRawExprResolverImpl::do_recursive_resolve(const ParseNode *node, ObRawExpr case T_OP_XOR: { ObOpRawExpr *m_expr = NULL; int64_t num_child = 2; - if (OB_FAIL(process_node_with_children(node, num_child, m_expr))) { + if (OB_FAIL(process_node_with_children(node, num_child, m_expr, + node->type_ == T_OP_XOR ? false : is_root_expr))) { LOG_WARN("fail to process node with children", K(ret), K(node)); } else if (OB_ISNULL(ctx_.session_info_)) { ret = OB_ERR_UNEXPECTED; @@ -660,7 +666,7 @@ int ObRawExprResolverImpl::do_recursive_resolve(const ParseNode *node, ObRawExpr case T_OP_IN: // get through case T_OP_NOT_IN: { - if (OB_FAIL(process_in_or_not_in_node(node, expr))) { + if (OB_FAIL(process_in_or_not_in_node(node, is_root_expr, expr))) { LOG_WARN("fail to process any or all node", K(ret), K(node)); } break; @@ -4240,7 +4246,9 @@ int ObRawExprResolverImpl::process_like_node(const ParseNode *node, ObRawExpr *& return ret; } -int ObRawExprResolverImpl::process_in_or_not_in_node(const ParseNode *node, ObRawExpr *&expr) +int ObRawExprResolverImpl::process_in_or_not_in_node(const ParseNode *node, + const bool is_root_expr, + ObRawExpr *&expr) { int ret = OB_SUCCESS; ObOpRawExpr *in_expr = NULL; @@ -4258,8 +4266,9 @@ int ObRawExprResolverImpl::process_in_or_not_in_node(const ParseNode *node, ObRa LOG_WARN("create ObOpRawExpr failed", K(ret)); } else if (OB_FAIL(SMART_CALL(recursive_resolve(node->children_[0], sub_expr1)))) { LOG_WARN("resolve left raw expr failed", K(ret)); - } else if (OB_FAIL(SMART_CALL(recursive_resolve(node->children_[1], sub_expr2)))) { - LOG_WARN("resolve right child failed", K(ret)); + } else if (OB_FAIL(resolve_right_branch_of_in_op(node->children_[1], node->type_, sub_expr1, + is_root_expr, sub_expr2))) { + LOG_WARN("failed to convert", K(ret)); } else if (OB_ISNULL(sub_expr1) || OB_ISNULL(sub_expr2)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("resolve get invalid expr", K(ret), K(sub_expr1), K(sub_expr2)); @@ -4326,6 +4335,60 @@ int ObRawExprResolverImpl::process_in_or_not_in_node(const ParseNode *node, ObRa return ret; } +int ObRawExprResolverImpl::resolve_right_branch_of_in_op(const ParseNode *node, + const ObItemType op_type, + const ObRawExpr *left_expr, + const bool is_root_condition, + ObRawExpr *&right_expr) +{ + int ret = OB_SUCCESS; + bool is_enable_rewrite = false; + bool is_question_mark = true; + if (OB_ISNULL(node)|| OB_ISNULL(left_expr)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid argument", K(ret)); + } else if (OB_FAIL(ObInListResolver::check_inlist_rewrite_enable(*node, + op_type, + *left_expr, + ctx_.current_scope_, + is_root_condition, + ctx_.is_need_print_, + !ctx_.is_extract_param_type_, /* equal to is_prepare_protocol*/ + NULL != ctx_.secondary_namespace_ || ctx_.is_for_dynamic_sql_ || ctx_.is_for_dbms_sql_, + ctx_.session_info_, + ctx_.param_list_, + ctx_.stmt_, + ctx_.expr_factory_.get_allocator(), + is_question_mark, + is_enable_rewrite))) { + LOG_WARN("failed to check inlist rewrite enable", K(ret)); + } else if (is_enable_rewrite) { + ObQueryRefRawExpr *sub_query_expr = NULL; + if (OB_FAIL(ctx_.expr_factory_.create_raw_expr(T_REF_QUERY, sub_query_expr))) { + LOG_WARN("create ObOpRawExpr failed", K(ret)); + } else if (OB_ISNULL(sub_query_expr) || OB_ISNULL(ctx_.inlist_infos_) ) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid argument", K(ret), KP(sub_query_expr), KP(ctx_.session_info_), K(ret)); + } else { + ObInListInfo inlist_info; + inlist_info.in_list_expr_ = sub_query_expr; + inlist_info.in_list_ = node; + inlist_info.column_cnt_ = left_expr->get_expr_type() == T_OP_ROW ? left_expr->get_param_count() : 1; + inlist_info.row_cnt_ = node->num_child_; + inlist_info.is_question_mark_ = is_question_mark; + if (OB_FAIL(ctx_.inlist_infos_->push_back(inlist_info))) { + LOG_WARN("failed to push back", K(ret)); + } else { + right_expr = sub_query_expr; + } + } + /* do as normal process */ + } else if (OB_FAIL(SMART_CALL(recursive_resolve(node, right_expr)))) { + LOG_WARN("resolve left raw expr failed", K(ret)); + } else {/* do nothing */} + return ret; +} + int ObRawExprResolverImpl::process_case_node(const ParseNode *node, ObRawExpr *&expr) { int ret = OB_SUCCESS; diff --git a/src/sql/resolver/expr/ob_raw_expr_resolver_impl.h b/src/sql/resolver/expr/ob_raw_expr_resolver_impl.h index 3f2b49bd45..0d0e8ff45a 100644 --- a/src/sql/resolver/expr/ob_raw_expr_resolver_impl.h +++ b/src/sql/resolver/expr/ob_raw_expr_resolver_impl.h @@ -36,6 +36,7 @@ public: common::ObIArray &udf_exprs, common::ObIArray &op_exprs, common::ObIArray &user_var_exprs, + common::ObIArray &inlist_infos, common::ObIArray &match_exprs); bool is_contains_assignment() {return is_contains_assignment_;} @@ -50,16 +51,15 @@ public: pl::ObPLBlockNS *secondary_namespace, pl::ObProcType &proc_type); int resolve_func_node_of_obj_access_idents(const ParseNode &func_node, ObQualifiedName &q_name); - int check_name_type( - ObQualifiedName &q_name, ObStmtScope scope, AccessNameType &type); + int check_name_type(ObQualifiedName &q_name, ObStmtScope scope, AccessNameType &type); // types and constants + int recursive_resolve(const ParseNode *node, ObRawExpr *&expr, bool is_root_expr = false); private: // disallow copy DISALLOW_COPY_AND_ASSIGN(ObRawExprResolverImpl); // function members int try_negate_const(ObRawExpr *&expr, const int64_t neg_cnt, int64_t &remain_reg_cnt); - int do_recursive_resolve(const ParseNode *node, ObRawExpr *&expr); - int recursive_resolve(const ParseNode *node, ObRawExpr *&expr); + int do_recursive_resolve(const ParseNode *node, ObRawExpr *&expr, bool is_root_expr = false); int process_datatype_or_questionmark(const ParseNode &node, ObRawExpr *&expr); int process_system_variable_node(const ParseNode *node, ObRawExpr *&expr); int process_char_charset_node(const ParseNode *node, ObRawExpr *&expr); @@ -68,7 +68,7 @@ private: int process_outer_join_symbol_node(const ParseNode *node, ObRawExpr *&expr); int process_column_ref_node(const ParseNode *node, ObRawExpr *&expr); template - int process_node_with_children(const ParseNode *node, int64_t children_num, T *&raw_expr); + int process_node_with_children(const ParseNode *node, int64_t children_num, T *&raw_expr, bool is_root_expr = false); int process_any_or_all_node(const ParseNode *node, ObRawExpr *&expr); int process_not_node(const ParseNode *node, ObRawExpr *&expr); int process_user_var_node(const ParseNode *node, ObRawExpr *&expr); @@ -80,7 +80,7 @@ private: int add_params_to_op_expr(ObRawExpr *op_param_1, ObRawExpr *op_param_2, ObOpRawExpr *op_expr); int transform_between_expr(ObRawExpr **btw_params, ObRawExpr *&out_expr, const bool is_not_btw); int process_between_node(const ParseNode *node, ObRawExpr *&expr); - int process_in_or_not_in_node(const ParseNode *node, ObRawExpr *&expr); + int process_in_or_not_in_node(const ParseNode *node, const bool is_root_expr, ObRawExpr *&expr); int process_case_node(const ParseNode *node, ObRawExpr *&expr); int process_sub_query_node(const ParseNode *node, ObRawExpr *&expr); int process_agg_node(const ParseNode *node, ObRawExpr *&expr); @@ -230,7 +230,11 @@ private: int resolve_left_node_of_obj_access_idents(const ParseNode &node, ObQualifiedName &q_name); int resolve_right_node_of_obj_access_idents(const ParseNode &node, ObQualifiedName &q_name); - + int resolve_right_branch_of_in_op(const ParseNode *node, + const ObItemType op_type, + const ObRawExpr *left_expr, + const bool is_root_condition, + ObRawExpr *&right_expr); private: // data members ObExprResolveContext &ctx_; @@ -240,7 +244,8 @@ private: template int ObRawExprResolverImpl::process_node_with_children(const ParseNode *node, int64_t children_num, - T *&raw_expr) + T *&raw_expr, + bool is_root_expr) { int ret = common::OB_SUCCESS; if (OB_ISNULL(node) || OB_ISNULL(node->children_) @@ -255,7 +260,7 @@ int ObRawExprResolverImpl::process_node_with_children(const ParseNode *node, if (OB_ISNULL(node->children_[i])) { ret = common::OB_ERR_UNEXPECTED; SQL_RESV_LOG(ERROR, "invalid node children", K(ret), K(i), K(node)); - } else if (OB_FAIL(recursive_resolve(node->children_[i], sub_expr))) { + } else if (OB_FAIL(recursive_resolve(node->children_[i], sub_expr, is_root_expr))) { SQL_RESV_LOG(WARN, "resolve left child failed", K(ret)); } else if (OB_FAIL(raw_expr->add_param_expr(sub_expr))) { SQL_RESV_LOG(WARN, "fail to set param expr", K(ret), K(sub_expr)); diff --git a/src/sql/resolver/expr/ob_raw_expr_util.cpp b/src/sql/resolver/expr/ob_raw_expr_util.cpp index 27ef724b2e..502e67f584 100644 --- a/src/sql/resolver/expr/ob_raw_expr_util.cpp +++ b/src/sql/resolver/expr/ob_raw_expr_util.cpp @@ -1443,14 +1443,18 @@ int ObRawExprUtils::make_raw_expr_from_str(const char *expr_str, if (OB_SUCC(ret)) { ObArray op_exprs; ObSEArray user_var_exprs; + ObArray inlist_infos; ObSEArray match_exprs; ObRawExprResolverImpl expr_resolver(resolve_ctx); // generate raw expr if (OB_FAIL(expr_resolver.resolve(parsed_expr, expr, columns, sys_vars, *sub_query_info, aggr_exprs, win_exprs, - udf_info, op_exprs, user_var_exprs, match_exprs))) { + udf_info, op_exprs, user_var_exprs, inlist_infos, match_exprs))) { _LOG_WARN("failed to resolve expr tree, err=%d", ret); - } + } else if (OB_UNLIKELY(!inlist_infos.empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("in expr is not supported here", K(parsed_expr->type_)); + } else {/* do nothing */} } // destroy syntax tree parser.free_result(parse_result); @@ -3073,13 +3077,14 @@ int ObRawExprUtils::build_raw_expr(ObRawExprFactory &expr_factory, ctx.secondary_namespace_ = ns; ctx.tg_timing_event_ = tg_timing_event; ObSEArray user_var_exprs; + ObArray inlist_infos; ObSEArray match_exprs; ObRawExprResolverImpl expr_resolver(ctx); if (OB_FAIL(session_info.get_name_case_mode(ctx.case_mode_))) { LOG_WARN("fail to get name case mode", K(ret)); } else if (OB_FAIL(expr_resolver.resolve(&node, expr, columns, sys_vars, sub_query_info, aggr_exprs, win_exprs, - udf_info, op_exprs, user_var_exprs, match_exprs))) { + udf_info, op_exprs, user_var_exprs, inlist_infos, match_exprs))) { LOG_WARN("resolve expr failed", K(ret)); } else { /*do nothing*/ } } @@ -3869,7 +3874,7 @@ int ObRawExprUtils::extract_column_exprs(const ObRawExpr *raw_expr, ObIArray &column_exprs) { int ret = OB_SUCCESS; - if (NULL == raw_expr) { + if (OB_ISNULL(raw_expr)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid raw expr", K(ret), K(raw_expr)); } else { @@ -3934,6 +3939,21 @@ int ObRawExprUtils::extract_column_exprs(const ObIArray &exprs, return ret; } +int ObRawExprUtils::extract_column_exprs(const ObIArray &exprs, + const common::ObIArray &table_ids, + ObIArray &column_exprs) +{ + int ret = OB_SUCCESS; + for (int64_t i = 0; OB_SUCC(ret) && i < table_ids.count(); ++i) { + if (OB_FAIL(extract_column_exprs(exprs, + table_ids.at(i), + column_exprs))) { + LOG_WARN("Failed to extract column exprs", K(ret)); + } + } + return ret; +} + int ObRawExprUtils::mark_column_explicited_reference(ObRawExpr &expr) { int ret = OB_SUCCESS; @@ -3990,17 +4010,23 @@ int ObRawExprUtils::extract_column_ids(const ObRawExpr *raw_expr, common::ObIArr int ObRawExprUtils::extract_table_ids(const ObRawExpr *raw_expr, common::ObIArray &table_ids) { int ret = OB_SUCCESS; - if (NULL == raw_expr) { + if (OB_ISNULL(raw_expr)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid raw expr", K(ret), K(raw_expr)); + } else if (T_REF_COLUMN == raw_expr->get_expr_type()) { + if (OB_FAIL(add_var_to_array_no_dup(table_ids, + static_cast(raw_expr)->get_table_id()))) { + LOG_WARN("failed to add var to array", K(ret)); + } + } else if (raw_expr->has_flag(IS_PSEUDO_COLUMN)) { + if (OB_FAIL(add_var_to_array_no_dup(table_ids, + static_cast(raw_expr)->get_table_id()))) { + LOG_WARN("failed to add var to array", K(ret)); + } } else { - if (T_REF_COLUMN == raw_expr->get_expr_type()) { - ret = add_var_to_array_no_dup(table_ids, - static_cast(raw_expr)->get_table_id()); - } else { - int64_t N = raw_expr->get_param_count(); - for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) { - ret = extract_table_ids(raw_expr->get_param_expr(i), table_ids); + for (int64_t i = 0; OB_SUCC(ret) && i < raw_expr->get_param_count(); ++i) { + if (OB_FAIL(extract_table_ids(raw_expr->get_param_expr(i), table_ids))) { + LOG_WARN("failed to extract table ids", K(ret)); } } } diff --git a/src/sql/resolver/expr/ob_raw_expr_util.h b/src/sql/resolver/expr/ob_raw_expr_util.h index fa9abf89a3..3fc3c3909c 100644 --- a/src/sql/resolver/expr/ob_raw_expr_util.h +++ b/src/sql/resolver/expr/ob_raw_expr_util.h @@ -422,6 +422,9 @@ public: static int extract_column_exprs(const common::ObIArray &exprs, int64_t table_id, common::ObIArray &column_exprs); + static int extract_column_exprs(const common::ObIArray &exprs, + const common::ObIArray &table_ids, + common::ObIArray &column_exprs); // no need to add cast. static int extract_column_exprs(const ObRawExpr *expr, ObIArray &column_exprs); diff --git a/src/sql/resolver/ob_resolver_utils.cpp b/src/sql/resolver/ob_resolver_utils.cpp index 715c28a2c3..e454c6680b 100644 --- a/src/sql/resolver/ob_resolver_utils.cpp +++ b/src/sql/resolver/ob_resolver_utils.cpp @@ -41,6 +41,7 @@ #include "sql/parser/ob_parser_utils.h" #include "lib/json/ob_json_print_utils.h" #include "sql/engine/expr/ob_expr_unistr.h" +#include "sql/resolver/dml/ob_inlist_resolver.h" #include "lib/charset/ob_ctype.h" namespace oceanbase @@ -2154,14 +2155,17 @@ int ObResolverUtils::resolve_obj_access_ref_node(ObRawExprFactory &expr_factory, ObArray op_exprs; ObSEArray user_var_exprs; ObSEArray match_exprs; + ObArray inlist_infos; if (OB_FAIL(expr_resolver.resolve(node, expr, columns, sys_vars, sub_query_info, aggr_exprs, - win_exprs, udf_info, op_exprs, user_var_exprs, match_exprs))) { + win_exprs, udf_info, op_exprs, user_var_exprs, + inlist_infos, match_exprs))) { LOG_WARN("failed to resolve expr tree", K(ret)); } else if (OB_UNLIKELY(1 != columns.count()) || OB_UNLIKELY(!sys_vars.empty()) || OB_UNLIKELY(!sub_query_info.empty()) || OB_UNLIKELY(!aggr_exprs.empty()) || OB_UNLIKELY(!win_exprs.empty()) + || OB_UNLIKELY(!inlist_infos.empty()) || OB_UNLIKELY(!match_exprs.empty())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("expr is invalid", K(op_exprs.empty()), K(columns.count()), K(sys_vars.count()), @@ -3305,6 +3309,7 @@ int ObResolverUtils::resolve_const_expr(ObResolverParams ¶ms, ObArray sys_vars; ObArray op_exprs; ObSEArray user_var_exprs; + ObArray inlist_infos; ObSEArray match_exprs; ObCollationType collation_connection = CS_TYPE_INVALID; ObCharsetType character_set_connection = CHARSET_INVALID; @@ -3331,7 +3336,7 @@ int ObResolverUtils::resolve_const_expr(ObResolverParams ¶ms, LOG_WARN("fail to get name case mode", K(ret)); } else if (OB_FAIL(expr_resolver.resolve(&node, const_expr, columns, sys_vars, sub_query_info, aggr_exprs, win_exprs, - udf_info, op_exprs, user_var_exprs, match_exprs))) { + udf_info, op_exprs, user_var_exprs, inlist_infos, match_exprs))) { LOG_WARN("resolve expr failed", K(ret)); } else if (OB_FAIL(resolve_columns_for_const_expr(const_expr, columns, params))) { LOG_WARN("resolve columnts for const expr failed", K(ret)); @@ -3341,6 +3346,9 @@ int ObResolverUtils::resolve_const_expr(ObResolverParams ¶ms, } else if (udf_info.count() > 0) { ret = OB_ERR_UNEXPECTED; LOG_ERROR("UDFInfo should not found be here!!!", K(ret)); + } else if (OB_UNLIKELY(!inlist_infos.empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("in_expr should not been found here", K(ret)); } else if (match_exprs.count() > 0) { ret = OB_ERR_UNEXPECTED; LOG_ERROR("fulltext search expr should not found be here", K(ret)); @@ -4269,6 +4277,7 @@ int ObResolverUtils::resolve_partition_range_value_expr(ObResolverParams ¶ms ObArray part_column_refs; ObArray op_exprs; ObSEArray user_var_exprs; + ObArray inlist_infos; ObSEArray match_exprs; ObExprResolveContext ctx(*params.expr_factory_, params.session_info_->get_timezone_info(), OB_NAME_CASE_INVALID); ctx.dest_collation_ = collation_connection; @@ -4281,20 +4290,22 @@ int ObResolverUtils::resolve_partition_range_value_expr(ObResolverParams ¶ms LOG_WARN("fail to get name case mode", K(ret)); } else if (OB_FAIL(expr_resolver.resolve(&node, part_value_expr, columns, sys_vars, sub_query_info, aggr_exprs, win_exprs, udf_info, - op_exprs, user_var_exprs, match_exprs))) { + op_exprs, user_var_exprs, inlist_infos, match_exprs))) { LOG_WARN("resolve expr failed", K(ret)); } else if (sub_query_info.count() > 0) { ret = OB_ERR_PARTITION_FUNCTION_IS_NOT_ALLOWED; } else if (OB_FAIL(resolve_columns_for_partition_range_value_expr(part_value_expr, columns))) { LOG_WARN("resolve columns failed", K(ret)); - } - if (OB_SUCC(ret) && udf_info.count() > 0) { + } else if (udf_info.count() > 0) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "udf"); + } else if (OB_UNLIKELY(!inlist_infos.empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("in_expr should not been found here", K(ret)); } else if (OB_UNLIKELY(match_exprs.count() > 0)) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "fulltext search func"); - } + } else { /* do nothing */ } if (OB_SUCC(ret)) { if (OB_ISNULL(part_value_expr)) { @@ -4436,6 +4447,7 @@ int ObResolverUtils::resolve_partition_range_value_expr(ObResolverParams ¶ms ObArray part_column_refs; ObArray op_exprs; ObSEArray user_var_exprs; + ObArray inlist_infos; ObSEArray match_exprs; ObExprResolveContext ctx(*params.expr_factory_, params.session_info_->get_timezone_info(), @@ -4458,6 +4470,7 @@ int ObResolverUtils::resolve_partition_range_value_expr(ObResolverParams ¶ms udf_info, op_exprs, user_var_exprs, + inlist_infos, match_exprs))) { LOG_WARN("resolve expr failed", K(ret)); } else if (sub_query_info.count() > 0) { @@ -4465,14 +4478,16 @@ int ObResolverUtils::resolve_partition_range_value_expr(ObResolverParams ¶ms } else if (OB_FAIL(resolve_columns_for_partition_range_value_expr(part_value_expr, columns))) { LOG_WARN("resolve columns failed", K(ret)); - } - if (OB_SUCC(ret) && udf_info.count() > 0) { + } else if (OB_UNLIKELY(udf_info.count() > 0)) { ret = OB_ERR_UNEXPECTED; LOG_ERROR("UDFInfo should not found be here!!!", K(ret)); + } else if (OB_UNLIKELY(!inlist_infos.empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("in_expr should not been found here", K(ret)); } else if (OB_UNLIKELY(match_exprs.count() > 0)) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "fulltext search func"); - } + } else {/* do nothing */} if (OB_SUCC(ret)) { if (OB_ISNULL(part_value_expr)) { @@ -4599,6 +4614,7 @@ int ObResolverUtils::resolve_partition_expr(ObResolverParams ¶ms, ObArray tmp_part_keys; ObArray op_exprs; ObSEArray user_var_exprs; + ObArray inlist_infos; ObSEArray match_exprs; ObCollationType collation_connection = CS_TYPE_INVALID; ObCharsetType character_set_connection = CHARSET_INVALID; @@ -4625,7 +4641,7 @@ int ObResolverUtils::resolve_partition_expr(ObResolverParams ¶ms, LOG_WARN("fail to get name case mode", K(ret)); } else if (OB_FAIL(expr_resolver.resolve(&node, part_expr, columns, sys_vars, sub_query_info, aggr_exprs, win_exprs, udf_info, - op_exprs, user_var_exprs, match_exprs))) { + op_exprs, user_var_exprs, inlist_infos, match_exprs))) { LOG_WARN("resolve expr failed", K(ret)); } else if (sub_query_info.count() > 0) { ret = OB_ERR_PARTITION_FUNCTION_IS_NOT_ALLOWED; @@ -4638,6 +4654,9 @@ int ObResolverUtils::resolve_partition_expr(ObResolverParams ¶ms, } else if (udf_info.count() > 0) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "udf"); + } else if (OB_UNLIKELY(!inlist_infos.empty())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("in_expr should not been found here", K(ret)); } else if (OB_UNLIKELY(match_exprs.count() > 0)) { ret = OB_NOT_SUPPORTED; LOG_USER_ERROR(OB_NOT_SUPPORTED, "fulltext search func"); @@ -9076,6 +9095,103 @@ int ObResolverUtils::is_negative_ora_nmb(const ObObjParam &obj_param, bool &is_n return ret; } +/* If ParseNode is a const param, fast know obj_type、collation_type、collation level */ +int ObResolverUtils::fast_get_param_type(const ParseNode &node, + const ParamStore *param_store, + const ObCollationType connect_collation, + const ObCollationType nchar_collation, + const ObCollationType server_collation, + const bool enable_decimal_int, + ObIAllocator &alloc, + ObObjType &obj_type, + ObCollationType &coll_type, + ObCollationLevel &coll_level) +{ + int ret = OB_SUCCESS; + if (T_QUESTIONMARK == node.type_) { + if (OB_ISNULL(param_store) || + OB_UNLIKELY(node.value_ < 0 || node.value_ >= param_store->count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid argument", K(ret)); + } else { + obj_type = param_store->at(node.value_).get_param_meta().get_type(); + coll_type = param_store->at(node.value_).get_param_meta().get_collation_type(); + coll_level = param_store->at(node.value_).get_param_meta().get_collation_level(); + } + } else if (IS_DATATYPE_OP(node.type_)) { + if (T_VARCHAR == node.type_ || T_CHAR == node.type_ || + T_NVARCHAR2 == node.type_ || T_NCHAR == node.type_) { + bool is_nchar = T_NVARCHAR2 == node.type_ || T_NCHAR == node.type_; + obj_type = lib::is_mysql_mode() && is_nchar ? ObVarcharType : + static_cast(node.type_); + coll_level = CS_LEVEL_COERCIBLE; + if (OB_UNLIKELY(node.str_len_ > OB_MAX_LONGTEXT_LENGTH)) { + ret = OB_ERR_INVALID_INPUT_ARGUMENT; + } else if (lib::is_oracle_mode()) { + coll_type = is_nchar ? nchar_collation : server_collation; + if (node.str_len_ == 0) { + obj_type = is_nchar ? ObNCharType : ObCharType; + } + } else { + if (0 == node.num_child_) { + coll_type = is_nchar ? CS_TYPE_UTF8MB4_GENERAL_CI : connect_collation; + } else if (NULL != node.children_[0] && T_CHARSET == node.children_[0]->type_) { + ObString charset(node.children_[0]->str_len_, node.children_[0]->str_value_); + ObCharsetType charset_type = ObCharset::charset_type(charset.trim()); + coll_type = ObCharset::get_default_collation(charset_type); + } else { + coll_type = connect_collation; + } + } + } else if (T_IEEE754_NAN == node.type_ || T_IEEE754_INFINITE == node.type_) { + obj_type = ObDoubleType; + coll_type = CS_TYPE_BINARY; + coll_level = CS_LEVEL_NUMERIC; + } else if (T_BOOL == node.type_) { + obj_type = ObTinyIntType; + coll_type = CS_TYPE_BINARY; + coll_level = CS_LEVEL_NUMERIC; + } else if (T_UINT64 == node.type_ || T_INT == node.type_ || T_NUMBER == node.type_) { + if ((lib::is_oracle_mode() && NULL != node.str_value_) || T_NUMBER == node.type_) { + bool use_decimalint_as_result = false; + int tmp_ret = OB_E(EventTable::EN_ENABLE_ORA_DECINT_CONST) OB_SUCCESS; + int16_t precision = PRECISION_UNKNOWN_YET; + int16_t scale = SCALE_UNKNOWN_YET; + ObDecimalInt *decint = nullptr; + if (OB_SUCCESS == tmp_ret && enable_decimal_int) { + int32_t val_len = 0; + ret = wide::from_string(node.str_value_, node.str_len_, alloc, scale, precision, val_len, decint); + use_decimalint_as_result = precision <= OB_MAX_DECIMAL_POSSIBLE_PRECISION && + scale <= OB_MAX_DECIMAL_POSSIBLE_PRECISION && + scale >= 0 && + precision >= scale && + precision <= OB_MAX_NUMBER_PRECISION; + } + if (use_decimalint_as_result) { + obj_type = ObDecimalIntType; + } else { + obj_type = ObNumberType; + } + } else { + obj_type = T_INT == node.type_ ? ObIntType : ObUInt64Type; + } + coll_type = CS_TYPE_BINARY; + coll_level = CS_LEVEL_NUMERIC; + } else { + obj_type = static_cast(node.type_); + coll_type = CS_TYPE_BINARY; + coll_level = CS_LEVEL_NUMERIC; + } + if (OB_SUCC(ret)) { + if (ObMaxType == obj_type || CS_TYPE_INVALID == coll_type || CS_LEVEL_INVALID == coll_level) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("resolve node type failed.", K(ret), K(obj_type), K(coll_type), K(coll_level)); + } + } + } + return ret; +} + int ObResolverUtils::check_allowed_alter_operations_for_mlog( const uint64_t tenant_id, const obrpc::ObAlterTableArg &arg, @@ -9186,6 +9302,96 @@ int ObResolverUtils::check_allowed_alter_operations_for_mlog( return ret; } +int ObResolverUtils::create_values_table_query(ObSQLSessionInfo *session_info, + ObIAllocator *allocator, + ObRawExprFactory *expr_factory, + ObQueryCtx *query_ctx, + ObSelectStmt *select_stmt, + ObValuesTableDef *table_def) +{ + int ret = OB_SUCCESS; + TableItem *table_item = NULL; + ObString alias_name; + if (OB_ISNULL(session_info) || OB_ISNULL(allocator) || OB_ISNULL(expr_factory) || + OB_ISNULL(query_ctx)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("got unexpected ptr", K(ret)); + } else if (OB_ISNULL(table_item = select_stmt->create_table_item(*allocator))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_ERROR("create table item failed"); + } else if (OB_FAIL(select_stmt->generate_values_table_name(*allocator, alias_name))) { + LOG_WARN("failed to generate func table name", K(ret)); + } else { + table_item->table_id_ = query_ctx->available_tb_id_--; + table_item->table_name_ = alias_name; + table_item->alias_name_ = alias_name; + table_item->type_ = TableItem::VALUES_TABLE; + table_item->is_view_table_ = false; + table_item->values_table_def_ = table_def; + if (OB_FAIL(select_stmt->add_table_item(session_info, table_item))) { + LOG_WARN("add table item failed", K(ret)); + } else if (OB_FAIL(select_stmt->add_from_item(table_item->table_id_))) { + LOG_WARN("add from table failed", K(ret)); + } + } + if (OB_SUCC(ret)) { + int64_t column_cnt = table_def->column_cnt_; + ObIArray &select_items = select_stmt->get_select_items(); + bool has_select_item = !select_items.empty(); + if (OB_UNLIKELY(table_def->column_types_.count() != column_cnt) || + OB_UNLIKELY(has_select_item && select_items.count() != column_cnt)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("got unexpected ptr", K(ret), K(column_cnt), K(table_def->column_types_.count()), K(select_items.count())); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < column_cnt; ++i) { + ObColumnRefRawExpr *column_expr = NULL; + ObSqlString tmp_col_name; + char *buf = NULL; + if (OB_FAIL(expr_factory->create_raw_expr(T_REF_COLUMN, column_expr))) { + LOG_WARN("create column ref raw expr failed", K(ret)); + } else if (OB_ISNULL(column_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN(("value desc is null"), K(ret)); + } else if (OB_FAIL(column_expr->add_flag(IS_COLUMN))) { + LOG_WARN("failed to add flag IS_COLUMN", K(ret)); + } else if (OB_FAIL(tmp_col_name.append_fmt("column_%ld", i))) { + LOG_WARN("failed to append fmt", K(ret)); + } else if (OB_ISNULL(buf = static_cast(allocator->alloc(tmp_col_name.length())))) { + ret = common::OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to allocate memory", K(ret), K(buf)); + } else { + column_expr->set_result_type(table_def->column_types_.at(i)); + column_expr->set_ref_id(table_item->table_id_, i + OB_APP_MIN_COLUMN_ID); + MEMCPY(buf, tmp_col_name.ptr(), tmp_col_name.length()); + ObString column_name(tmp_col_name.length(), buf); + column_expr->set_column_attr(table_item->table_name_, column_name); + ColumnItem column_item; + column_item.expr_ = column_expr; + column_item.table_id_ = column_expr->get_table_id(); + column_item.column_id_ = column_expr->get_column_id(); + column_item.column_name_ = column_expr->get_column_name(); + if (OB_FAIL(select_stmt->add_column_item(column_item))) { + LOG_WARN("failed to add column item", K(ret)); + } else if (has_select_item) { + SelectItem &select_item = select_items.at(i); + select_item.expr_ = column_expr; + } else { + SelectItem select_item; + select_item.alias_name_ = column_expr->get_column_name(); + select_item.expr_name_ = column_expr->get_column_name(); + select_item.is_real_alias_ = false; + select_item.expr_ = column_expr; + if (OB_FAIL(select_stmt->add_select_item(select_item))) { + LOG_WARN("failed to add select item", K(ret)); + } + } + } + } + } + } + return ret; +} + int64_t ObResolverUtils::get_mysql_max_partition_num(const uint64_t tenant_id) { int64_t max_partition_num = OB_MAX_PARTITION_NUM_MYSQL; diff --git a/src/sql/resolver/ob_resolver_utils.h b/src/sql/resolver/ob_resolver_utils.h index c5cedd5610..e1d3e5e678 100644 --- a/src/sql/resolver/ob_resolver_utils.h +++ b/src/sql/resolver/ob_resolver_utils.h @@ -820,10 +820,25 @@ public: static int check_keystore_status(const uint64_t tenant_id, ObSchemaChecker &schema_checker); static int check_encryption_name(common::ObString &encryption_name, bool &need_encrypt); static int check_not_supported_tenant_name(const common::ObString &tenant_name); - static int check_allowed_alter_operations_for_mlog( - const uint64_t tenant_id, - const obrpc::ObAlterTableArg &arg, - const share::schema::ObTableSchema &table_schema); + static int check_allowed_alter_operations_for_mlog(const uint64_t tenant_id, + const obrpc::ObAlterTableArg &arg, + const share::schema::ObTableSchema &table_schema); + static int fast_get_param_type(const ParseNode &parse_node, + const ParamStore *param_store, + const ObCollationType connect_collation, + const ObCollationType nchar_collation, + const ObCollationType server_collation, + const bool enable_decimal_int, + ObIAllocator &alloc, + ObObjType &obj_type, + ObCollationType &coll_type, + ObCollationLevel &coll_level); + static int create_values_table_query(ObSQLSessionInfo *session_info, + ObIAllocator *allocator, + ObRawExprFactory *expr_factory, + ObQueryCtx *query_ctx, + ObSelectStmt *select_stmt, + ObValuesTableDef *table_def); static int64_t get_mysql_max_partition_num(const uint64_t tenant_id); static int check_schema_valid_for_mview(const share::schema::ObTableSchema &table_schema); diff --git a/src/sql/rewrite/ob_query_range.cpp b/src/sql/rewrite/ob_query_range.cpp index bfc8fecce4..5802d151d0 100644 --- a/src/sql/rewrite/ob_query_range.cpp +++ b/src/sql/rewrite/ob_query_range.cpp @@ -164,7 +164,8 @@ int ObQueryRange::init_query_range_ctx(ObIAllocator &allocator, const ParamsIArray *params, const bool phy_rowid_for_table_loc, const bool ignore_calc_failure, - const bool use_in_optimization) + const bool use_in_optimization, + const int64_t index_prefix/* = -1*/) { int ret = OB_SUCCESS; void *ptr = NULL; @@ -183,6 +184,7 @@ int ObQueryRange::init_query_range_ctx(ObIAllocator &allocator, query_range_ctx_ = new(ptr) ObQueryRangeCtx(exec_ctx, expr_constraints, params); query_range_ctx_->phy_rowid_for_table_loc_ = phy_rowid_for_table_loc; query_range_ctx_->ignore_calc_failure_ = ignore_calc_failure; + query_range_ctx_->index_prefix_ = index_prefix; query_range_ctx_->range_optimizer_max_mem_size_ = exec_ctx->get_my_session()->get_range_optimizer_max_mem_size(); query_range_ctx_->cur_datetime_ = exec_ctx->get_physical_plan_ctx()->get_cur_time().get_datetime(); if (0 == query_range_ctx_->range_optimizer_max_mem_size_) { @@ -272,12 +274,13 @@ int ObQueryRange::preliminary_extract_query_range(const ColumnIArray &range_colu ObExecContext *exec_ctx, ExprConstrantArray *expr_constraints /* = NULL */, const ParamsIArray *params /* = NULL */, - const bool use_in_optimization /* = false */) + const bool use_in_optimization /* = false */, + const int64_t index_prefix/* = -1*/) { int ret = OB_SUCCESS; ObArenaAllocator ctx_allocator(ObModIds::OB_QUERY_RANGE_CTX); if (OB_FAIL(init_query_range_ctx(ctx_allocator, range_columns, exec_ctx, expr_constraints, params, - false, true, use_in_optimization))) { + false, true, use_in_optimization, index_prefix))) { LOG_WARN("init query range context failed", K(ret)); } else if (OB_ISNULL(query_range_ctx_)) { ret = OB_NOT_INIT; @@ -742,7 +745,8 @@ int ObQueryRange::preliminary_extract_query_range(const ColumnIArray &range_colu const ParamsIArray *params /* = NULL */, const bool phy_rowid_for_table_loc /* = false*/, const bool ignore_calc_failure /* = true*/, - const bool use_in_optimization /* = false */) + const bool use_in_optimization /* = false */, + const int64_t index_prefix/* = -1*/) { int ret = OB_SUCCESS; ObKeyPartList and_ranges; @@ -755,7 +759,7 @@ int ObQueryRange::preliminary_extract_query_range(const ColumnIArray &range_colu ObArenaAllocator ctx_allocator(ObModIds::OB_QUERY_RANGE_CTX); if (OB_FAIL(init_query_range_ctx(ctx_allocator, range_columns, exec_ctx, expr_constraints, params, phy_rowid_for_table_loc, - ignore_calc_failure, use_in_optimization))) { + ignore_calc_failure, use_in_optimization, index_prefix))) { LOG_WARN("init query range context failed", K(ret)); } else if (OB_ISNULL(query_range_ctx_)) { ret = OB_NOT_INIT; @@ -4758,6 +4762,9 @@ int ObQueryRange::is_key_part(const ObKeyPartId &id, ObKeyPartPos *&pos, bool &i if (OB_SUCCESS == map_ret && OB_NOT_NULL(pos) && (max_off == -1 || (max_off != - 1 && pos->offset_ <= max_off))) { is_key_part = true; + if (query_range_ctx_->index_prefix_ > -1 && pos->offset_ >= query_range_ctx_->index_prefix_) { + is_key_part = false; + } SQL_REWRITE_LOG(DEBUG, "id pair is key part", K_(id.table_id), K_(id.column_id)); } else if (OB_HASH_NOT_EXIST != map_ret) { ret = map_ret; diff --git a/src/sql/rewrite/ob_query_range.h b/src/sql/rewrite/ob_query_range.h index 09191fd357..bac6844e4d 100644 --- a/src/sql/rewrite/ob_query_range.h +++ b/src/sql/rewrite/ob_query_range.h @@ -102,6 +102,7 @@ private: row_in_offsets_(), only_one_expr_(false), is_oracle_char_gt_varchar_(false), + index_prefix_(-1), cur_datetime_(0) { } @@ -126,6 +127,7 @@ private: ObSEArray row_in_offsets_; bool only_one_expr_; bool is_oracle_char_gt_varchar_; + int64_t index_prefix_; int64_t cur_datetime_; }; public: @@ -387,7 +389,8 @@ public: ObExecContext *exec_ctx, ExprConstrantArray *expr_constraints = NULL, const ParamsIArray *params = NULL, - const bool use_in_optimization = false); + const bool use_in_optimization = false, + const int64_t index_prefix = -1); /** * @brief * @param range_columns: columns used to extract range, index column or partition column @@ -410,7 +413,8 @@ public: const ParamsIArray *params = NULL, const bool phy_rowid_for_table_loc = false, const bool ignore_calc_failure = true, - const bool use_in_optimization = false); + const bool use_in_optimization = false, + const int64_t index_prefix = -1); // final_extract_query_range extracts the final query range of its physical plan. // It will get the real-time value of some const which are unknown during physical plan generating. @@ -529,7 +533,8 @@ private: const ParamsIArray *params, const bool phy_rowid_for_table_loc, const bool ignore_calc_failure, - const bool use_in_optimization); + const bool use_in_optimization, + const int64_t index_prefix); void destroy_query_range_ctx(common::ObIAllocator &allocator); int add_expr_offsets(ObIArray &cur_pos, const ObKeyPart *cur_key); int extract_valid_exprs(const ExprIArray &root_exprs, diff --git a/src/sql/rewrite/ob_stmt_comparer.cpp b/src/sql/rewrite/ob_stmt_comparer.cpp index 1429abe2e2..8216ab54d2 100644 --- a/src/sql/rewrite/ob_stmt_comparer.cpp +++ b/src/sql/rewrite/ob_stmt_comparer.cpp @@ -1670,6 +1670,7 @@ int ObStmtComparer::compare_set_stmt(const ObSelectStmt *first, return ret; } +/* two values table uncomparable as default */ int ObStmtComparer::compare_values_table_item(const ObDMLStmt *first, const TableItem *first_table, const ObDMLStmt *second, @@ -1680,49 +1681,18 @@ int ObStmtComparer::compare_values_table_item(const ObDMLStmt *first, int ret = OB_SUCCESS; ObStmtCompareContext context(first, second, map_info, &first->get_query_ctx()->calculable_items_); relation = QueryRelation::QUERY_UNCOMPARABLE; - if (OB_ISNULL(first) || OB_ISNULL(first_table) - || OB_ISNULL(second) || OB_ISNULL(second_table)) { + ObValuesTableDef *first_def = nullptr; + ObValuesTableDef *second_def = nullptr; + const int64_t max_compare_count = 2000; + if (OB_ISNULL(first) || OB_ISNULL(first_table) || OB_ISNULL(second) || OB_ISNULL(second_table) || + OB_UNLIKELY(!first_table->is_values_table() || !second_table->is_values_table()) || + OB_ISNULL(first_def = first_table->values_table_def_) || + OB_ISNULL(second_def = second_table->values_table_def_)) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("param has null", K(first), K(first_table), K(second), K(second_table)); - } else if (first_table->is_values_table() && - second_table->is_values_table() && - first->get_column_size(first_table->table_id_) == second->get_column_size(second_table->table_id_) && - first_table->table_values_.count() % first->get_column_size(first_table->table_id_) == 0 && - second_table->table_values_.count() % second->get_column_size(second_table->table_id_) == 0) { - //Perhaps in the future, the comparison of different row orders can be considered - int64_t match_count = 0; - bool is_match = true; - for (int64_t i = 0; OB_SUCC(ret) && !is_match && i < first_table->table_values_.count(); ++i) { - bool is_match = false; - if (i >= second_table->table_values_.count()) { - break; - } else if (OB_FAIL(is_same_condition(first_table->table_values_.at(i), - second_table->table_values_.at(i), - context, - is_match))) { - LOG_WARN("failed to check is condition equal", K(ret)); - } else if (!is_match) { - // do nothing - } else if (OB_FAIL(append(map_info.equal_param_map_, context.equal_param_info_))) { - LOG_WARN("failed to append exprs", K(ret)); - } else if (OB_FAIL(append(map_info.const_param_map_, context.const_param_info_))) { - LOG_WARN("failed to append exprs", K(ret)); - } else { - ++match_count; - } - } - if (OB_SUCC(ret) && is_match) { - if (match_count == first_table->table_values_.count() && - match_count == second_table->table_values_.count()) {//first table is equal second table - relation = QueryRelation::QUERY_EQUAL; - } else if (match_count == first_table->table_values_.count()) {//first table is subset second table - relation = QueryRelation::QUERY_LEFT_SUBSET; - } else if (match_count == second_table->table_values_.count()) {////second table is subset first table - relation = QueryRelation::QUERY_RIGHT_SUBSET; - } - } - } else { - /*do nothing*/ + LOG_WARN("param has null", K(ret), KP(first), KP(first_table), KP(second), KP(second_table), + KP(first_def), KP(second_def)); + } else if (first_def == second_def) { + relation = QueryRelation::QUERY_EQUAL; } return ret; } diff --git a/src/sql/rewrite/ob_transform_conditional_aggr_coalesce.cpp b/src/sql/rewrite/ob_transform_conditional_aggr_coalesce.cpp new file mode 100644 index 0000000000..a6fe93c1f7 --- /dev/null +++ b/src/sql/rewrite/ob_transform_conditional_aggr_coalesce.cpp @@ -0,0 +1,1095 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#define USING_LOG_PREFIX SQL_REWRITE +#include "sql/rewrite/ob_transform_conditional_aggr_coalesce.h" +#include "sql/rewrite/ob_transform_utils.h" +#include "sql/optimizer/ob_optimizer_util.h" +#include "common/ob_smart_call.h" +#include "share/stat/ob_opt_stat_manager.h" +#include "sql/optimizer/ob_log_plan.h" +using namespace oceanbase::sql; +using namespace oceanbase::common; + +namespace oceanbase +{ +using namespace common; +namespace sql +{ +const int64_t ObTransformConditionalAggrCoalesce::MIN_COND_AGGR_CNT_FOR_COALESCE = 20; +const int64_t ObTransformConditionalAggrCoalesce::MAX_GBY_NDV_PRODUCT_FOR_COALESCE = 1000; +const double ObTransformConditionalAggrCoalesce::MIN_CUT_RATIO_FOR_COALESCE = 100; + +int ObTransformConditionalAggrCoalesce::transform_one_stmt( + common::ObIArray &parent_stmts, ObDMLStmt *&stmt, bool &trans_happened) +{ + int ret = OB_SUCCESS; + bool force_trans_wo_pullup = false; + bool force_no_trans_wo_pullup = false; + bool force_trans_with_pullup = false; + bool force_no_trans_with_pullup = false; + bool valid_wo_pullup = false; + bool valid_with_pullup = false; + TransFlagPair trans_happen_flags(false, false); + TransformParam trans_param; + trans_happened = false; + ObSelectStmt *select_stmt = NULL; + ObDMLStmt *parent_stmt = !parent_stmts.empty() ? parent_stmts.at(parent_stmts.count()-1).stmt_ : stmt; + bool trace_ignore_cost = (OB_E(EventTable::EN_COALESCE_AGGR_IGNORE_COST) OB_SUCCESS) != OB_SUCCESS; + if (OB_ISNULL(stmt) || OB_ISNULL(ctx_) || OB_ISNULL(stmt->get_query_ctx())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("param has null", K(ret), K(stmt), K(ctx_)); + } else if (stmt->get_query_ctx()->optimizer_features_enable_version_ < COMPAT_VERSION_4_2_3 || + (stmt->get_query_ctx()->optimizer_features_enable_version_ >= COMPAT_VERSION_4_3_0 && + stmt->get_query_ctx()->optimizer_features_enable_version_ < COMPAT_VERSION_4_3_2)) { + // do nothing + } else if (OB_FAIL(check_hint_valid(*stmt, + force_trans_wo_pullup, + force_no_trans_wo_pullup, + force_trans_with_pullup, + force_no_trans_with_pullup))) { + LOG_WARN("failed to check hint valid", K(ret)); + } else if (OB_FAIL(check_basic_validity(stmt, trans_param, select_stmt, + valid_wo_pullup, valid_with_pullup))) { + LOG_WARN("failed to check stmt validity", K(ret)); + } else if (!valid_wo_pullup && !valid_with_pullup) { + // do nothing + } else if (!force_no_trans_wo_pullup && + valid_wo_pullup && OB_FAIL(try_transform_wo_pullup(select_stmt, + parent_stmt, + force_trans_wo_pullup || trace_ignore_cost, + trans_param, + trans_happen_flags.first))) { + LOG_WARN("failed to try transform without pullup", K(ret)); + } else if (!force_no_trans_with_pullup && + valid_with_pullup && OB_FAIL(try_transform_with_pullup( + select_stmt, + parent_stmt, + force_trans_with_pullup || trace_ignore_cost, + trans_param, + trans_happen_flags.second))) { + LOG_WARN("failed to try transform with pullup", K(ret)); + } else if (!trans_happen_flags.first && !trans_happen_flags.second) { + // do nothing + } else if (OB_FAIL(add_transform_hint(*stmt, &trans_happen_flags))) { + LOG_WARN("failed to add transform hint", K(ret)); + } else { + trans_happened = true; + LOG_TRACE("succeed to coalesce conditional aggregate functions", K(trans_happened)); + OPT_TRACE("succeed to coalesce conditional aggregate functions", K(trans_happened)); + } + return ret; +} + +int ObTransformConditionalAggrCoalesce::check_basic_validity(ObDMLStmt *stmt, + TransformParam &trans_param, + ObSelectStmt *&select_stmt, + bool &valid_wo_pullup, + bool &valid_with_pullup) +{ + int ret = OB_SUCCESS; + bool has_rownum = false; + bool cnt_unpullupable_aggr = false; + valid_wo_pullup = false; + valid_with_pullup = false; + if (OB_ISNULL(stmt)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null pointer", K(ret)); + } else if (!stmt->is_select_stmt()) { + // do nothing + } else if (OB_FALSE_IT(select_stmt = static_cast(stmt))) { + } else if (OB_FAIL(select_stmt->has_rownum(has_rownum))) { + LOG_WARN("failed to get rownum info", K(ret)); + } else if (select_stmt->get_aggr_item_size() < 1 || select_stmt->is_set_stmt() || + select_stmt->is_hierarchical_query() || select_stmt->has_sequence() || + select_stmt->has_rollup() || select_stmt->has_grouping_sets() || + select_stmt->has_cube() || has_rownum) { + // do nothing + } else if (OB_FAIL(collect_cond_aggrs_info(select_stmt, trans_param, cnt_unpullupable_aggr))) { + LOG_WARN("failed to check aggr items", K(ret)); + } else { + valid_wo_pullup = trans_param.cond_aggrs_wo_extra_dep_.count() > 0; + valid_with_pullup = !cnt_unpullupable_aggr && + trans_param.cond_aggrs_with_extra_dep_.count() > 0; + } + return ret; +} + +// collect conditional aggregate functions that are possible candidates for coalescing along with +// their dependent columns beyond the group by exprs, and determine whether the aggr items can be +// entirely pulluped. +int ObTransformConditionalAggrCoalesce::collect_cond_aggrs_info(ObSelectStmt *select_stmt, + TransformParam &trans_param, + bool &cnt_unpullupable_aggr) +{ + int ret = OB_SUCCESS; + cnt_unpullupable_aggr = false; + if (OB_ISNULL(select_stmt)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null pointer", K(ret)); + } + for (int64_t i = 0; OB_SUCC(ret) && i < select_stmt->get_aggr_item_size(); i++) { + ObAggFunRawExpr *aggr_expr = NULL; + bool is_target_aggr_type = false; + bool is_param_distinct = false; + bool is_cond_aggr = false; + ObCaseOpRawExpr *case_expr = NULL; + bool is_case_when_valid = false; + ObSEArray extra_dep_cols; + if (OB_ISNULL(aggr_expr = select_stmt->get_aggr_item(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null expr", K(ret)); + } else if (OB_FALSE_IT(is_param_distinct = aggr_expr->is_param_distinct())) { + } else if (OB_FALSE_IT(is_target_aggr_type = check_aggr_type(aggr_expr->get_expr_type()))) { + } else if (OB_FAIL(check_cond_aggr_form(aggr_expr, case_expr, is_cond_aggr))) { + LOG_WARN("failed to check cond aggr form", K(ret)); + } else if (!is_cond_aggr) { + // do nothing + } else if (OB_FAIL(check_case_when_validity(select_stmt, aggr_expr, case_expr, + is_case_when_valid))) { + LOG_WARN("failed to check case expr rewritable", K(ret)); + } else if (!is_case_when_valid) { + // do nothing + } else if (OB_FAIL(extract_extra_dep_cols(case_expr->get_when_param_exprs(), + select_stmt->get_group_exprs(), + extra_dep_cols))) { + LOG_WARN("failed to check extra dependent columns", K(ret)); + } else if (extra_dep_cols.count() == 0) { + if (OB_FAIL(trans_param.cond_aggrs_wo_extra_dep_.push_back(aggr_expr))) { + LOG_WARN("failed to push back expr", K(ret)); + } + } else if (trans_param.extra_dep_cols_.count() == 0) { + if (OB_FAIL(trans_param.cond_aggrs_with_extra_dep_.push_back(aggr_expr))) { + LOG_WARN("failed to push back expr", K(ret)); + } else if (OB_FAIL(trans_param.extra_dep_cols_.assign(extra_dep_cols))) { + LOG_WARN("failed to assign expr array", K(ret)); + } + } else if (ObOptimizerUtil::subset_exprs(extra_dep_cols, trans_param.extra_dep_cols_)) { + if (OB_FAIL(trans_param.cond_aggrs_with_extra_dep_.push_back(aggr_expr))) { + LOG_WARN("failed to push back expr", K(ret)); + } + } else { + // multiple sets of dependent columns are currently prohibited for trans_with_pullup + cnt_unpullupable_aggr = true; + } + cnt_unpullupable_aggr |= (!is_target_aggr_type || is_param_distinct); + } + return ret; +} + +bool ObTransformConditionalAggrCoalesce::check_aggr_type(ObItemType aggr_type) +{ + bool is_target_aggr_type = false; + if (T_FUN_MAX == aggr_type || + T_FUN_MIN == aggr_type || + T_FUN_SUM == aggr_type || + T_FUN_COUNT == aggr_type || + T_FUN_COUNT_SUM == aggr_type || + T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS == aggr_type || + T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE == aggr_type || + T_FUN_SYS_BIT_AND == aggr_type || + T_FUN_SYS_BIT_OR == aggr_type || + T_FUN_SYS_BIT_XOR == aggr_type) { + is_target_aggr_type = true; + } else { + is_target_aggr_type = false; + } + return is_target_aggr_type; +} + +int ObTransformConditionalAggrCoalesce::check_cond_aggr_form(ObAggFunRawExpr *aggr_expr, + ObCaseOpRawExpr *&case_expr, + bool &is_cond_aggr) +{ + int ret = OB_SUCCESS; + is_cond_aggr = true; + if (OB_ISNULL(aggr_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null pointer", K(ret)); + } else if (aggr_expr->get_param_count() != 1) { + is_cond_aggr = false; + } else if (OB_ISNULL(aggr_expr->get_param_expr(0))) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("unexpected null param expr", K(ret)); + } else if (aggr_expr->get_param_expr(0)->is_case_op_expr()) { + case_expr = static_cast(aggr_expr->get_param_expr(0)); + } else { + is_cond_aggr = false; + } + return ret; +} + +// After rewriting, both when exprs and then exprs will have their computation times changed, +// so we need to make sure that their computation results do not change with the computation times. +// At the same time, the calculation of then exprs will be advanced, and it is necessary to ensure +// that the risk of reporting errors after they are advanced is low. +int ObTransformConditionalAggrCoalesce::check_case_when_validity(ObSelectStmt *select_stmt, + ObAggFunRawExpr *cond_aggr, + ObCaseOpRawExpr *case_expr, + bool &is_case_when_valid) +{ + int ret = OB_SUCCESS; + ObSEArray depend_exprs; + ObSEArray then_else_exprs; + is_case_when_valid = true; + bool satisfy_fd = true; + bool is_error_free = true; + bool can_calc_times_change = true; + if (OB_ISNULL(case_expr) || OB_ISNULL(select_stmt) || OB_ISNULL(cond_aggr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null expr", K(ret)); + } else if (OB_FAIL(select_stmt->get_column_exprs(depend_exprs))) { + LOG_WARN("failed to get column exprs", K(ret)); + } else if (OB_FAIL(append_array_no_dup(depend_exprs, select_stmt->get_group_exprs()))) { + LOG_WARN("failed to append exprs", K(ret)); + } else if (OB_FAIL(append(then_else_exprs, case_expr->get_then_param_exprs()))) { + LOG_WARN("failed to append exprs", K(ret)); + } else if (OB_FAIL(then_else_exprs.push_back(case_expr->get_default_param_expr()))) { + LOG_WARN("failed to push back expr", K(ret)); + } else { + // check when exprs + for (int64_t i = 0; OB_SUCC(ret) && is_case_when_valid && + i < case_expr->get_when_param_exprs().count(); i++) { + if (OB_FAIL(ObOptimizerUtil::expr_calculable_by_exprs(case_expr->get_when_param_expr(i), + depend_exprs, + true, + true, + satisfy_fd))) { + LOG_WARN("failed to judge expr is calculable with dependent exprs given", K(ret)); + } else if (!satisfy_fd) { + is_case_when_valid = false; + } + } + + // check then/else exprs + for (int64_t i = 0; OB_SUCC(ret) && is_case_when_valid && i < then_else_exprs.count(); i++) { + if (OB_FAIL(ObTransformUtils::check_error_free_expr(then_else_exprs.at(i), is_error_free))) { + LOG_WARN("failed to check error free exprs", K(ret)); + } else if (!is_error_free) { + is_case_when_valid = false; + } else if (OB_FAIL(ObOptimizerUtil::is_const_expr_recursively(then_else_exprs.at(i), + depend_exprs, + can_calc_times_change))) { + LOG_WARN("failed to check const expr recursively", K(ret)); + } else if (!can_calc_times_change) { + is_case_when_valid = false; + } + } + } + return ret; +} + +int ObTransformConditionalAggrCoalesce::extract_extra_dep_cols(ObIArray &target_exprs, + ObIArray &exclude_exprs, + ObIArray &extra_dep_cols) +{ + int ret = OB_SUCCESS; + for (int64_t i = 0; OB_SUCC(ret) && i < target_exprs.count(); i++) { + if (OB_FAIL(inner_extract_extra_dep_cols(target_exprs.at(i), exclude_exprs, extra_dep_cols))) { + LOG_WARN("failed to inner extract extra dep cols", K(ret)); + } + } + return ret; +} + +// Extract the columns that target expr depends on in addition to static consts and exclude exprs. +// Note: assume target expr has been determined to be rewritable already. +// e.g. {exclude_exprs(c2+c3),target_expr(c1+c2+c3)} => extra_dep_cols(c1) +int ObTransformConditionalAggrCoalesce::inner_extract_extra_dep_cols( + ObRawExpr *target_expr, + ObIArray &exclude_exprs, + ObIArray &extra_dep_cols) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(target_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null expr", K(ret)); + } else if (ObOptimizerUtil::find_item(exclude_exprs, target_expr)) { + // do nothing + } else if (target_expr->is_column_ref_expr()) { + if (OB_FAIL(add_var_to_array_no_dup(extra_dep_cols, target_expr))) { + LOG_WARN("failed to push back expr", K(ret)); + } + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < target_expr->get_param_count(); i++) { + if (OB_FAIL(SMART_CALL(inner_extract_extra_dep_cols(target_expr->get_param_expr(i), + exclude_exprs, + extra_dep_cols)))) { + LOG_WARN("failed to inner extract extra dep cols", K(ret)); + } + } + } + return ret; +} + +int ObTransformConditionalAggrCoalesce::try_transform_wo_pullup(ObSelectStmt *select_stmt, + ObDMLStmt *parent_stmt, + bool force_trans, + TransformParam &trans_param, + bool &trans_happened) +{ + int ret = OB_SUCCESS; + ObSEArray coalesced_case_exprs; + ObSEArray new_aggr_items; + ObSEArray param_constraints; + bool is_aggr_count_decrease = false; + if (OB_ISNULL(select_stmt)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null expr", K(ret)); + } else if (trans_param.cond_aggrs_wo_extra_dep_.count() < 1) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected empty cond aggrs", K(ret)); + } else if (OB_FAIL(coalesce_cond_aggrs(select_stmt->get_aggr_items(), + trans_param.cond_aggrs_wo_extra_dep_, + coalesced_case_exprs, + new_aggr_items, + param_constraints))) { + LOG_WARN("failed to coalesce conditional aggrs", K(ret)); + } else if (OB_FAIL(check_aggrs_count_decrease(select_stmt->get_aggr_items(), + new_aggr_items, + is_aggr_count_decrease))) { + LOG_WARN("failed to check aggrs count decrease", K(ret)); + } else if (!force_trans && !is_aggr_count_decrease) { + LOG_TRACE("reject coalesce without pullup due to increased aggregate functions"); + OPT_TRACE("reject coalesce without pullup due to increased aggregate functions"); + } else if (OB_FAIL(do_transform_wo_pullup(select_stmt, + trans_param.cond_aggrs_wo_extra_dep_, + coalesced_case_exprs, + new_aggr_items, + param_constraints))) { + LOG_WARN("failed to do transform without pullup", K(ret)); + } else if (OB_FAIL(refresh_project_name(parent_stmt, select_stmt))) { + LOG_WARN("failed to refresh column name", K(ret)); + } else { + trans_happened = true; + } + return ret; +} + +int ObTransformConditionalAggrCoalesce::try_transform_with_pullup(ObSelectStmt *select_stmt, + ObDMLStmt *parent_stmt, + bool force_trans, + TransformParam &trans_param, + bool &trans_happened) +{ + int ret = OB_SUCCESS; + ObSEArray coalesced_case_exprs; + ObSEArray new_aggr_items; + ObSEArray param_constraints; + bool is_aggr_count_decrease = false; + bool hit_threshold = false; + if (OB_ISNULL(select_stmt)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null expr", K(ret)); + } else if (trans_param.cond_aggrs_with_extra_dep_.count() < 1 || + trans_param.extra_dep_cols_.count() < 1) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected empty array", K(ret)); + } else if (OB_FAIL(coalesce_cond_aggrs(select_stmt->get_aggr_items(), + trans_param.cond_aggrs_with_extra_dep_, + coalesced_case_exprs, + new_aggr_items, + param_constraints))) { + LOG_WARN("failed to coalesce cond aggrs", K(ret)); + } else if (OB_FAIL(check_aggrs_count_decrease(select_stmt->get_aggr_items(), + new_aggr_items, + is_aggr_count_decrease))) { + LOG_WARN("failed to check aggrs count decrease", K(ret)); + } else if (!force_trans && !is_aggr_count_decrease) { + LOG_TRACE("reject coalesce with pullup due to increased aggregate functions"); + OPT_TRACE("reject coalesce with pullup due to increased aggregate functions"); + } else if (!force_trans && OB_FAIL(check_statistics_threshold(select_stmt, + trans_param, + hit_threshold))) { + LOG_WARN("failed to check statistics threshold", K(ret)); + } else if (!force_trans && !hit_threshold) { + LOG_TRACE("reject coalesce with pullup due to statistics threshold"); + OPT_TRACE("reject coalesce with pullup due to statistics threshold"); + } else if (OB_FAIL(do_transform_with_pullup(select_stmt, + trans_param.cond_aggrs_with_extra_dep_, + trans_param.extra_dep_cols_, + coalesced_case_exprs, + new_aggr_items, + param_constraints))) { + LOG_WARN("failed to do transform with pullup", K(ret)); + } else if (OB_FAIL(refresh_project_name(parent_stmt, select_stmt))) { + LOG_WARN("failed to refresh column name", K(ret)); + } else { + trans_happened = true; + } + return ret; +} + +int ObTransformConditionalAggrCoalesce::check_statistics_threshold(ObSelectStmt *select_stmt, + TransformParam &trans_param, + bool &hit_threshold) +{ + int ret = OB_SUCCESS; + hit_threshold = false; + TableItem* base_table = NULL; + ObSEArray cols_in_groupby; + const ObTableSchema *table_schema = NULL; + int64_t row_num = 0; + int64_t ndv_product = 1; + double cut_ratio = 0; + int64_t cond_aggr_cnt = trans_param.cond_aggrs_with_extra_dep_.count(); + if (OB_ISNULL(ctx_) || OB_ISNULL(ctx_->schema_checker_) || OB_ISNULL(ctx_->session_info_) || + OB_ISNULL(ctx_->opt_stat_mgr_) || OB_ISNULL(select_stmt)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (select_stmt->get_table_size() != 1 || + select_stmt->get_joined_tables().count() > 0 || + select_stmt->get_semi_info_size() > 0) { + LOG_TRACE("access more than one base table, disable rewrite"); + OPT_TRACE("access more than one base table, disable rewrite"); + } else if (OB_ISNULL(base_table = select_stmt->get_table_item(0))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null pointer", K(ret)); + } else if (!base_table->is_basic_table()) { + // in order to evaluate the rewrite gains more accurately, + // stmt is required to access only one base table + LOG_TRACE("access more than one base table, disable rewrite"); + OPT_TRACE("access more than one base table, disable rewrite"); + } else if (OB_FAIL(ObRawExprUtils::extract_column_exprs(select_stmt->get_group_exprs(), + cols_in_groupby))) { + LOG_WARN("failed to extract column exprs", K(ret)); + } else if (OB_FAIL(append(cols_in_groupby, trans_param.extra_dep_cols_))) { + LOG_WARN("failed to append exprs", K(ret)); + } else if (OB_FAIL(ctx_->schema_checker_->get_table_schema( + ctx_->session_info_->get_effective_tenant_id(), + base_table->ref_id_, table_schema))) { + LOG_WARN("get table schema failed", K(ret)); + } else if (OB_ISNULL(table_schema)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else { + // collect ndv product for cols_in_groupby + for (int64_t i = 0; OB_SUCC(ret) && i < cols_in_groupby.count(); i++) { + uint64_t column_id = 0; + uint64_t table_id = 0; + ObColumnRefRawExpr* col = NULL; + ObOptColumnStatHandle handle; + if (OB_ISNULL(cols_in_groupby.at(i)) || !cols_in_groupby.at(i)->is_column_ref_expr()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected column ref expr", K(ret)); + } else if (OB_FALSE_IT(col = static_cast(cols_in_groupby.at(i)))) { + } else if (OB_FALSE_IT(column_id = col->get_column_id())) { + } else if (OB_FALSE_IT(table_id = col->get_table_id())) { + } else if (OB_FAIL(ctx_->opt_stat_mgr_->get_column_stat( + ctx_->session_info_->get_effective_tenant_id(), + base_table->ref_id_, + table_schema->is_partitioned_table() ? -1 : base_table->ref_id_, + column_id, handle))) { + LOG_WARN("fail get full table column stat", K(ret), K(table_id), K(column_id)); + } else if (OB_ISNULL(handle.stat_)) { + ret = OB_ERR_UNEXPECTED; + } else { + ndv_product = ndv_product * handle.stat_->get_num_distinct(); + } + } + + // get estimated rownum of basic table + if (OB_SUCC(ret)) { + ObSEArray tablet_ids; + ObSEArray partition_ids; + ObSEArray first_level_part_ids; + ObSEArray int64_partition_ids; + ObSEArray table_stats; + if (OB_FAIL(table_schema->get_all_tablet_and_object_ids(tablet_ids, + partition_ids, + &first_level_part_ids))) { + LOG_WARN("failed to get all partition ids"); + } else if (OB_FAIL(append(int64_partition_ids, partition_ids))) { + LOG_WARN("failed to append partition ids", K(ret)); + } else if (OB_FAIL(ctx_->opt_stat_mgr_->get_table_stat( + ctx_->session_info_->get_effective_tenant_id(), + table_schema->get_table_id(), + int64_partition_ids, + table_stats))) { + LOG_WARN("failed to get table stats", K(ret)); + } + for (int64_t i = 0; OB_SUCC(ret) && i < table_stats.count(); i++) { + row_num += table_stats.at(i).get_row_count(); + } + } + + // check statistics thresholds + if (OB_SUCC(ret)) { + cut_ratio = (ndv_product == 0) ? 0 : row_num * 1.0 / ndv_product; + if (cond_aggr_cnt < MIN_COND_AGGR_CNT_FOR_COALESCE) { + OPT_TRACE("reject rewrite due to conditional aggr count", K(cond_aggr_cnt)); + } else if (ndv_product > MAX_GBY_NDV_PRODUCT_FOR_COALESCE) { + OPT_TRACE("reject rewrite due to ndv product", K(ndv_product)); + } else if (cut_ratio < MIN_CUT_RATIO_FOR_COALESCE) { + OPT_TRACE("reject rewrite due to cut ratio", K(cut_ratio), K(ndv_product), K(row_num)); + } else { + hit_threshold = true; + } + } + } + return ret; +} + +// 根据是否有 distinct 对两类聚合函数分别计数,不能糅在一起,因为带 distinct 聚合函数计算代价远超普通聚合函数 +int ObTransformConditionalAggrCoalesce::check_aggrs_count_decrease(ObIArray &old_aggrs, + ObIArray &new_aggrs, + bool &is_cnt_decrease) +{ + int ret = OB_SUCCESS; + int64_t old_distinct_cnt = 0; + int64_t old_normal_cnt = 0; + int64_t new_distinct_cnt = 0; + int64_t new_normal_cnt = 0; + if (old_aggrs.count() < new_aggrs.count()) { + is_cnt_decrease = false; + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < old_aggrs.count(); i++) { + ObAggFunRawExpr* aggr = NULL; + if OB_ISNULL(aggr = old_aggrs.at(i)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (aggr->is_param_distinct()) { + old_distinct_cnt += 1; + } else { + old_normal_cnt += 1; + } + } + for (int64_t i = 0; OB_SUCC(ret) && i < new_aggrs.count(); i++) { + ObAggFunRawExpr* aggr = NULL; + if OB_ISNULL(aggr = new_aggrs.at(i)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (aggr->is_param_distinct()) { + new_distinct_cnt += 1; + } else { + new_normal_cnt += 1; + } + } + is_cnt_decrease = (old_distinct_cnt >= new_distinct_cnt) && (old_normal_cnt >= new_normal_cnt); + } + return ret; +} + +int ObTransformConditionalAggrCoalesce::do_transform_wo_pullup( + ObSelectStmt *select_stmt, + ObIArray &cond_aggrs, + ObIArray &coalesced_case_exprs, + ObIArray &new_aggr_items, + ObIArray &constraints) +{ + int ret = OB_SUCCESS; + ObSEArray cond_aggrs_for_replace; + ObStmtExprReplacer replacer; + if (OB_ISNULL(select_stmt) || OB_ISNULL(ctx_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null pointer", K(ret)); + } else if (OB_FAIL(append(ctx_->plan_const_param_constraints_, constraints))) { + LOG_WARN("failed to append constraints", K(ret)); + } else if (OB_FAIL(append(cond_aggrs_for_replace, cond_aggrs))) { + LOG_WARN("failed to append exprs", K(ret)); + } else { + // replace cond aggrs with new case exprs + replacer.remove_all(); + replacer.add_scope(SCOPE_HAVING); + replacer.add_scope(SCOPE_SELECT); + replacer.add_scope(SCOPE_ORDERBY); + replacer.set_recursive(false); + if (OB_FAIL(replacer.add_replace_exprs(cond_aggrs_for_replace, coalesced_case_exprs))) { + LOG_WARN("failed to add replace exprs", K(ret)); + } else if (OB_FAIL(select_stmt->iterate_stmt_expr(replacer))) { + LOG_WARN("failed to iterate stmt expr", K(ret)); + } else if (OB_FAIL(select_stmt->get_aggr_items().assign(new_aggr_items))) { + LOG_WARN("failed to assign aggr items", K(ret)); + } + } + return ret; +} + +int ObTransformConditionalAggrCoalesce::do_transform_with_pullup( + ObSelectStmt *select_stmt, + ObIArray &cond_aggrs, + ObIArray &extra_dep_cols, + ObIArray &coalesced_case_exprs, + ObIArray &new_aggr_items, + ObIArray &constraints) +{ + int ret = OB_SUCCESS; + ObSEArray from_tables; + ObSEArray semi_infos; + ObSEArray condition_exprs; + ObSEArray pushdown_select; + TableItem *view_table = NULL; + ObSelectStmt *view_stmt = NULL; + + if (OB_ISNULL(select_stmt)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + + // 1. coalesce cond aggrs + } else if (OB_FAIL(do_transform_wo_pullup(select_stmt, + cond_aggrs, + coalesced_case_exprs, + new_aggr_items, + constraints))) { + LOG_WARN("failed to do transform without pullup", K(ret)); + + // 2. create inline view + } else if (OB_FAIL(select_stmt->get_from_tables(from_tables))) { + LOG_WARN("failed to get from tables", K(ret)); + } else if (OB_FAIL(semi_infos.assign(select_stmt->get_semi_infos()))) { + LOG_WARN("failed to assign semi infos", K(ret)); + } else if (OB_FAIL(condition_exprs.assign(select_stmt->get_condition_exprs()))) { + LOG_WARN("failed to assign semi infos", K(ret)); + } else if (OB_FAIL(collect_pushdown_select(select_stmt, + extra_dep_cols, + coalesced_case_exprs, + pushdown_select))) { + LOG_WARN("failed to collect pushdown select", K(ret)); + } else if (OB_FALSE_IT(select_stmt->get_condition_exprs().reuse())) { + } else if (OB_FALSE_IT(select_stmt->get_aggr_items().reuse())) { + } else if (OB_FAIL(ObTransformUtils::replace_with_empty_view(ctx_, + select_stmt, + view_table, + from_tables, + &semi_infos))) { + LOG_WARN("failed to create empty view", K(ret)); + } else if (OB_FAIL(ObTransformUtils::create_inline_view(ctx_, + select_stmt, + view_table, + from_tables, + &condition_exprs, + &semi_infos, + &pushdown_select, + &select_stmt->get_group_exprs()))) { + LOG_WARN("failed to create inline view", K(ret)); + } else if (OB_ISNULL(view_stmt = view_table->ref_query_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("null view query", K(ret)); + } else if (OB_FAIL(append(view_stmt->get_group_exprs(), extra_dep_cols))) { + LOG_WARN("failed to append extra dependent cols", K(ret)); + + // 3. construct aggr exprs to merge the aggregation results of view stmt + } else if (OB_FAIL(create_and_replace_aggrs_for_merge(select_stmt, view_stmt))) { + LOG_WARN("failed to create aggrs for merge", K(ret)); + } + return ret; +} + +int ObTransformConditionalAggrCoalesce::coalesce_cond_aggrs(ObIArray &base_aggrs, + ObIArray &cond_aggrs, + ObIArray &case_exprs, + ObIArray &new_aggrs, + ObIArray &constraints) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(ctx_) || OB_ISNULL(ctx_->expr_factory_) || OB_ISNULL(ctx_->session_info_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } + for (int64_t i = 0; OB_SUCC(ret) && i < base_aggrs.count(); i++) { + ObAggFunRawExpr* aggr_expr = NULL; + if (OB_ISNULL(aggr_expr = base_aggrs.at(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (ObOptimizerUtil::find_item(cond_aggrs, aggr_expr)) { + // do nothing + } else if (OB_FAIL(new_aggrs.push_back(aggr_expr))) { + LOG_WARN("failed to push back expr", K(ret)); + } + } + for (int64_t i = 0; OB_SUCC(ret) && i < cond_aggrs.count(); i++) { + ObAggFunRawExpr *cond_aggr = NULL; + ObCaseOpRawExpr *case_expr = NULL; + ObSEArray new_then_exprs; + ObRawExpr *new_default_expr; + ObCaseOpRawExpr *new_case_expr = NULL; + ObRawExpr *cast_case_expr = NULL; + bool is_cond_aggr = false; + if (OB_ISNULL(cond_aggr = cond_aggrs.at(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (OB_FAIL(check_cond_aggr_form(cond_aggr, case_expr, is_cond_aggr))) { + LOG_WARN("failed to check cond aggr form", K(ret)); + } else if (OB_UNLIKELY(!is_cond_aggr) || OB_ISNULL(case_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected aggr form", K(ret)); + } else { + // build new then exprs + ObAggFunRawExpr* new_aggr_expr = NULL; + bool is_sharable = false; + for (int64_t i = 0; OB_SUCC(ret) && i < case_expr->get_then_expr_size(); i++) { + ObRawExpr* then_expr = NULL; + if (OB_ISNULL(then_expr = case_expr->get_then_param_expr(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (OB_FAIL(build_aggr_expr(cond_aggr->get_expr_type(), + cond_aggr->is_param_distinct(), + then_expr, + new_aggr_expr))) { + LOG_WARN("failed to build aggr expr", K(ret)); + } else if (OB_FAIL(try_share_aggr(new_aggrs, new_aggr_expr, is_sharable, constraints))) { + LOG_WARN("failed to try to share aggr expr", K(ret)); + } else if (!is_sharable && OB_FAIL(new_aggrs.push_back(new_aggr_expr))) { + LOG_WARN("failed to push back expr", K(ret)); + } else if (OB_FAIL(new_then_exprs.push_back(new_aggr_expr))) { + LOG_WARN("failed to push back expr", K(ret)); + } + } + + // build new default expr + ObRawExpr* default_expr = NULL; + if (OB_FAIL(ret)) { + } else if (OB_ISNULL(default_expr = case_expr->get_default_param_expr())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (OB_FAIL(build_aggr_expr(cond_aggr->get_expr_type(), + cond_aggr->is_param_distinct(), + default_expr, + new_aggr_expr))) { + LOG_WARN("failed to build aggr expr", K(ret)); + } else if (OB_FAIL(try_share_aggr(new_aggrs, new_aggr_expr, is_sharable, constraints))) { + LOG_WARN("failed to try to share aggr expr", K(ret)); + } else if (!is_sharable && OB_FAIL(new_aggrs.push_back(new_aggr_expr))) { + LOG_WARN("failed to push back expr", K(ret)); + } else { + new_default_expr = new_aggr_expr; + } + + // build new case expr + if (OB_FAIL(ret)) { + } else if (OB_FAIL(ObTransformUtils::build_case_when_expr(ctx_, + case_expr->get_when_param_exprs(), + new_then_exprs, + new_default_expr, + new_case_expr))) { + LOG_WARN("failed to build case when exprs", K(ret)); + } else if (OB_FALSE_IT(cast_case_expr = new_case_expr)) { + } else if (ObTransformUtils::add_cast_for_replace_if_need(*ctx_->expr_factory_, + cond_aggr, + cast_case_expr, + ctx_->session_info_)) { + LOG_WARN("failed to add cast", K(ret)); + } else if (OB_FAIL(case_exprs.push_back(cast_case_expr))) { + LOG_WARN("failed to push back expr", K(ret)); + } + } + } + return ret; +} + +int ObTransformConditionalAggrCoalesce::build_aggr_expr(ObItemType expr_type, + bool is_param_distinct, + ObRawExpr *param_expr, + ObAggFunRawExpr *&aggr_expr) +{ + int ret = OB_SUCCESS; + ObRawExprFactory *expr_factory = NULL; + if (OB_ISNULL(ctx_) || OB_ISNULL(expr_factory = ctx_->expr_factory_) || + OB_ISNULL(ctx_->session_info_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret), K(ctx_), K(expr_factory)); + } else if (OB_FAIL(expr_factory->create_raw_expr(expr_type, + aggr_expr))) { + LOG_WARN("create aggr expr failed", K(ret)); + } else if (OB_ISNULL(aggr_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (OB_FAIL(aggr_expr->add_real_param_expr(param_expr))) { + LOG_WARN("fail to set param expr", K(ret)); + } else if (OB_FALSE_IT(aggr_expr->set_param_distinct(is_param_distinct))) { + } else if (OB_FAIL(aggr_expr->formalize(ctx_->session_info_))) { + LOG_WARN("failed to formalize aggregate function", K(ret)); + } else if (OB_FAIL(aggr_expr->pull_relation_id())) { + LOG_WARN("failed to pull relation id and levels", K(ret)); + } + return ret; +} + +// check if target expr already exists in base aggrs, and if so, share the same expression. +int ObTransformConditionalAggrCoalesce::try_share_aggr(ObIArray &base_aggrs, + ObAggFunRawExpr *&target_aggr, + bool &is_sharable, + ObIArray &constraints) +{ + int ret = OB_SUCCESS; + is_sharable = false; + ObExprEqualCheckContext equal_ctx; + equal_ctx.override_const_compare_ = true; + ObPhysicalPlanCtx *plan_ctx = NULL; + if (OB_ISNULL(target_aggr) || OB_ISNULL(ctx_) || OB_ISNULL(ctx_->exec_ctx_) || + OB_ISNULL(plan_ctx = ctx_->exec_ctx_->get_physical_plan_ctx())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } + for (int64_t i = 0; OB_SUCC(ret) && !is_sharable && i < base_aggrs.count(); i++) { + ObAggFunRawExpr *cur_aggr = NULL; + if (OB_ISNULL(cur_aggr = base_aggrs.at(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (cur_aggr->same_as(*target_aggr, &equal_ctx)) { + target_aggr = cur_aggr; + is_sharable = true; + cur_aggr->set_explicited_reference(); + // constraints need to be added if the same_as judgement relies on specific const value + for(int64_t i = 0; OB_SUCC(ret) && i < equal_ctx.param_expr_.count(); i++) { + ObPCConstParamInfo param_info; + int64_t param_idx = equal_ctx.param_expr_.at(i).param_idx_; + if (OB_UNLIKELY(param_idx < 0 || param_idx >= plan_ctx->get_param_store().count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected error", K(ret), K(param_idx), + K(plan_ctx->get_param_store().count())); + } else if (OB_FAIL(param_info.const_idx_.push_back(param_idx))) { + LOG_WARN("failed to push back param idx", K(ret)); + } else if (OB_FAIL(param_info.const_params_.push_back( + plan_ctx->get_param_store().at(param_idx)))) { + LOG_WARN("failed to push back value", K(ret)); + } else if (OB_FAIL(constraints.push_back(param_info))) { + LOG_WARN("failed to push back param info", K(ret)); + } else {/*do nothing*/} + } + } + } + return ret; +} + +// collect view's select exprs according to pseudo columns、aggr items、groupby exprs +int ObTransformConditionalAggrCoalesce::collect_pushdown_select( + ObSelectStmt *select_stmt, + ObIArray &extra_cols, + ObIArray &coalesced_case_exprs, + ObIArray &pushdown_select) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(select_stmt)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (OB_FAIL(ObTransformUtils::pushdown_pseudo_column_like_exprs(*select_stmt, true, + pushdown_select))) { + LOG_WARN("faile to pushdown pseudo column like exprs", K(ret)); + } else if (OB_FAIL(append_array_no_dup(pushdown_select, coalesced_case_exprs))) { + LOG_WARN("faile to pushdown coalesced case exprs", K(ret)); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < select_stmt->get_aggr_item_size(); i++) { + if (OB_FAIL(add_var_to_array_no_dup(pushdown_select, + static_cast(select_stmt->get_aggr_items().at(i))))) { + LOG_WARN("failed to add var", K(ret)); + } + } + } + return ret; +} + +int ObTransformConditionalAggrCoalesce::create_and_replace_aggrs_for_merge(ObSelectStmt *select_stmt, + ObSelectStmt *view_stmt) +{ + int ret = OB_SUCCESS; + ObRawExpr *select_expr = NULL; + TableItem *table = NULL; + ObSEArray cols_for_replace; + ObSEArray aggrs_for_merge; + if (OB_ISNULL(select_stmt) || OB_ISNULL(view_stmt) || OB_ISNULL(ctx_) || + OB_ISNULL(ctx_->expr_factory_) || OB_ISNULL(ctx_->session_info_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (OB_UNLIKELY(1 != select_stmt->get_table_items().count()) + || OB_ISNULL(table = select_stmt->get_table_item(0))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpect select stmt", K(ret), K(select_stmt->get_from_item_size()), K(table)); + } else if (select_stmt->get_aggr_item_size() != 0) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("expect empty aggr items", K(ret), K(select_stmt->get_aggr_item_size())); + } else { + for (int i = 0; OB_SUCC(ret) && i < view_stmt->get_select_item_size(); i++) { + ObRawExpr *col_expr = NULL; + ObAggFunRawExpr* aggr_for_merge = NULL; + ObItemType aggr_type = T_INVALID; + if (OB_ISNULL(select_expr = view_stmt->get_select_item(i).expr_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpect null expr", K(ret)); + } else if (!select_expr->has_flag(CNT_AGG)) { + // do nothing + } else if (OB_FAIL(get_aggr_type(select_expr, aggr_type))) { + LOG_WARN("failed to get aggr type for merge", K(ret)); + } else if (OB_ISNULL(col_expr = select_stmt->get_column_expr_by_id(table->table_id_, + i + OB_APP_MIN_COLUMN_ID))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (OB_FAIL(create_aggr_for_merge(aggr_type, col_expr, aggr_for_merge))) { + LOG_WARN("failed to create aggr for merge", K(ret)); + } else if (OB_FAIL(select_stmt->get_aggr_items().push_back(aggr_for_merge))) { + LOG_WARN("failed to push back expr", K(ret)); + } else if (OB_FAIL(cols_for_replace.push_back(col_expr))) { + LOG_WARN("failed to push back expr", K(ret)); + } else if (OB_FAIL(aggrs_for_merge.push_back(aggr_for_merge))) { + LOG_WARN("failed to push back expr", K(ret)); + } + } + if (OB_SUCC(ret)) { + ObStmtExprReplacer replacer; + replacer.remove_all(); + replacer.add_scope(SCOPE_HAVING); + replacer.add_scope(SCOPE_SELECT); + replacer.add_scope(SCOPE_ORDERBY); + replacer.set_recursive(false); + if (OB_FAIL(replacer.add_replace_exprs(cols_for_replace, aggrs_for_merge))) { + LOG_WARN("failed to add replace exprs", K(ret)); + } else if (OB_FAIL(select_stmt->iterate_stmt_expr(replacer))) { + LOG_WARN("failed to iterate stmt expr", K(ret)); + } + } + } + return ret; +} + +int ObTransformConditionalAggrCoalesce::create_aggr_for_merge(ObItemType aggr_type, + ObRawExpr *param_expr, + ObAggFunRawExpr *&aggr_expr) +{ + int ret = OB_SUCCESS; + ObItemType aggr_type_for_merge; + if (OB_ISNULL(ctx_) || OB_ISNULL(ctx_->session_info_) || OB_ISNULL(ctx_->expr_factory_) || + OB_ISNULL(param_expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (T_FUN_MAX == aggr_type || + T_FUN_MIN == aggr_type || + T_FUN_SUM == aggr_type || + T_FUN_COUNT_SUM == aggr_type || + T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE == aggr_type || + T_FUN_SYS_BIT_AND == aggr_type || + T_FUN_SYS_BIT_OR == aggr_type || + T_FUN_SYS_BIT_XOR == aggr_type) { + aggr_type_for_merge = aggr_type; + } else if (T_FUN_COUNT == aggr_type) { + aggr_type_for_merge = T_FUN_COUNT_SUM; + } else if (T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS == aggr_type) { + aggr_type_for_merge = T_FUN_APPROX_COUNT_DISTINCT_SYNOPSIS_MERGE; + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected aggr type", K(ret), K(aggr_type)); + } + if (OB_SUCC(ret) && OB_FAIL(build_aggr_expr(aggr_type_for_merge, false, param_expr, aggr_expr))) { + LOG_WARN("failed to build aggr expr", K(ret)); + } + return ret; +} + +int ObTransformConditionalAggrCoalesce::refresh_project_name(ObDMLStmt *parent_stmt, + ObSelectStmt *select_stmt) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(parent_stmt) || OB_ISNULL(select_stmt) || + OB_ISNULL(ctx_) || OB_ISNULL(ctx_->allocator_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (parent_stmt == static_cast(select_stmt)) { + // do nothing + } else if (OB_FAIL(ObTransformUtils::refresh_select_items_name(*ctx_->allocator_, select_stmt))) { + LOG_WARN("failed to refresh select items name", K(ret)); + } else { + bool found = false; + for (int64_t i = 0; OB_SUCC(ret) && !found && i < parent_stmt->get_table_size(); ++i) { + TableItem *table_item = parent_stmt->get_table_item(i); + if (OB_ISNULL(table_item)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("table_item is null", K(i)); + } else if (!table_item->is_generated_table() || !table_item->is_temp_table()) { + // do nothing + } else if (table_item->ref_query_ != select_stmt) { + // do nothing + } else if (OB_FAIL(ObTransformUtils::refresh_column_items_name(parent_stmt, + table_item->table_id_))) { + LOG_WARN("failed to refresh column items name", K(ret)); + } else { + found = true; + } + } + } + return ret; +} + +// 这个函数假设输入表达式中的聚合函数类型有且仅有一种 (纯聚合 / case when 聚合) +int ObTransformConditionalAggrCoalesce::get_aggr_type(ObRawExpr* expr, ObItemType &aggr_type) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret)); + } else if (!IS_AGGR_FUN(expr->get_expr_type())) { + // do nothing + } else if (aggr_type != T_INVALID && aggr_type != expr->get_expr_type()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected aggr type", K(ret)); + } else { + aggr_type = expr->get_expr_type(); + } + for (int64_t i = 0; OB_SUCC(ret) && i < expr->get_param_count(); i++) { + if (OB_FAIL(SMART_CALL(get_aggr_type(expr->get_param_expr(i), aggr_type)))) { + LOG_WARN("failed to get aggr type for merge", K(ret)); + } + } + return ret; +} + +int ObTransformConditionalAggrCoalesce::check_hint_valid(ObDMLStmt &stmt, + bool &force_trans_wo_pullup, + bool &force_no_trans_wo_pullup, + bool &force_trans_with_pullup, + bool &force_no_trans_with_pullup) +{ + int ret = OB_SUCCESS; + const ObQueryHint *query_hint = NULL; + const ObCoalesceAggrHint *myhint = static_cast(get_hint(stmt.get_stmt_hint())); + if (OB_ISNULL(query_hint = stmt.get_stmt_hint().query_hint_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret), K(query_hint)); + } else { + force_trans_wo_pullup = NULL != myhint && myhint->enable_trans_wo_pullup(); + force_no_trans_wo_pullup = !force_trans_wo_pullup && query_hint->has_outline_data(); + force_trans_with_pullup = NULL != myhint && myhint->enable_trans_with_pullup(); + force_no_trans_with_pullup = !force_trans_with_pullup && query_hint->has_outline_data(); + } + return ret; +} + +int ObTransformConditionalAggrCoalesce::construct_transform_hint(ObDMLStmt &stmt, void *trans_params) +{ + int ret = OB_SUCCESS; + ObCoalesceAggrHint *hint = NULL; + TransFlagPair *trans_flags = NULL; + const ObQueryHint *query_hint = NULL; + if (OB_ISNULL(ctx_) || OB_ISNULL(ctx_->allocator_) || + OB_ISNULL(query_hint = stmt.get_stmt_hint().query_hint_) || + OB_ISNULL(trans_flags = static_cast(trans_params))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret), K(ctx_), K(query_hint)); + } else if (OB_FAIL(ObQueryHint::create_hint(ctx_->allocator_, T_COALESCE_AGGR, hint))) { + LOG_WARN("failed to create hint", K(ret)); + } else { + hint->set_enable_trans_wo_pullup(trans_flags->first); + hint->set_enable_trans_with_pullup(trans_flags->second); + const ObCoalesceAggrHint *myhint = static_cast(get_hint(stmt.get_stmt_hint())); + bool use_hint = NULL != myhint && ((myhint->enable_trans_wo_pullup() && trans_flags->first) || + (myhint->enable_trans_with_pullup() && trans_flags->second)); + if (OB_FAIL(ctx_->outline_trans_hints_.push_back(hint))) { + LOG_WARN("failed to push back hint", K(ret)); + } else if (use_hint && OB_FAIL(ctx_->add_used_trans_hint(myhint))) { + LOG_WARN("failed to add used trans hint", K(ret)); + } else { + hint->set_qb_name(ctx_->src_qb_name_); + } + } + return ret; +} + +} /* namespace sql */ +} /* namespace oceanbase */ \ No newline at end of file diff --git a/src/sql/rewrite/ob_transform_conditional_aggr_coalesce.h b/src/sql/rewrite/ob_transform_conditional_aggr_coalesce.h new file mode 100644 index 0000000000..bca08b91e9 --- /dev/null +++ b/src/sql/rewrite/ob_transform_conditional_aggr_coalesce.h @@ -0,0 +1,175 @@ +/** + * Copyright (c) 2021 OceanBase + * OceanBase CE is licensed under Mulan PubL v2. + * You can use this software according to the terms and conditions of the Mulan PubL v2. + * You may obtain a copy of Mulan PubL v2 at: + * http://license.coscl.org.cn/MulanPubL-2.0 + * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, + * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, + * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. + * See the Mulan PubL v2 for more details. + */ + +#ifndef _OB_TRANSFORM_CONDITIONAL_AGGR_COALESCE_H_ +#define _OB_TRANSFORM_CONDITIONAL_AGGR_COALESCE_H_ 1 + +#include "sql/rewrite/ob_transform_rule.h" +#include "sql/resolver/dml/ob_select_stmt.h" + +namespace oceanbase +{ +namespace sql +{ +/** @brief ObTransformConditionalAggrCoalesce attempts to coalesce conditional aggregation functions +* "aggr_func(case_expr)" with similar structure, in order to reduce the number of aggregation +* functions and the calculation times of case expr. +* +* There are two rewrite paths: [trans_without_pullup] and [trans_with_pullup]. +* e.g. [trans_without_pullup] +* select count(distinct case when c1 = 0 then 0 else c2 end), +* count(distinct case when c1 = 1 then 0 else c2 end), +* count(distinct case when c1 = 2 then 0 else c2 end) +* from t1 group by c1; +* +* ==> coalesce +* +* select case when c1 = 0 then count(distinct 0) else count(distinct c2) end, +* case when c1 = 1 then count(distinct 0) else count(distinct c2) end, +* case when c1 = 2 then count(distinct 0) else count(distinct c2) end +* from t1 group by c1; +* +* e.g. [trans_with_pullup] +* select count(case when c1 = 0 then 0 else c2 end), +* count(case when c1 = 1 then 0 else c2 end), +* count(case when c1 = 2 then 0 else c2 end) +* from t1; +* +* ==> coalesce + pullup +* +* select count_sum(v.s1), count_sum(v.s2), count_sum(v.s3) +* from ( +* select case when c1 = 0 then count(0) else count(c2) end as s1, +* case when c1 = 1 then count(0) else count(c2) end as s2, +* case when c1 = 2 then count(0) else count(c2) end as s3 +* from t1 +* group by c1) v; +*/ +class ObTransformConditionalAggrCoalesce : public ObTransformRule +{ +public: + static const int64_t MIN_COND_AGGR_CNT_FOR_COALESCE; + static const int64_t MAX_GBY_NDV_PRODUCT_FOR_COALESCE; + static const double MIN_CUT_RATIO_FOR_COALESCE; + + explicit ObTransformConditionalAggrCoalesce(ObTransformerCtx *ctx) + : ObTransformRule(ctx, TransMethod::POST_ORDER, T_COALESCE_AGGR) {} + virtual ~ObTransformConditionalAggrCoalesce() {} + virtual int transform_one_stmt(common::ObIArray &parent_stmts, + ObDMLStmt *&stmt, + bool &trans_happened) override; + virtual int construct_transform_hint(ObDMLStmt &stmt, void *trans_params) override; + +private: + typedef std::pair TransFlagPair; + struct TransformParam { + TransformParam() : cond_aggrs_wo_extra_dep_(), cond_aggrs_with_extra_dep_(), + extra_dep_cols_() { } + ~TransformParam() {} + TO_STRING_KV(K_(cond_aggrs_wo_extra_dep), K_(cond_aggrs_with_extra_dep), K_(extra_dep_cols)); + + // cond aggrs whose when exprs do not rely on columns beyond group by clause + ObSEArray cond_aggrs_wo_extra_dep_; + // cond aggrs whose when exprs rely on columns beyond group by clause + ObSEArray cond_aggrs_with_extra_dep_; + // extra dependent columns of cond_aggrs_with_extra_dep_ + ObSEArray extra_dep_cols_; + }; + int check_hint_valid(ObDMLStmt &stmt, + bool &force_trans_wo_pullup, + bool &force_no_trans_wo_pullup, + bool &force_trans_with_pullup, + bool &force_no_trans_with_pullup); + int check_basic_validity(ObDMLStmt *stmt, + TransformParam &trans_param, + ObSelectStmt *&select_stmt, + bool &valid_wo_pullup, + bool &valid_with_pullup); + int collect_cond_aggrs_info(ObSelectStmt *select_stmt, + TransformParam &trans_param, + bool &cnt_unpullupable_aggr); + bool check_aggr_type(ObItemType aggr_type); + int check_cond_aggr_form(ObAggFunRawExpr *aggr_expr, + ObCaseOpRawExpr *&case_expr, + bool &is_cond_aggr); + int check_case_when_validity(ObSelectStmt *select_stmt, + ObAggFunRawExpr *cond_aggr, + ObCaseOpRawExpr *case_expr, + bool &is_case_when_valid); + int extract_extra_dep_cols(ObIArray &target_exprs, + ObIArray &exclude_exprs, + ObIArray &extra_dep_cols); + int inner_extract_extra_dep_cols(ObRawExpr *target_expr, + ObIArray &exclude_exprs, + ObIArray &extra_dep_cols); + int try_transform_wo_pullup(ObSelectStmt *select_stmt, + ObDMLStmt *parent_stmt, + bool force_trans, + TransformParam &trans_param, + bool &trans_happened); + int try_transform_with_pullup(ObSelectStmt *select_stmt, + ObDMLStmt *parent_stmt, + bool force_trans, + TransformParam &trans_param, + bool &trans_happened); + int coalesce_cond_aggrs(ObIArray &base_aggrs, + ObIArray &cond_aggrs, + ObIArray &case_exprs, + ObIArray &new_aggrs, + ObIArray &constraints); + int build_aggr_expr(ObItemType expr_type, + bool is_param_distinct, + ObRawExpr *param_expr, + ObAggFunRawExpr *&aggr_expr); + int try_share_aggr(ObIArray &base_aggrs, + ObAggFunRawExpr *&target_aggr, + bool &is_shareable, + ObIArray &constraints); + int check_statistics_threshold(ObSelectStmt *select_stmt, + TransformParam &trans_param, + bool &hit_threshold); + int collect_pushdown_select(ObSelectStmt *select_stmt, + ObIArray &extra_cols, + ObIArray &coalesced_case_exprs, + ObIArray &pushdown_select); + int create_and_replace_aggrs_for_merge(ObSelectStmt *select_stmt, ObSelectStmt *view_stmt); + int create_aggr_for_merge(ObItemType aggr_type, + ObRawExpr *param_expr, + ObAggFunRawExpr *&aggr_expr); + int check_aggrs_count_decrease(ObIArray &old_aggrs, + ObIArray &new_aggrs, + bool &is_cnt_decrease); + int check_rewrite_gain(TableItem* base_table, + ObIArray &cols_in_groupby, + int64_t cond_aggr_cnt, + bool ignore_threshold, + double &gain); + int do_transform_wo_pullup(ObSelectStmt *select_stmt, + ObIArray &cond_aggrs, + ObIArray &coalesced_case_exprs, + ObIArray &new_aggr_items, + ObIArray &constraints); + int do_transform_with_pullup(ObSelectStmt *select_stmt, + ObIArray &cond_aggrs, + ObIArray &extra_dep_cols, + ObIArray &coalesced_case_exprs, + ObIArray &new_aggr_items, + ObIArray &constraints); + int refresh_project_name(ObDMLStmt *parent_stmt, ObSelectStmt *select_stmt); + int get_aggr_type(ObRawExpr* expr, ObItemType &aggr_type); + DISALLOW_COPY_AND_ASSIGN(ObTransformConditionalAggrCoalesce); + +}; + +} //namespace sql +} //namespace oceanbase +#endif /* _OB_TRANSFORM_CONDITIONAL_AGGR_COALESCE_H_ */ \ No newline at end of file diff --git a/src/sql/rewrite/ob_transform_const_propagate.cpp b/src/sql/rewrite/ob_transform_const_propagate.cpp index 3dde8154ed..600a3fbf30 100644 --- a/src/sql/rewrite/ob_transform_const_propagate.cpp +++ b/src/sql/rewrite/ob_transform_const_propagate.cpp @@ -1493,6 +1493,12 @@ int ObTransformConstPropagate::check_cast_const_expr(ExprConstInfo &const_info, need_cast, is_scale_adjust_cast))) { LOG_WARN("failed to check need cast expr", K(ret)); + if (OB_ERR_INVALID_TYPE_FOR_OP == ret) { // rewrite should not happened instead of failed a error + ret = OB_SUCCESS; + is_valid = false; + } else { + LOG_WARN("failed to check need cast expr", K(ret)); + } } else if (!need_cast) { // do nothing } else if (OB_FAIL(ObSQLUtils::calc_const_or_calculable_expr(ctx_->exec_ctx_, @@ -1515,7 +1521,12 @@ int ObTransformConstPropagate::check_cast_const_expr(ExprConstInfo &const_info, ObObjType cmp_type = ObMaxType; int64_t eq_cmp = 0; if (OB_FAIL(ret)) { - LOG_WARN("failed to cast obj to dest type", K(ret), K(value), K(dst_type.get_type())); + if (OB_ERR_INVALID_DATATYPE == ret) { // rewrite should not happened instead of failed a error + ret = OB_SUCCESS; + is_valid = false; + } else { + LOG_WARN("failed to cast obj to dest type", K(ret), K(value), K(dst_type.get_type())); + } } else if (OB_FAIL(ObExprResultTypeUtil::get_relational_cmp_type(cmp_type, value.get_type(), dest_val->get_type()))) { diff --git a/src/sql/rewrite/ob_transform_join_limit_pushdown.cpp b/src/sql/rewrite/ob_transform_join_limit_pushdown.cpp index 04b5c4fbff..9babe19655 100644 --- a/src/sql/rewrite/ob_transform_join_limit_pushdown.cpp +++ b/src/sql/rewrite/ob_transform_join_limit_pushdown.cpp @@ -335,12 +335,14 @@ int ObTransformJoinLimitPushDown::split_cartesian_tables(ObSelectStmt *select_st if (OB_ISNULL(select_stmt)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); - } else if (OB_FAIL(check_contain_correlated_function_table(select_stmt, is_contain))) { + } else if (OB_FAIL(ObTransformUtils::check_contain_correlated_function_table(select_stmt, + is_contain))) { LOG_WARN("failed to check contain correlated function table", K(ret)); } else if (is_contain) { OPT_TRACE("contain correlated function table, do not push down limit"); - } else if (OB_FAIL(check_contain_correlated_json_table(select_stmt, is_contain))) { - LOG_WARN("failed to check contain correlated function table", K(ret)); + } else if (OB_FAIL(ObTransformUtils::check_contain_correlated_json_table(select_stmt, + is_contain))) { + LOG_WARN("failed to check contain correlated json table", K(ret)); } else if (is_contain) { OPT_TRACE("contain correlated json table, do not push down limit"); } else if (OB_FAIL(ObTransformUtils::check_contain_correlated_lateral_table(select_stmt, @@ -368,48 +370,6 @@ int ObTransformJoinLimitPushDown::split_cartesian_tables(ObSelectStmt *select_st return ret; } -int ObTransformJoinLimitPushDown::check_contain_correlated_function_table(ObDMLStmt *stmt, bool &is_contain) -{ - int ret = OB_SUCCESS; - is_contain = false; - for (int i = 0; OB_SUCC(ret) && !is_contain && i < stmt->get_table_items().count(); ++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_function_table()) { - //do nothing - } else if (OB_ISNULL(table->function_table_expr_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpect null expr", K(ret)); - } else if (!table->function_table_expr_->get_relation_ids().is_empty()) { - is_contain = true; - } - } - return ret; -} - -int ObTransformJoinLimitPushDown::check_contain_correlated_json_table(ObDMLStmt *stmt, bool &is_contain) -{ - int ret = OB_SUCCESS; - is_contain = false; - for (int i = 0; OB_SUCC(ret) && !is_contain && i < stmt->get_table_items().count(); ++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_json_table()) { - //do nothing - } else if (OB_ISNULL(table->json_table_def_) || OB_ISNULL(table->json_table_def_->doc_expr_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpect null expr", K(ret)); - } else if (!table->json_table_def_->doc_expr_->get_relation_ids().is_empty()) { - is_contain = true; - } - } - return ret; -} - int ObTransformJoinLimitPushDown::check_cartesian(ObSelectStmt *stmt, UnionFind &uf, bool &is_valid) @@ -447,7 +407,7 @@ int ObTransformJoinLimitPushDown::check_cartesian(ObSelectStmt *stmt, } else if (OB_FAIL(ObRawExprUtils::extract_table_ids(cond, where_table_ids))) { LOG_WARN("failed to extract table ids", K(ret)); - } else if (OB_FAIL(connect_tables(stmt, where_table_ids, from_tables, uf))) { + } else if (OB_FAIL(ObTransformUtils::connect_tables(where_table_ids, from_tables, uf))) { LOG_WARN("failed to connect tables", K(ret)); } } @@ -458,7 +418,7 @@ int ObTransformJoinLimitPushDown::check_cartesian(ObSelectStmt *stmt, if (OB_ISNULL(semi)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); - } else if (OB_FAIL(connect_tables(stmt, semi->left_table_ids_, from_tables, uf))) { + } else if (OB_FAIL(ObTransformUtils::connect_tables(semi->left_table_ids_, from_tables, uf))) { LOG_WARN("failed to connect tables", K(ret)); } } @@ -481,7 +441,7 @@ int ObTransformJoinLimitPushDown::check_cartesian(ObSelectStmt *stmt, // do nothing } else if (OB_FAIL(ObRawExprUtils::extract_table_ids(expr, orderby_table_ids))) { LOG_WARN("failed to collect orderby table sets", K(ret)); - } else if (OB_FAIL(connect_tables(stmt, orderby_table_ids, from_tables, uf))) { + } else if (OB_FAIL(ObTransformUtils::connect_tables(orderby_table_ids, from_tables, uf))) { LOG_WARN("failed to connect tables", K(ret)); } } @@ -490,67 +450,6 @@ int ObTransformJoinLimitPushDown::check_cartesian(ObSelectStmt *stmt, return ret; } -int ObTransformJoinLimitPushDown::connect_tables(ObSelectStmt *stmt, - const ObIArray &table_ids, - const ObIArray &from_tables, - UnionFind &uf) -{ - int ret = OB_SUCCESS; - ObSEArray indices; - if (OB_ISNULL(stmt)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("get unexpected null", K(ret)); - } else if (OB_FAIL(get_idx_from_table_ids(stmt, - table_ids, - from_tables, - indices))) { - LOG_WARN("failed to get indices of from table ids", K(ret)); - } else if (indices.count() <= 1) { - // do nothing - } else { - // connect tables appeared in a condition - // we store table indices instead of table item to leverage the consumption - for (int64_t i = 1; OB_SUCC(ret) && i < indices.count(); ++i) { - if (OB_FAIL(uf.connect(indices.at(0), indices.at(i)))) { - LOG_WARN("failed to connect nodes", K(ret), K(i)); - } - } - } - return ret; -} - -int ObTransformJoinLimitPushDown::get_idx_from_table_ids(ObSelectStmt *stmt, - const ObIArray &src_table_ids, - const ObIArray &target_tables, - ObIArray &indices) -{ - int ret = OB_SUCCESS; - for (int64_t i = 0; OB_SUCC(ret) && i < target_tables.count(); ++i) { - TableItem *target_table = target_tables.at(i); - if (OB_ISNULL(target_table)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("get unexpected null", K(ret)); - } - for (int64_t j = 0; OB_SUCC(ret) && j < src_table_ids.count(); ++j) { - if (src_table_ids.at(j) == target_table->table_id_) { - if (OB_FAIL(indices.push_back(i))) { - LOG_WARN("failed to push back index", K(ret)); - } - break; - } else if (target_table->is_joined_table()) { - if (is_contain(static_cast(target_table)->single_table_ids_, - src_table_ids.at(j))) { - if (!is_contain(indices, i) && OB_FAIL(indices.push_back(i))) { - LOG_WARN("failed to push back index", K(ret)); - } - break; - } - } - } - } - return ret; -} - int ObTransformJoinLimitPushDown::collect_cartesian_infos(ObSelectStmt *stmt, UnionFind &uf, ObIArray &helpers) diff --git a/src/sql/rewrite/ob_transform_join_limit_pushdown.h b/src/sql/rewrite/ob_transform_join_limit_pushdown.h index c4f50cb2a5..3da7dcee4d 100644 --- a/src/sql/rewrite/ob_transform_join_limit_pushdown.h +++ b/src/sql/rewrite/ob_transform_join_limit_pushdown.h @@ -113,22 +113,12 @@ private: int check_cartesian(ObSelectStmt *stmt, UnionFind &uf, bool &is_valid); - int connect_tables(ObSelectStmt *stmt, - const ObIArray &table_ids, - const ObIArray &from_tables, - UnionFind &uf); - int collect_cartesian_infos(ObSelectStmt *stmt, UnionFind &uf, ObIArray &helpers); int collect_cartesian_exprs(ObSelectStmt *stmt, LimitPushDownHelper *helper); - int get_idx_from_table_ids(ObSelectStmt *stmt, - const ObIArray &src_table_ids, - const ObIArray &target_tables, - ObIArray &indices); - int check_table_validity(const ObIArray &target_tables, bool &is_valid); int collect_pushdown_exprs(ObSelectStmt *stmt, diff --git a/src/sql/rewrite/ob_transform_min_max.cpp b/src/sql/rewrite/ob_transform_min_max.cpp index 57df69a9ea..a8c3a2c520 100644 --- a/src/sql/rewrite/ob_transform_min_max.cpp +++ b/src/sql/rewrite/ob_transform_min_max.cpp @@ -44,12 +44,11 @@ ObTransformMinMax::~ObTransformMinMax() } int ObTransformMinMax::transform_one_stmt(common::ObIArray &parent_stmts, - ObDMLStmt *&stmt, - bool &trans_happened) + ObDMLStmt *&stmt, + bool &trans_happened) { int ret = OB_SUCCESS; bool is_valid = false; - ObAggFunRawExpr *aggr_expr = NULL; trans_happened = false; UNUSED(parent_stmts); if (OB_ISNULL(stmt) || OB_ISNULL(ctx_)) { @@ -60,13 +59,12 @@ int ObTransformMinMax::transform_one_stmt(common::ObIArray &par OPT_TRACE("not select stmt"); } else if (OB_FAIL(check_transform_validity(*ctx_, static_cast(stmt), - aggr_expr, is_valid))) { LOG_WARN("failed to check transform validity", K(ret)); } else if (!is_valid) { //do nothing - OPT_TRACE("can not transform"); - } else if (OB_FAIL(do_transform(static_cast(stmt), aggr_expr))) { + OPT_TRACE("can not do min max transform"); + } else if (OB_FAIL(do_transform(static_cast(stmt)))) { LOG_WARN("failed to transform column aggregate", K(ret)); } else if (OB_FAIL(add_transform_hint(*stmt))) { LOG_WARN("failed to add transform hint", K(ret)); @@ -78,12 +76,9 @@ int ObTransformMinMax::transform_one_stmt(common::ObIArray &par int ObTransformMinMax::check_transform_validity(ObTransformerCtx &ctx, ObSelectStmt *select_stmt, - ObAggFunRawExpr *&aggr_expr, bool &is_valid) { int ret = OB_SUCCESS; - const ObAggFunRawExpr *expr = NULL; - aggr_expr = NULL; is_valid = false; bool has_rownum = false; if (OB_ISNULL(select_stmt)) { @@ -91,54 +86,76 @@ int ObTransformMinMax::check_transform_validity(ObTransformerCtx &ctx, LOG_WARN("unexpected null", K(ret), K(select_stmt)); } else if (select_stmt->has_recursive_cte() || select_stmt->has_hierarchical_query()) { OPT_TRACE("stmt has recusive cte or hierarchical query"); - } else if (select_stmt->get_from_item_size() != 1 || - select_stmt->get_from_item(0).is_joined_ || - select_stmt->get_aggr_item_size() != 1 || - !select_stmt->is_scala_group_by()) { - OPT_TRACE("not a simple query"); + } else if (select_stmt->get_from_item_size() != 1 || select_stmt->get_from_item(0).is_joined_ + || select_stmt->get_aggr_item_size() < 1 || !select_stmt->is_scala_group_by() + || select_stmt->is_contains_assignment() + || (select_stmt->get_aggr_item_size() > 1 && select_stmt->get_semi_info_size() > 0)) { + OPT_TRACE("not a simple aggr query"); } else if (OB_FAIL(select_stmt->has_rownum(has_rownum))) { LOG_WARN("failed to check if select stmt has rownum", K(ret)); } else if (has_rownum) { OPT_TRACE("stmt has rownum"); - } else if (OB_ISNULL(expr = select_stmt->get_aggr_items().at(0))) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("params have null", K(ret), KP(expr)); - } 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(ctx, select_stmt, expr->get_param_expr(0), is_valid))) { - LOG_WARN("failed to check is valid index column", K(ret)); + } else if (OB_FAIL(is_valid_aggr_items(ctx, *select_stmt, is_valid))) { + LOG_WARN("failed to check aggr items", K(ret)); } else if (!is_valid) { - OPT_TRACE("aggr expr is not include index column"); - } else if (OB_FAIL(is_valid_select_list(*select_stmt, expr, is_valid))) { + OPT_TRACE("aggr expr is invalid"); + } else if (OB_FAIL(is_valid_select_list(*select_stmt, is_valid))) { LOG_WARN("failed to check is valid select list", K(ret)); } else if (!is_valid) { - OPT_TRACE("select list is const or aggr_expr"); - } else if (OB_FAIL(is_valid_having(select_stmt, expr, is_valid))) { - LOG_WARN("fail to check is valid having", K(ret)); + OPT_TRACE("select list is not const or aggr_expr"); + } else if (OB_FAIL(is_valid_having_list(*select_stmt, is_valid))) { + LOG_WARN("failed to check is valid having", K(ret)); } else if (!is_valid) { OPT_TRACE("having condition is invalid"); + } else if (select_stmt->get_aggr_item_size() > 1 && OB_FAIL(is_valid_order_list(*select_stmt, is_valid))) { + LOG_WARN("failed to check is valid order by", K(ret)); + } else if (!is_valid) { + OPT_TRACE("order by is invalid for multi min/max"); } else { - aggr_expr = select_stmt->get_aggr_items().at(0); - LOG_TRACE("Succeed to check transform validity", K(is_valid)); + LOG_TRACE("Succeed to check minmax transform validity", K(is_valid)); } return ret; } -int ObTransformMinMax::do_transform(ObSelectStmt *select_stmt, ObAggFunRawExpr *aggr_expr) +int ObTransformMinMax::do_transform(ObSelectStmt *select_stmt) { int ret = OB_SUCCESS; - if (OB_ISNULL(select_stmt) || OB_ISNULL(aggr_expr) || - OB_ISNULL(ctx_) || OB_ISNULL(ctx_->expr_factory_)) { + if (OB_ISNULL(select_stmt)) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("params have null", K(ret), K(select_stmt), K(aggr_expr), K(ctx_)); + LOG_WARN("params have null", K(ret), K(select_stmt)); + } else if (select_stmt->get_aggr_item_size() == 1) { + if (OB_FAIL(do_single_minmax_transform(select_stmt))) { + LOG_WARN("failed to transform single minmax", K(ret)); + } + } else if (select_stmt->get_aggr_item_size() > 1) { + if (OB_FAIL(do_multi_minmax_transform(select_stmt))) { + LOG_WARN("failed to transform multi minmax", K(ret)); + } + } + return ret; +} + +int ObTransformMinMax::do_single_minmax_transform(ObSelectStmt *select_stmt) +{ + int ret = OB_SUCCESS; + ObRawExpr *aggr_param = NULL; + ObRawExpr *new_aggr_param = NULL; + ObSelectStmt *child_stmt = NULL; + ObSEArray old_exprs; + ObSEArray new_exprs; + if (OB_ISNULL(select_stmt) || OB_ISNULL(ctx_) || OB_ISNULL(ctx_->expr_factory_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("params have null", K(ret), K(select_stmt), K(ctx_)); + } else if (select_stmt->get_aggr_item_size() != 1) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected param", K(ret)); } else { - ObSelectStmt *child_stmt = NULL; - ObRawExpr *new_tmp_aggr_expr = NULL; - ObSEArray old_exprs; - ObSEArray new_exprs; + ObAggFunRawExpr *aggr_expr = select_stmt->get_aggr_item(0); ObRawExprCopier copier(*ctx_->expr_factory_); - if (OB_FAIL(ObTransformUtils::create_simple_view(ctx_, select_stmt, child_stmt))) { + if (OB_ISNULL(aggr_expr) || OB_ISNULL(aggr_param = aggr_expr->get_param_expr(0))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("aggr expr unexpected null", K(ret)); + } else if (OB_FAIL(ObTransformUtils::create_simple_view(ctx_, select_stmt, child_stmt))) { LOG_WARN("failed to create simple view", K(ret)); } else if (OB_FAIL(select_stmt->get_column_exprs(old_exprs))) { LOG_WARN("failed to get column exprs", K(ret)); @@ -146,14 +163,14 @@ int ObTransformMinMax::do_transform(ObSelectStmt *select_stmt, ObAggFunRawExpr * LOG_WARN("failed to get select exprs", K(ret)); } else if (OB_FAIL(copier.add_replaced_expr(old_exprs, new_exprs))) { LOG_WARN("failed to add replace pair", K(ret)); - } else if (OB_FAIL(copier.copy(aggr_expr, new_tmp_aggr_expr))) { + } else if (OB_FAIL(copier.copy(aggr_param, new_aggr_param))) { LOG_WARN("failed to copy expr", K(ret)); - } else if (OB_FAIL(set_child_condition(child_stmt, new_tmp_aggr_expr))) { - LOG_WARN("fail to set child condition", K(ret)); - } else if (OB_FAIL(set_child_order_item(child_stmt, new_tmp_aggr_expr))) { - LOG_WARN("fail to set child order item", K(ret)); + } else if (OB_FAIL(set_child_condition(child_stmt, new_aggr_param))) { + LOG_WARN("failed to set child condition", K(ret)); + } else if (OB_FAIL(set_child_order_item(child_stmt, new_aggr_param, aggr_expr->get_expr_type()))) { + LOG_WARN("failed to set child order item", K(ret)); } else if (OB_FAIL(ObTransformUtils::set_limit_expr(child_stmt, ctx_))) { - LOG_WARN("fail to set child limit item", K(ret)); + LOG_WARN("failed to set child limit item", K(ret)); } else { LOG_TRACE("Succeed to do transform min max", K(*select_stmt)); } @@ -161,182 +178,378 @@ int ObTransformMinMax::do_transform(ObSelectStmt *select_stmt, ObAggFunRawExpr * return ret; } -int ObTransformMinMax::is_valid_select_list(const ObSelectStmt &stmt, - const ObAggFunRawExpr *aggr_expr, - bool &is_valid) +int ObTransformMinMax::do_multi_minmax_transform(ObSelectStmt *select_stmt) { int ret = OB_SUCCESS; - is_valid = false; - if (OB_ISNULL(aggr_expr)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("params have null", K(ret), KP(aggr_expr)); + ObSelectStmt *view_child_stmt = NULL; + ObSelectStmt *child_stmt = NULL; + ObAggFunRawExpr *aggr_expr = NULL; + ObRawExpr *aggr_param = NULL; + ObRawExpr *new_aggr_param = NULL; + ObSEArray old_exprs; + ObSEArray new_exprs; + ObArray aggr_items; + ObArray query_ref_exprs; + ObQueryRefRawExpr *query_ref_expr = NULL; + ObRawExpr *target_expr = NULL; + if (OB_ISNULL(select_stmt) || OB_ISNULL(ctx_) || OB_ISNULL(ctx_->expr_factory_)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("params have null", K(ret), K(select_stmt), K(ctx_)); } else { - for (int64_t i = 0; OB_SUCC(ret) && i < stmt.get_select_item_size(); ++i) { - const ObRawExpr *expr = stmt.get_select_item(i).expr_; - if (OB_ISNULL(expr)) { + ObRawExprCopier copier(*ctx_->expr_factory_); + if (OB_FAIL(ObTransformUtils::create_simple_view(ctx_, select_stmt, view_child_stmt))) { + LOG_WARN("failed to create simple view", K(ret)); + } else if (OB_FAIL(select_stmt->get_column_exprs(old_exprs))) { + LOG_WARN("failed to get column exprs", K(ret)); + } else if (OB_FAIL(view_child_stmt->get_select_exprs(new_exprs))) { + LOG_WARN("failed to get select exprs", K(ret)); + } else if (OB_FAIL(copier.add_replaced_expr(old_exprs, new_exprs))) { + LOG_WARN("failed to add replace pair", K(ret)); + } + for (int64_t i = 0; OB_SUCC(ret) && i < select_stmt->get_aggr_item_size(); ++i) { + if (OB_ISNULL(aggr_expr = select_stmt->get_aggr_item(i)) + || OB_ISNULL(aggr_param = aggr_expr->get_param_expr(0))) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("expr should not be NULL", K(ret)); - } else if (expr->is_const_expr()) { - /* do nothing */ - } else if (OB_FAIL(is_valid_aggr_expr(stmt, expr, aggr_expr, is_valid))) { - LOG_WARN("failed to check expr is valid aggr", K(ret)); - } else if (!is_valid) { - break; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(copier.copy(aggr_param, new_aggr_param))) { + LOG_WARN("failed to copy expr", K(ret)); + } else if (OB_FAIL(deep_copy_subquery_for_aggr(*view_child_stmt, + new_aggr_param, + aggr_expr->get_expr_type(), + child_stmt))) { + LOG_WARN("failed to deep copy subquery for aggr", K(ret)); + } else if (OB_FAIL(aggr_items.push_back(aggr_expr))) { + LOG_WARN("failed to push back aggr item", K(ret)); + } else if (OB_FAIL(ctx_->expr_factory_->create_raw_expr(T_REF_QUERY, query_ref_expr))) { + LOG_WARN("failed to create query ref expr", K(ret)); + } else if (OB_ISNULL(query_ref_expr) || OB_ISNULL(child_stmt) + || OB_UNLIKELY(child_stmt->get_select_item_size() != 1) + || OB_ISNULL(target_expr = child_stmt->get_select_item(0).expr_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null expr or select item size of child stmt", K(ret)); + } else { + query_ref_expr->set_ref_stmt(child_stmt); + query_ref_expr->set_output_column(1); + if (OB_FAIL(query_ref_expr->add_column_type(target_expr->get_result_type()))) { + LOG_WARN("add column type to subquery ref expr failed", K(ret)); + } else if (OB_FAIL(query_ref_expr->formalize(ctx_->session_info_))) { + LOG_WARN("failed to formalize coalesce query expr", K(ret)); + } else if (OB_FAIL(query_ref_exprs.push_back(query_ref_expr))) { + LOG_WARN("failed to push back query ref expr", K(ret)); + } + } + } + // adjust select_stmt + if (OB_FAIL(ret)) { + } else if (OB_FAIL(select_stmt->replace_relation_exprs(aggr_items, query_ref_exprs))) { + LOG_WARN("failed to replace aggr exprs to query ref exprs", K(ret)); + } else if (OB_FAIL(select_stmt->get_condition_exprs().assign(select_stmt->get_having_exprs()))) { + LOG_WARN("failed to assign condition exprs", K(ret)); + } else { + select_stmt->get_from_items().reset(); + select_stmt->get_table_items().reset(); + select_stmt->get_aggr_items().reset(); + select_stmt->get_having_exprs().reset(); + select_stmt->get_column_items().reset(); + if (OB_FAIL(select_stmt->adjust_subquery_list())) { + LOG_WARN("failed to adjust subquery list", K(ret)); + } else if (OB_FAIL(select_stmt->formalize_stmt(ctx_->session_info_))) { + LOG_WARN("failed to formalize stmt", K(ret)); + } else { + LOG_TRACE("succeed to do transform min max", KPC(select_stmt)); } } } return ret; } -int ObTransformMinMax::is_valid_aggr_expr(const ObSelectStmt &stmt, - const ObRawExpr *expr, - const ObAggFunRawExpr *aggr_expr, - bool &is_valid) +int ObTransformMinMax::deep_copy_subquery_for_aggr(const ObSelectStmt &copied_stmt, + ObRawExpr *aggr_param, + ObItemType aggr_type, + ObSelectStmt *&child_stmt) +{ + int ret = OB_SUCCESS; + ObDMLStmt *dml_stmt = NULL; + ObRawExpr *new_aggr_param = NULL; + ObString qb_name; + TableItem *table = NULL; + ObArray select_exprs; + ObArray new_select_exprs; + int64_t new_aggr_param_index = -1; + if (OB_ISNULL(ctx_) || OB_ISNULL(ctx_->expr_factory_) || OB_ISNULL(ctx_->stmt_factory_) + || OB_ISNULL(ctx_->allocator_) || OB_ISNULL(aggr_param)) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("params have null", K(ret), K(aggr_param), K(ctx_)); + } else if (OB_FAIL(ObTransformUtils::deep_copy_stmt(*ctx_->stmt_factory_, + *ctx_->expr_factory_, + &copied_stmt, + dml_stmt))) { + LOG_WARN("failed to deep copy stmt", K(ret)); + } else if (OB_ISNULL(child_stmt = static_cast(dml_stmt))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("child stmt is null", K(ret)); + } else if (OB_FAIL(child_stmt->adjust_statement_id(ctx_->allocator_, + ctx_->src_qb_name_, + ctx_->src_hash_val_))) { + LOG_WARN("failed to adjust statement id", K(ret)); + } else if (OB_FAIL(child_stmt->get_qb_name(qb_name))) { + LOG_WARN("failed to get qb name", K(ret)); + } else if (child_stmt->get_table_size() != 1 + || OB_ISNULL(table = child_stmt->get_table_item(0))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected table size or table item is null", K(ret), KPC(child_stmt)); + } else if (OB_FALSE_IT(table->qb_name_ = qb_name)) { + } else if (OB_FAIL(child_stmt->update_stmt_table_id(ctx_->allocator_, copied_stmt))) { + LOG_WARN("failed to update stmt table id", K(ret)); + } else if (OB_FAIL(copied_stmt.get_select_exprs(select_exprs))) { + LOG_WARN("failed to get select exprs", K(ret)); + } else if (!ObOptimizerUtil::find_item(select_exprs, aggr_param, &new_aggr_param_index)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed to find new aggr param", K(ret)); + } else if (OB_FAIL(child_stmt->get_select_exprs(new_select_exprs))) { + LOG_WARN("failed to get select exprs", K(ret)); + } else if (new_aggr_param_index < 0 || new_aggr_param_index >= new_select_exprs.count() + || OB_ISNULL(new_aggr_param = new_select_exprs.at(new_aggr_param_index))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("failed to find new aggr param", K(ret)); + } else { + child_stmt->get_select_items().reset(); + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(ObTransformUtils::create_select_item(*ctx_->allocator_, + new_aggr_param, + child_stmt))) { + LOG_WARN("failed to create select item", K(ret)); + } else if (OB_FAIL(set_child_condition(child_stmt, new_aggr_param))) { + LOG_WARN("failed to set child condition", K(ret)); + } else if (OB_FAIL(set_child_order_item(child_stmt, new_aggr_param, aggr_type))) { + LOG_WARN("failed to set child order item", K(ret)); + } else if (OB_FAIL(ObTransformUtils::set_limit_expr(child_stmt, ctx_))) { + LOG_WARN("failed to set child limit item", K(ret)); + } else if (OB_FAIL(child_stmt->rebuild_tables_hash())) { + LOG_WARN("failed to rebuild table hash", K(ret)); + } else if (OB_FAIL(child_stmt->update_column_item_rel_id())) { + LOG_WARN("failed to update column item by id", K(ret)); + } else if (OB_FAIL(child_stmt->formalize_stmt(ctx_->session_info_))) { + LOG_WARN("failed to formalize stmt", K(ret)); + } + return ret; +} + +int ObTransformMinMax::is_valid_aggr_items(ObTransformerCtx &ctx, + const ObSelectStmt &stmt, + bool &is_valid) +{ + int ret = OB_SUCCESS; + const ObAggFunRawExpr *expr = NULL; + bool valid = true; + EqualSets &equal_sets = ctx.equal_sets_; + ObSEArray const_exprs; + ObSEArray valid_col_exprs; + ObArenaAllocator alloc(ObMemAttr(MTL_ID(), "RewriteMinMax")); + if (OB_FAIL(stmt.get_stmt_equal_sets(equal_sets, alloc, true))) { + LOG_WARN("failed to get stmt equal sets", K(ret)); + } else if (OB_FAIL(ObOptimizerUtil::compute_const_exprs(stmt.get_condition_exprs(), + const_exprs))) { + LOG_WARN("failed to compute const equivalent exprs", K(ret)); + } + for (int64_t i = 0; OB_SUCC(ret) && valid && i < stmt.get_aggr_item_size(); ++i) { + if (OB_ISNULL(expr = stmt.get_aggr_item(i))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("params have null", K(ret), KP(expr)); + } 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"); + valid = false; + } else if (OB_ISNULL(expr->get_param_expr(0))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null expr", K(ret)); + } else if (is_contain(valid_col_exprs, expr->get_param_expr(0))) { + valid = true; + } else if (OB_FAIL(is_valid_index_column(ctx, + &stmt, + expr->get_param_expr(0), + &equal_sets, + &const_exprs, + valid))) { + LOG_WARN("failed to check is valid index column", K(ret)); + } else if (!valid) { + OPT_TRACE("aggr expr is not include index column"); + } else if (OB_FAIL(valid_col_exprs.push_back(expr->get_param_expr(0)))) { + LOG_WARN("failed to push back valid column", K(ret)); + } + } + equal_sets.reuse(); + is_valid = valid; + return ret; +} + +int ObTransformMinMax::is_valid_select_list(const ObSelectStmt &stmt, bool &is_valid) +{ + int ret = OB_SUCCESS; + bool valid = true; + for (int64_t i = 0; OB_SUCC(ret) && valid && i < stmt.get_select_item_size(); ++i) { + if (OB_FAIL(is_valid_select_expr(stmt.get_select_item(i).expr_, valid))) { + LOG_WARN("failed to check expr is valid aggr", K(ret)); + } + } + is_valid = valid; + return ret; +} + +int ObTransformMinMax::is_valid_select_expr(const ObRawExpr *expr, bool &is_valid) { int ret = OB_SUCCESS; bool is_stack_overflow = false; - is_valid = false; - const ObRawExpr *param = NULL; - if (OB_ISNULL(aggr_expr) || OB_ISNULL(expr)) { + bool valid = true; + if (OB_ISNULL(expr)) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("expr should not be NULL", K(ret), KP(expr), KP(aggr_expr)); - } else if (OB_FAIL(check_stack_overflow(is_stack_overflow))) { - LOG_WARN("failed to check stack overflow", K(ret)); - } else if (is_stack_overflow) { - ret = OB_SIZE_OVERFLOW; - LOG_WARN("too deep recursive", K(ret), K(is_stack_overflow)); - } else if (expr == aggr_expr) { - is_valid = true; + LOG_WARN("expr should not be NULL", K(ret), KP(expr)); + } else if (expr->has_flag(IS_AGG) || expr->is_const_expr()) { + /* do nothing */ } else if (expr->has_flag(CNT_AGG)) { - for (int64_t i = 0; OB_SUCC(ret) && i < expr->get_param_count(); ++i) { - if (OB_ISNULL(param = expr->get_param_expr(i))) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("param is null", K(ret)); - } else if (param->is_const_expr()) { - /* do nothing */ - } else if (param->has_flag(CNT_AGG)) { - if (OB_FAIL(SMART_CALL(is_valid_aggr_expr(stmt, param, aggr_expr, is_valid)))) { - LOG_WARN("failed to check is_valid_expr", K(ret)); - } else if (!is_valid) { - break; - } - } else { - break; + for (int64_t i = 0; OB_SUCC(ret) && valid && i < expr->get_param_count(); ++i) { + if (OB_FAIL(SMART_CALL(is_valid_select_expr(expr->get_param_expr(i), valid)))) { + LOG_WARN("failed to check is_valid_expr", K(ret)); } } + } else { + valid = false; } + is_valid = valid; return ret; } -int ObTransformMinMax::is_valid_having(const ObSelectStmt *stmt, - const ObAggFunRawExpr *column_aggr_expr, - bool &is_expected) +int ObTransformMinMax::is_valid_having_list(const ObSelectStmt &stmt, bool &is_valid) { - // 改写时,having需要满足的条件: - // having的表达式中如果含有column,只能够是select_item中的聚集表达式 - // e.g. select max(c1) from t1 having max(c1) > 1; 是满足改写条件的 - // select max(c1) from t1 having c1 > 1; 是不满足改写条件的 int ret = OB_SUCCESS; - is_expected = true; - bool is_unexpected = false; - if (OB_ISNULL(stmt)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("stmt is null", K(ret)); - } - for (int64_t i = 0; OB_SUCC(ret) && !is_unexpected && i < stmt->get_having_expr_size(); ++i) { - if (OB_FAIL(find_unexpected_having_expr(column_aggr_expr, - stmt->get_having_exprs().at(i), - is_unexpected))) { - LOG_WARN("fail to find unexpected having expr", K(ret), K(i)); + bool valid = true; + for (int64_t i = 0; OB_SUCC(ret) && valid && i < stmt.get_having_expr_size(); ++i) { + if (OB_FAIL(is_valid_having_expr(stmt.get_having_exprs().at(i), valid))) { + LOG_WARN("failed to check having expr", K(ret), K(i)); } } - if (OB_SUCC(ret)) { - is_expected = !is_unexpected; - } + is_valid = valid; return ret; } -int ObTransformMinMax::find_unexpected_having_expr(const ObAggFunRawExpr *aggr_expr, - const ObRawExpr *cur_expr, - bool &is_unexpected) +int ObTransformMinMax::is_valid_having_expr(const ObRawExpr *expr, bool &is_valid) { int ret = OB_SUCCESS; - if (OB_ISNULL(cur_expr)) { + bool valid = true; + if (OB_ISNULL(expr)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("current expr is null", K(ret)); - } else if (aggr_expr == cur_expr) { - // do nothing - } else if (cur_expr->is_column_ref_expr() || - cur_expr->is_query_ref_expr()) { - is_unexpected = true; - } else if (cur_expr->has_flag(CNT_COLUMN) || - cur_expr->has_flag(CNT_SUB_QUERY)) { - for (int64_t i = 0; OB_SUCC(ret) && i < cur_expr->get_param_count(); ++i) { - if (OB_FAIL(SMART_CALL(find_unexpected_having_expr(aggr_expr, - cur_expr->get_param_expr(i), - is_unexpected)))) { - LOG_WARN("failed to find unexpected having expr", K(ret)); + } else if (expr->has_flag(IS_AGG)) { + /* do nothing*/ + } else if (expr->is_column_ref_expr() || expr->has_flag(CNT_SUB_QUERY)) { + valid = false; + } else if (expr->has_flag(CNT_COLUMN)) { + for (int64_t i = 0; OB_SUCC(ret) && valid && i < expr->get_param_count(); ++i) { + if (OB_FAIL(SMART_CALL(is_valid_having_expr(expr->get_param_expr(i), valid)))) { + LOG_WARN("failed to check having expr", K(ret)); } } } + is_valid = valid; + return ret; +} + +int ObTransformMinMax::is_valid_order_list(const ObSelectStmt &stmt, bool &is_valid) +{ + int ret = OB_SUCCESS; + bool valid = true; + for (int64_t i = 0; OB_SUCC(ret) && valid && i < stmt.get_order_item_size(); ++i) { + if (OB_FAIL(is_valid_order_expr(stmt.get_order_item(i).expr_, valid))) { + LOG_WARN("failed to check having expr", K(ret), K(i)); + } + } + is_valid = valid; + return ret; +} + +int ObTransformMinMax::is_valid_order_expr(const ObRawExpr *expr, bool &is_valid) +{ + int ret = OB_SUCCESS; + bool valid = true; + if (OB_ISNULL(expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("current expr is null", K(ret)); + } else if (expr->has_flag(IS_AGG)) { + /* do nothing*/ + } else if (expr->is_column_ref_expr()) { + valid = false; + } else if (expr->has_flag(CNT_COLUMN)) { + for (int64_t i = 0; OB_SUCC(ret) && valid && i < expr->get_param_count(); ++i) { + if (OB_FAIL(SMART_CALL(is_valid_order_expr(expr->get_param_expr(i), valid)))) { + LOG_WARN("failed to check order expr", K(ret)); + } + } + } + is_valid = valid; return ret; } int ObTransformMinMax::is_valid_index_column(ObTransformerCtx &ctx, const ObSelectStmt *stmt, const ObRawExpr *expr, + EqualSets *equal_sets, + ObIArray *const_exprs, bool &is_valid) { int ret = OB_SUCCESS; const TableItem *table_item = NULL; const ObColumnRefRawExpr *col_expr = NULL; bool is_match_index = false; - ObArenaAllocator alloc; - EqualSets &equal_sets = ctx.equal_sets_; - ObSEArray const_exprs; + bool need_check_query_range = false; is_valid = false; 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()) { /* do nothing */ - } else if (FALSE_IT(col_expr = static_cast(expr))) { + } else if (OB_FALSE_IT(col_expr = static_cast(expr))) { } else if (OB_ISNULL(table_item = stmt->get_table_item_by_id(col_expr->get_table_id()))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table item is null", K(ret)); } else if (!table_item->is_basic_table()) { /* do nothing */ - } else if (OB_FAIL(const_cast(stmt)->get_stmt_equal_sets(equal_sets, alloc, true))) { - LOG_WARN("failed to get stmt equal sets", K(ret)); - } 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_, - stmt, - col_expr, - is_match_index, - &equal_sets, &const_exprs))) { - LOG_WARN("failed to check whether column matches index", K(ret)); - } else if (is_match_index) { - is_valid = true; + } else { + if (stmt->get_aggr_item_size() > 1 && stmt->get_condition_size() > 0) { + need_check_query_range = true; + } + if (OB_FAIL(ObTransformUtils::is_match_index(ctx.sql_schema_guard_, + stmt, + col_expr, + is_match_index, + equal_sets, + const_exprs, + NULL, + false, + need_check_query_range, + &ctx))) { + LOG_WARN("failed to check whether column matches index", K(ret)); + } else if (is_match_index) { + is_valid = true; + } } - equal_sets.reuse(); return ret; } -int ObTransformMinMax::set_child_order_item(ObSelectStmt *stmt, ObRawExpr *aggr_expr) +int ObTransformMinMax::set_child_order_item(ObSelectStmt *stmt, + ObRawExpr *aggr_param, + ObItemType aggr_type) { int ret = OB_SUCCESS; - if (OB_ISNULL(stmt) || OB_ISNULL(aggr_expr) - || OB_ISNULL(aggr_expr->get_param_expr(0))) { + if (OB_ISNULL(stmt) || OB_ISNULL(aggr_param)) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("params have null", K(ret), K(stmt), K(aggr_expr)); + LOG_WARN("params have null", K(ret), K(stmt), K(aggr_param)); } else { OrderItem new_order_item; - new_order_item.expr_ = aggr_expr->get_param_expr(0); - if (T_FUN_MAX == aggr_expr->get_expr_type()) { + new_order_item.expr_ = aggr_param; + if (T_FUN_MAX == aggr_type) { new_order_item.order_type_ = default_desc_direction(); - } else if (T_FUN_MIN == aggr_expr->get_expr_type()) { + } else if (T_FUN_MIN == aggr_type) { new_order_item.order_type_ = default_asc_direction(); } else { ret = OB_ERR_UNEXPECTED; @@ -344,25 +557,22 @@ int ObTransformMinMax::set_child_order_item(ObSelectStmt *stmt, ObRawExpr *aggr_ } if (OB_SUCC(ret)) { if (OB_FAIL(stmt->add_order_item(new_order_item))) { - LOG_WARN("fail to add order item", K(ret), K(stmt), K(new_order_item)); + LOG_WARN("failed to add order item", K(ret), K(stmt), K(new_order_item)); } } } return ret; } -int ObTransformMinMax::set_child_condition(ObSelectStmt *stmt, ObRawExpr *aggr_expr) +int ObTransformMinMax::set_child_condition(ObSelectStmt *stmt, ObRawExpr *aggr_param) { int ret = OB_SUCCESS; ObOpRawExpr *not_null_expr = NULL; - ObRawExpr *aggr_param = NULL; bool is_not_null = false; ObArray constraints; - if (OB_ISNULL(stmt) || OB_ISNULL(aggr_expr) - || OB_ISNULL(aggr_param = aggr_expr->get_param_expr(0)) - || OB_ISNULL(ctx_)) { + if (OB_ISNULL(stmt) || OB_ISNULL(aggr_param) || OB_ISNULL(ctx_)) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("params have null", K(ret), K(stmt), K(aggr_expr)); + LOG_WARN("params have null", K(ret), K(stmt), K(aggr_param)); } else if (OB_FAIL(ObTransformUtils::is_expr_not_null(ctx_, stmt, aggr_param, NULLABLE_SCOPE::NS_WHERE, is_not_null, &constraints))) { LOG_WARN("failed to check expr not null", K(ret)); diff --git a/src/sql/rewrite/ob_transform_min_max.h b/src/sql/rewrite/ob_transform_min_max.h index 2247b5348b..d024160863 100644 --- a/src/sql/rewrite/ob_transform_min_max.h +++ b/src/sql/rewrite/ob_transform_min_max.h @@ -65,39 +65,45 @@ public: bool &trans_happened) override; static int check_transform_validity(ObTransformerCtx &ctx, ObSelectStmt *stmt, - ObAggFunRawExpr *&aggr_expr, bool &is_valid); private: - int do_transform(ObSelectStmt *select_stmt, ObAggFunRawExpr *aggr_expr); + int do_transform(ObSelectStmt *select_stmt); + + int do_single_minmax_transform(ObSelectStmt *select_stmt); + + int do_multi_minmax_transform(ObSelectStmt *select_stmt); + + int deep_copy_subquery_for_aggr(const ObSelectStmt &copied_stmt, + ObRawExpr *aggr_param, + ObItemType aggr_type, + ObSelectStmt *&child_stmt); static int is_valid_index_column(ObTransformerCtx &ctx, const ObSelectStmt *stmt, const ObRawExpr *expr, - bool &is_expected_index); + EqualSets *equal_sets, + ObIArray *const_exprs, + bool &is_valid); - static int is_valid_having(const ObSelectStmt *stmt, - const ObAggFunRawExpr *column_aggr_expr, - bool &is_expected); + int set_child_condition(ObSelectStmt *stmt, ObRawExpr *aggr_param); - static int is_valid_aggr_expr(const ObSelectStmt &stmt, - const ObRawExpr *expr, - const ObAggFunRawExpr *aggr_expr, - bool &is_valid); + int set_child_order_item(ObSelectStmt *stmt, ObRawExpr *aggr_param, ObItemType aggr_type); - static int find_unexpected_having_expr(const ObAggFunRawExpr *aggr_expr, - const ObRawExpr *cur_expr, - bool &is_unexpected); + static int is_valid_aggr_items(ObTransformerCtx &ctx, const ObSelectStmt &stmt, bool &is_valid); - int set_child_condition(ObSelectStmt *stmt, ObRawExpr *aggr_expr); + static int is_valid_select_list(const ObSelectStmt &stmt, bool &is_valid); - int set_child_order_item(ObSelectStmt *stmt, ObRawExpr *aggr_expr); + static int is_valid_select_expr(const ObRawExpr *expr, bool &is_valid); + + static int is_valid_having_list(const ObSelectStmt &stmt, bool &is_valid); + + static int is_valid_having_expr(const ObRawExpr *expr, bool &is_valid); + + static int is_valid_order_list(const ObSelectStmt &stmt, bool &is_valid); + + static int is_valid_order_expr(const ObRawExpr *expr, bool &is_valid); - /** - * @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 - */ - static int is_valid_select_list(const ObSelectStmt &stmt, const ObAggFunRawExpr *aggr_expr, bool &is_valid); DISALLOW_COPY_AND_ASSIGN(ObTransformMinMax); }; diff --git a/src/sql/rewrite/ob_transform_pre_process.h b/src/sql/rewrite/ob_transform_pre_process.h index 0de535d4f0..bc0d26cf2d 100644 --- a/src/sql/rewrite/ob_transform_pre_process.h +++ b/src/sql/rewrite/ob_transform_pre_process.h @@ -27,27 +27,6 @@ namespace sql typedef std::pair JoinTableIdPair; -class ObTransformPreProcess: public ObTransformRule -{ -public: - explicit ObTransformPreProcess(ObTransformerCtx *ctx) - : ObTransformRule(ctx, TransMethod::POST_ORDER) { } - virtual ~ObTransformPreProcess() {} - - virtual int transform_one_stmt(common::ObIArray &parent_stmts, - ObDMLStmt *&stmt, - bool &trans_happened) override; - - static int transform_expr(ObRawExprFactory &expr_factory, - const ObSQLSessionInfo &session, - ObRawExpr *&expr, - bool &trans_happened); -private: - virtual int need_transform(const common::ObIArray &parent_stmts, - const int64_t current_level, - const ObDMLStmt &stmt, - bool &need_trans) override; -// used for transform in expr to or exprs struct DistinctObjMeta { ObObjType obj_type_; @@ -70,9 +49,31 @@ struct DistinctObjMeta bool cs_level_equal = lib::is_oracle_mode() ? true : (coll_level_ == other.coll_level_); return obj_type_ == other.obj_type_ && coll_type_ == other.coll_type_ && cs_level_equal; } - TO_STRING_KV(K_(obj_type), K_(coll_type)); + TO_STRING_KV(K_(obj_type), K_(coll_type), K_(coll_level)); }; +class ObTransformPreProcess: public ObTransformRule +{ +public: + explicit ObTransformPreProcess(ObTransformerCtx *ctx) + : ObTransformRule(ctx, TransMethod::POST_ORDER) { } + virtual ~ObTransformPreProcess() {} + + virtual int transform_one_stmt(common::ObIArray &parent_stmts, + ObDMLStmt *&stmt, + bool &trans_happened) override; + + static int transform_expr(ObRawExprFactory &expr_factory, + const ObSQLSessionInfo &session, + ObRawExpr *&expr, + bool &trans_happened); +private: + virtual int need_transform(const common::ObIArray &parent_stmts, + const int64_t current_level, + const ObDMLStmt &stmt, + bool &need_trans) override; +// used for transform in expr to or exprs + /* * following functions are used to add all rowkey columns */ diff --git a/src/sql/rewrite/ob_transform_query_push_down.cpp b/src/sql/rewrite/ob_transform_query_push_down.cpp index d8a9ebf860..6b1e201798 100644 --- a/src/sql/rewrite/ob_transform_query_push_down.cpp +++ b/src/sql/rewrite/ob_transform_query_push_down.cpp @@ -246,7 +246,8 @@ int ObTransformQueryPushDown::check_transform_validity(ObSelectStmt *select_stmt can_transform = false; need_distinct = false; transform_having = false; - if (OB_ISNULL(ctx_) || OB_ISNULL(select_stmt) || OB_ISNULL(view_stmt)) { + if (OB_ISNULL(ctx_) || OB_ISNULL(select_stmt) || OB_ISNULL(view_stmt) || + OB_ISNULL(select_stmt->get_query_ctx())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("stmt is NULL", K(select_stmt), K(view_stmt), K(ret)); } else if (select_stmt->is_recursive_union() || @@ -254,8 +255,11 @@ int ObTransformQueryPushDown::check_transform_validity(ObSelectStmt *select_stmt (view_stmt->is_recursive_union() && !select_stmt->is_spj()) || select_stmt->is_set_stmt() || (select_stmt->has_sequence() && !view_stmt->is_spj()) || - view_stmt->has_ora_rowscn() || - view_stmt->is_values_table_query()) {//判断1, 2 + view_stmt->has_ora_rowscn()) {//判断1, 2 + can_transform = false; + OPT_TRACE("stmt is not spj"); + } else if (select_stmt->get_query_ctx()->optimizer_features_enable_version_ < COMPAT_VERSION_4_3_2 && + view_stmt->is_values_table_query()) { can_transform = false; OPT_TRACE("stmt is not spj"); } else if (OB_FAIL(check_rownum_push_down(select_stmt, view_stmt, check_status))) {//判断3 diff --git a/src/sql/rewrite/ob_transform_rule.cpp b/src/sql/rewrite/ob_transform_rule.cpp index 5f876199dd..cf950537b6 100644 --- a/src/sql/rewrite/ob_transform_rule.cpp +++ b/src/sql/rewrite/ob_transform_rule.cpp @@ -136,6 +136,7 @@ const char* ObTransformerCtx::get_trans_type_string(uint64_t trans_type) TRANS_TYPE_TO_STR(SELECT_EXPR_PULLUP) TRANS_TYPE_TO_STR(PROCESS_DBLINK) TRANS_TYPE_TO_STR(DECORRELATE) + TRANS_TYPE_TO_STR(CONDITIONAL_AGGR_COALESCE) TRANS_TYPE_TO_STR(MV_REWRITE) default: return NULL; } @@ -310,19 +311,27 @@ int ObTransformRule::accept_transform(common::ObIArray &parent_ trans_happened = true; } else if (ctx_->is_set_stmt_oversize_) { LOG_TRACE("not accept transform because large set stmt", K(ctx_->is_set_stmt_oversize_)); - } else if (OB_FAIL(evaluate_cost(parent_stmts, trans_stmt, true, - trans_stmt_cost, is_expected, check_ctx))) { - LOG_WARN("failed to evaluate cost for the transformed stmt", K(ret)); - } else if ((!check_original_plan && stmt_cost_ >= 0) || !is_expected) { - trans_happened = is_expected && trans_stmt_cost < stmt_cost_; - } else if (OB_FAIL(evaluate_cost(parent_stmts, stmt, false, - stmt_cost_, is_original_expected, - check_original_plan ? check_ctx : NULL))) { - LOG_WARN("failed to evaluate cost for the origin stmt", K(ret)); - } else if (!is_original_expected) { - trans_happened = is_original_expected; + } else if (ctx_->in_accept_transform_) { + LOG_TRACE("not accept transform because already in one accepct transform", K(ctx_->in_accept_transform_)); } else { - trans_happened = trans_stmt_cost < stmt_cost_; + ctx_->in_accept_transform_ = true; + if (OB_FAIL(evaluate_cost(parent_stmts, trans_stmt, true, trans_stmt_cost, is_expected, + check_ctx))) { + LOG_WARN("failed to evaluate cost for the transformed stmt", K(ret)); + } else if ((!check_original_plan && stmt_cost_ >= 0) || !is_expected) { + trans_happened = is_expected && trans_stmt_cost < stmt_cost_; + } else if (OB_FAIL(evaluate_cost(parent_stmts, stmt, false, stmt_cost_, is_original_expected, + check_original_plan ? check_ctx : NULL))) { + LOG_WARN("failed to evaluate cost for the origin stmt", K(ret)); + } else if (!is_original_expected) { + trans_happened = is_original_expected; + } else { + trans_happened = trans_stmt_cost < stmt_cost_; + } + if (stmt->get_query_ctx()->get_injected_random_status()) { + trans_happened = true; + } + ctx_->in_accept_transform_ = false; } RESUME_OPT_TRACE; @@ -334,7 +343,7 @@ int ObTransformRule::accept_transform(common::ObIArray &parent_ OPT_TRACE("is expected plan:", is_expected); OPT_TRACE("is expected original plan:", is_original_expected); LOG_TRACE("reject transform because the cost is increased or the query plan is unexpected", - K_(ctx_->is_set_stmt_oversize), K_(stmt_cost), K(trans_stmt_cost), K(is_expected)); + K_(ctx_->is_set_stmt_oversize), K_(stmt_cost), K(trans_stmt_cost), K(is_expected)); } else if (OB_FAIL(adjust_transformed_stmt(parent_stmts, trans_stmt, tmp1, tmp2))) { LOG_WARN("failed to adjust transformed stmt", K(ret)); } else if (force_accept) { @@ -695,7 +704,8 @@ bool ObTransformRule::is_normal_disabled_transform(const ObDMLStmt &stmt) { return (stmt.is_hierarchical_query() && transform_method_ != TransMethod::ROOT_ONLY) || stmt.is_insert_all_stmt() || - stmt.is_values_table_query(); + (stmt.is_values_table_query() && NULL != stmt.get_query_ctx() && + !ObTransformUtils::is_enable_values_table_rewrite(stmt.get_query_ctx()->optimizer_features_enable_version_)); } int ObTransformRule::need_transform(const common::ObIArray &parent_stmts, diff --git a/src/sql/rewrite/ob_transform_rule.h b/src/sql/rewrite/ob_transform_rule.h index b110e1e711..a2216b8b79 100644 --- a/src/sql/rewrite/ob_transform_rule.h +++ b/src/sql/rewrite/ob_transform_rule.h @@ -70,6 +70,7 @@ struct ObTransformerCtx groupby_pushdown_stmts_(), is_spm_outline_(false), push_down_filters_(), + in_accept_transform_(false), iteration_level_(0) { } virtual ~ObTransformerCtx() {} @@ -131,6 +132,7 @@ struct ObTransformerCtx /* end used for hint and outline below */ bool is_spm_outline_; ObSEArray push_down_filters_; + bool in_accept_transform_; uint64_t iteration_level_; }; @@ -178,6 +180,7 @@ enum TRANSFORM_TYPE { SELECT_EXPR_PULLUP , PROCESS_DBLINK , DECORRELATE , + CONDITIONAL_AGGR_COALESCE , MV_REWRITE , TRANSFORM_TYPE_COUNT_PLUS_ONE , }; @@ -275,7 +278,9 @@ public: (1L << JOIN_LIMIT_PUSHDOWN) | (1L << CONST_PROPAGATE) | (1L << LEFT_JOIN_TO_ANTI) | - (1L << COUNT_TO_EXISTS); + (1L << COUNT_TO_EXISTS) | + (1L << CONDITIONAL_AGGR_COALESCE) | + (1L << SEMI_TO_INNER); static const uint64_t ALL_COST_BASED_RULES = (1L << OR_EXPANSION) | (1L << WIN_MAGIC) | diff --git a/src/sql/rewrite/ob_transform_semi_to_inner.cpp b/src/sql/rewrite/ob_transform_semi_to_inner.cpp index 1b7b946ca2..bb30464c60 100644 --- a/src/sql/rewrite/ob_transform_semi_to_inner.cpp +++ b/src/sql/rewrite/ob_transform_semi_to_inner.cpp @@ -18,6 +18,7 @@ #include "sql/optimizer/ob_log_join.h" #include "sql/optimizer/ob_log_function_table.h" #include "common/ob_smart_call.h" +#include "sql/optimizer/ob_log_values_table_access.h" using namespace oceanbase::sql; using namespace oceanbase::common; @@ -1501,6 +1502,11 @@ int ObTransformSemiToInner::find_operator(ObLogicalOperator* root, if (scan->get_table_id() == table_id) { table_op = scan; } + } else if (log_op_def::LOG_VALUES_TABLE_ACCESS == root->get_type()) { + ObLogValuesTableAccess *scan = static_cast(root); + if (scan->get_table_id() == table_id) { + table_op = scan; + } } else { for (int64_t i = 0; OB_SUCC(ret) && NULL == table_op && i < root->get_num_of_child(); ++i) { ObLogicalOperator *child = root->get_child(i); diff --git a/src/sql/rewrite/ob_transform_simplify_subquery.cpp b/src/sql/rewrite/ob_transform_simplify_subquery.cpp index bc30d6abc7..f965b38c4f 100644 --- a/src/sql/rewrite/ob_transform_simplify_subquery.cpp +++ b/src/sql/rewrite/ob_transform_simplify_subquery.cpp @@ -1251,7 +1251,7 @@ int ObTransformSimplifySubquery::eliminate_subquery(ObDMLStmt *stmt, { int ret = OB_SUCCESS; bool can_be_eliminated = false; - if (OB_ISNULL(expr)) { + if (OB_ISNULL(expr) || OB_ISNULL(stmt) || OB_ISNULL(stmt->get_query_ctx())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("expr is NULL in eliminate subquery", K(ret)); } else if (!expr->has_flag(CNT_SUB_QUERY)) { @@ -1269,7 +1269,10 @@ int ObTransformSimplifySubquery::eliminate_subquery(ObDMLStmt *stmt, } else if (OB_ISNULL(subquery = subq_expr->get_ref_stmt())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Subquery stmt is NULL", K(ret)); - } else if (subquery->is_contains_assignment() || subquery->is_values_table_query()) { + } else if (subquery->is_contains_assignment()) { + // do nothing + } else if (subquery->is_values_table_query() && + !ObTransformUtils::is_enable_values_table_rewrite(stmt->get_query_ctx()->optimizer_features_enable_version_)) { // do nothing } else if (OB_FAIL(subquery_can_be_eliminated_in_exists(expr->get_expr_type(), subquery, @@ -1448,18 +1451,20 @@ int ObTransformSimplifySubquery::groupby_can_be_eliminated_in_any_all(const ObSe // 3. 没有limit子句 // 4. 无聚集函数(select item中) // 5. 非常量select item列,全部包含在group exprs中 - if (OB_ISNULL(stmt)) { + if (OB_ISNULL(stmt) || OB_ISNULL(stmt->get_query_ctx())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("stmt is NULL", K(ret)); } else if (0 == stmt->get_table_size() || stmt->is_set_stmt()) { // Only non-set stmt will be eliminated and do nothing for other DML stmts: // 1. set -> No elimination // 2. select 1 + floor(2) (table_size == 0) -> No elimination + } else if (stmt->is_values_table_query() && + !ObTransformUtils::is_enable_values_table_rewrite(stmt->get_query_ctx()->optimizer_features_enable_version_)) { + /* do nothing */ } else if (stmt->has_group_by() && !stmt->has_having() && !stmt->has_limit() - && 0 == stmt->get_aggr_item_size() - && !stmt->is_values_table_query()) { + && 0 == stmt->get_aggr_item_size()) { // Check if select list is involved in group exprs ObRawExpr *s_expr = NULL; bool all_in_group_exprs = true; @@ -1488,7 +1493,8 @@ int ObTransformSimplifySubquery::eliminate_subquery_in_exists(ObDMLStmt *stmt, ObRawExprFactory *expr_factory = NULL; ObSelectStmt *subquery = NULL; bool add_limit_constraint = false; - if (OB_ISNULL(expr) || OB_ISNULL(ctx_) || OB_ISNULL(expr_factory = ctx_->expr_factory_)) { + if (OB_ISNULL(expr) || OB_ISNULL(ctx_) || OB_ISNULL(expr_factory = ctx_->expr_factory_) || + OB_ISNULL(stmt) || OB_ISNULL(stmt->get_query_ctx())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("NULL pointer Error", KP(expr), KP_(ctx), KP(expr_factory), K(ret)); } else if (T_OP_EXISTS == expr->get_expr_type() || T_OP_NOT_EXISTS == expr->get_expr_type()) { @@ -1500,7 +1506,8 @@ int ObTransformSimplifySubquery::eliminate_subquery_in_exists(ObDMLStmt *stmt, } else if (OB_ISNULL(subquery = subq_expr->get_ref_stmt())) { ret = OB_INVALID_ARGUMENT; LOG_WARN("Subquery stmt is NULL", K(ret)); - } else if (subquery->is_values_table_query()) { /* do nothing */ + } else if (subquery->is_values_table_query() && + !ObTransformUtils::is_enable_values_table_rewrite(stmt->get_query_ctx()->optimizer_features_enable_version_)) { /* do nothing */ //Just in case different parameters hit same plan, firstly we need add const param constraint } else if (OB_FAIL(need_add_limit_constraint(expr->get_expr_type(), subquery, add_limit_constraint))){ LOG_WARN("failed to check limit constraints", K(ret)); @@ -2216,8 +2223,7 @@ int ObTransformSimplifySubquery::check_stmt_can_trans_as_exists(ObSelectStmt *st bool contain_rownum = false; is_valid = false; match_index = false; - if (OB_ISNULL(stmt) || - OB_ISNULL(ctx_)) { + if (OB_ISNULL(stmt) || OB_ISNULL(ctx_) || OB_ISNULL(stmt->get_query_ctx())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("child stmt is null", K(ret)); } else if (OB_FAIL(stmt->has_rownum(contain_rownum))) { @@ -2262,7 +2268,8 @@ int ObTransformSimplifySubquery::check_stmt_can_trans_as_exists(ObSelectStmt *st stmt->is_hierarchical_query() || stmt->has_window_function() || stmt->has_rollup() || - stmt->is_values_table_query()) { + (stmt->is_values_table_query() && + !ObTransformUtils::is_enable_values_table_rewrite(stmt->get_query_ctx()->optimizer_features_enable_version_))) { LOG_TRACE("stmt not support trans in as exists", K(stmt->is_contains_assignment()), K(stmt->is_hierarchical_query()), K(stmt->has_window_function()), K(stmt->has_rollup()), K(stmt->is_values_table_query())); diff --git a/src/sql/rewrite/ob_transform_temp_table.cpp b/src/sql/rewrite/ob_transform_temp_table.cpp index db97d59892..1cedff565f 100644 --- a/src/sql/rewrite/ob_transform_temp_table.cpp +++ b/src/sql/rewrite/ob_transform_temp_table.cpp @@ -301,10 +301,9 @@ int ObTransformTempTable::check_stmt_can_materialize(ObSelectStmt *stmt, bool is } } else { // Currently, we will not push `limit` in stmt into cte - ObAggFunRawExpr *dummy = NULL; bool can_use_fast_min_max = false; STOP_OPT_TRACE; - if (OB_FAIL(ObTransformMinMax::check_transform_validity(*ctx_, stmt, dummy, can_use_fast_min_max))) { + if (OB_FAIL(ObTransformMinMax::check_transform_validity(*ctx_, stmt, can_use_fast_min_max))) { LOG_WARN("failed to check fast min max", K(ret)); } RESUME_OPT_TRACE; diff --git a/src/sql/rewrite/ob_transform_utils.cpp b/src/sql/rewrite/ob_transform_utils.cpp index 97fa179b89..d69cb66241 100644 --- a/src/sql/rewrite/ob_transform_utils.cpp +++ b/src/sql/rewrite/ob_transform_utils.cpp @@ -662,7 +662,7 @@ int ObTransformUtils::create_new_column_expr(ObTransformerCtx *ctx, new_column_ref->set_result_flag(NOT_NULL_FLAG); } if (OB_FAIL(new_column_ref->add_relation_id(stmt->get_table_bit_index(table_item.table_id_)))) { - LOG_WARN("failed to add relation id", K(ret)); + LOG_WARN("failed to add relation id", K(ret), K(table_item)); } else if (select_expr->is_column_ref_expr()) { const ObColumnRefRawExpr *old_col = static_cast(select_expr); const ColumnItem *old_col_item = NULL; @@ -1010,7 +1010,7 @@ int ObTransformUtils::refresh_select_items_name(ObIAllocator &allocator, ObSelec if (alias_name.empty()) { if (OB_FAIL(expr->get_name(name_buf, 64, pos))) { ret = OB_SUCCESS; - pos = sprintf(name_buf, "SEL_%ld", select_stmt->get_select_item_size() + 1); + pos = sprintf(name_buf, "SEL_%ld", i + 1); pos = (pos < 0 || pos >= 64) ? 0 : pos; } alias_name.assign(name_buf, static_cast(pos)); @@ -1027,7 +1027,7 @@ int ObTransformUtils::refresh_select_items_name(ObIAllocator &allocator, ObSelec return ret; } -int ObTransformUtils::refresh_column_items_name(ObSelectStmt *stmt, int64_t table_id) +int ObTransformUtils::refresh_column_items_name(ObDMLStmt *stmt, int64_t table_id) { int ret = OB_SUCCESS; ObSelectStmt *set_view_stmt = NULL; @@ -2378,7 +2378,7 @@ int ObTransformUtils::is_column_expr_not_null(ObNotNullContext &ctx, if (OB_ISNULL(stmt = ctx.stmt_) || OB_ISNULL(expr) || OB_ISNULL(table = stmt->get_table_item_by_id(expr->get_table_id()))) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("table item is null", K(ret), K(expr->get_table_id()), K(*stmt)); + LOG_WARN("table item is null", K(ret), K(expr), K(stmt)); } else if (is_virtual_table(table->ref_id_)) { // 'NOT NULL' of the virtual table is unreliable } else if (ObOptimizerUtil::find_item(ctx.right_table_ids_, table->table_id_)) { @@ -3444,6 +3444,80 @@ int ObTransformUtils::get_vaild_index_id(ObSqlSchemaGuard *schema_guard, return ret; } +// 获取用于抽取query range的column item, 取出stmt中的index column item, 直到第一个不在stmt中的为止 +// 例如:select min(c3) from t1 where c1 = 1 and c3 > 1, index i1(c1,c2,c3), 返回c1而不是c1,c3 +int ObTransformUtils::get_range_column_items_by_ids(const ObDMLStmt *stmt, + uint64_t table_id, + const ObIArray &column_ids, + ObIArray &column_items) +{ + int ret = OB_SUCCESS; + ColumnItem *column = NULL; + bool get_column_item = true; + if (OB_ISNULL(stmt)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("stmt is unexpected null", K(ret)); + } + for (int64_t i = 0; OB_SUCC(ret) && get_column_item && i < column_ids.count(); i++) { + column = stmt->get_column_item_by_id(table_id, column_ids.at(i)); + if (NULL != column) { + ret = column_items.push_back(*column); + } else { + get_column_item = false; + } + } + return ret; +} + +int ObTransformUtils::check_index_extract_query_range(const ObDMLStmt *stmt, + uint64_t table_id, + const ObIArray &index_cols, + const ObIArray &predicate_exprs, + ObTransformerCtx *ctx, + bool &is_match) +{ + int ret = OB_SUCCESS; + ObArray range_columns; + ObRawExpr *condition_expr = NULL; + ObArenaAllocator alloc(ObMemAttr(MTL_ID(), "RewriteMinMax")); + void *tmp_ptr = NULL; + ObQueryRange *query_range = NULL; + const ParamStore *params = NULL; + if (OB_ISNULL(stmt) || OB_ISNULL(ctx) || OB_ISNULL(ctx->exec_ctx_) + || OB_ISNULL(ctx->exec_ctx_->get_physical_plan_ctx())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("params have null", K(ret), K(stmt), K(ctx)); + } else { + const ObDataTypeCastParams dtc_params = ObBasicSessionInfo::create_dtc_params(ctx->session_info_); + params = &ctx->exec_ctx_->get_physical_plan_ctx()->get_param_store(); + if (OB_FAIL(ret)) { + } else if (OB_ISNULL(tmp_ptr = alloc.alloc(sizeof(ObQueryRange))) + || OB_ISNULL(query_range = new(tmp_ptr)ObQueryRange(alloc))) { + ret = OB_ALLOCATE_MEMORY_FAILED; + LOG_WARN("failed to allocate memory for query range", K(ret)); + } else if (OB_FAIL(get_range_column_items_by_ids(stmt, + table_id, + index_cols, + range_columns))) { + LOG_WARN("failed to get index column items by index cols", K(ret)); + } else if (OB_FAIL(query_range->preliminary_extract_query_range(range_columns, + predicate_exprs, + dtc_params, + ctx->exec_ctx_, + NULL, + params))) { + LOG_WARN("failed to extract query range", K(ret)); + } else if (query_range->get_range_exprs().count() != predicate_exprs.count()) { + is_match = false; + } + if (OB_NOT_NULL(query_range)) { + query_range->~ObQueryRange(); + query_range = NULL; + } + } + return ret; +} + int ObTransformUtils::is_match_index(ObSqlSchemaGuard *schema_guard, const ObDMLStmt *stmt, const ObColumnRefRawExpr *col_expr, @@ -3451,19 +3525,19 @@ int ObTransformUtils::is_match_index(ObSqlSchemaGuard *schema_guard, EqualSets *equal_sets, ObIArray *const_exprs, ObIArray *col_exprs, - const bool need_match_col_exprs) + const bool need_match_col_exprs, + const bool need_check_query_range, + ObTransformerCtx *ctx) { int ret = OB_SUCCESS; uint64_t table_ref_id = OB_INVALID_ID; const TableItem *table_item = NULL; ObSEArray index_ids; is_match = false; - if (OB_ISNULL(stmt) || OB_ISNULL(col_expr) - || OB_ISNULL(schema_guard)) { + if (OB_ISNULL(stmt) || OB_ISNULL(col_expr) || OB_ISNULL(schema_guard)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("params have null", K(ret), K(stmt), K(col_expr), K(schema_guard)); - } else if (OB_ISNULL(table_item = stmt->get_table_item_by_id( - col_expr->get_table_id()))) { + } else if (OB_ISNULL(table_item = stmt->get_table_item_by_id(col_expr->get_table_id()))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get_table_item", K(ret), K(table_item)); } else if (!table_item->is_basic_table()) { @@ -3473,6 +3547,19 @@ int ObTransformUtils::is_match_index(ObSqlSchemaGuard *schema_guard, } else { ObSEArray index_cols; const ObTableSchema *index_schema = NULL; + ObSEArray predicate_exprs; + ObRawExpr *condition_expr = NULL; + if (need_check_query_range) { + for (int64_t i = 0; OB_SUCC(ret) && i < stmt->get_condition_size(); i++) { + if (OB_ISNULL(condition_expr = const_cast(stmt->get_condition_expr(i)))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("condition expr is null", K(ret)); + } else if (!condition_expr->is_const_expr() + && OB_FAIL(predicate_exprs.push_back(condition_expr))) { + LOG_WARN("failed to push back condition expr", K(ret)); + } + } + } for (int64_t i = 0; OB_SUCC(ret) && !is_match && i < index_ids.count(); ++i) { index_cols.reuse(); if (OB_FAIL(schema_guard->get_table_schema(index_ids.at(i), table_item, index_schema))) { @@ -3497,6 +3584,14 @@ int ObTransformUtils::is_match_index(ObSqlSchemaGuard *schema_guard, col_exprs, need_match_col_exprs))) { LOG_WARN("failed to check is column match index", K(ret)); + } else if (is_match && need_check_query_range + && OB_FAIL(check_index_extract_query_range(stmt, + col_expr->get_table_id(), + index_cols, + predicate_exprs, + ctx, + is_match))) { + LOG_WARN("failed to check if index can extract query range", K(ret)); } } } @@ -7573,10 +7668,12 @@ int ObTransformUtils::create_inline_view(ObTransformerCtx *ctx, LOG_WARN("table is null", K(ret), K(table)); } else if (OB_FAIL(stmt->remove_from_item(table->table_id_, &remove_happened))) { LOG_WARN("failed to remove from item", K(ret)); - } else if (OB_FAIL(view_stmt->add_from_item(from_tables.at(i)->table_id_, is_joined_table))) { + } else if (OB_FAIL(view_stmt->add_from_item(table->table_id_, is_joined_table))) { LOG_WARN("failed to add from item", K(ret)); } else if (is_joined_table) { - if (OB_FAIL(append(basic_table_ids, static_cast(table)->single_table_ids_))) { + if (OB_FAIL(stmt->remove_joined_table_item(table->table_id_))) { + LOG_WARN("failed to remove joined table", K(ret)); + } else if (OB_FAIL(append(basic_table_ids, static_cast(table)->single_table_ids_))) { LOG_WARN("failed to append single table ids", K(ret)); } else if (OB_FAIL(view_stmt->add_joined_table(static_cast(table)))) { LOG_WARN("failed to add joined table", K(ret)); @@ -8540,6 +8637,18 @@ int ObTransformUtils::build_case_when_expr(ObTransformerCtx *ctx, return ret; } +int ObTransformUtils::check_error_free_exprs(ObIArray &exprs, bool &is_error_free) +{ + int ret = OB_SUCCESS; + is_error_free = true; + for (int64_t i = 0; OB_SUCC(ret) && is_error_free && i < exprs.count(); i++) { + if (OB_FAIL(check_error_free_expr(exprs.at(i), is_error_free))) { + LOG_WARN("failed to check error free expr", K(ret)); + } + } + return ret; +} + int ObTransformUtils::check_error_free_expr(ObRawExpr *expr, bool &is_error_free) { int ret = OB_SUCCESS; @@ -11562,9 +11671,15 @@ int ObTransformUtils::is_table_item_correlated( LOG_WARN("failed to check function table expr correlated", K(ret)); } } else if (table->is_values_table()) { - for (int64_t j = 0; OB_SUCC(ret) && !contains && j < table->table_values_.count(); ++j) { - if (OB_FAIL(is_correlated_expr(exec_params, table->table_values_.at(j), contains))) { - LOG_WARN("failed to check values table expr correlated", K(ret)); + if (OB_ISNULL(table->values_table_def_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpect null expr", K(ret)); + } else { + ObIArray &access_exprs = table->values_table_def_->access_exprs_; + for (int64_t j = 0; OB_SUCC(ret) && !contains && j < access_exprs.count(); ++j) { + if (OB_FAIL(is_correlated_expr(exec_params, access_exprs.at(j), contains))) { + LOG_WARN("failed to check values table expr correlated", K(ret)); + } } } } @@ -15306,6 +15421,578 @@ int ObTransformUtils::extract_nullable_exprs( return ret; } +/** + * @brief ObTransformUtils::cartesian_tables_pre_split + * Try to split view table and conditions into multiple view tables + * and conditions pairs. + * + * Connect from_items in the origin subquery according to the + * Connect Rules and split the table items into multiple sets. + * + * Connect Rules: + * 1. table items in the same condition expr; + * 2. table items in the same semi info; + * 3. table items in the same outer condition. + */ +int ObTransformUtils::cartesian_tables_pre_split(ObSelectStmt *subquery, + ObIArray &outer_conditions, + ObIArray> &all_connected_tables) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(subquery)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret), K(subquery)); + } else { + const int64_t N = subquery->get_from_item_size(); + UnionFind uf(N); + ObSEArray from_tables; + if (OB_FAIL(uf.init())) { + LOG_WARN("fail to initialize union find", K(ret)); + } + for (int64_t i = 0; OB_SUCC(ret) && i < subquery->get_from_item_size(); ++i) { + TableItem *cur_table = subquery->get_table_item(subquery->get_from_item(i)); + if (OB_ISNULL(cur_table)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(from_tables.push_back(cur_table))) { + LOG_WARN("fail to push back table id", K(ret)); + } + } + // connect tables according to subquery conditions + for (int64_t i = 0; OB_SUCC(ret) && i < subquery->get_condition_size(); ++i) { + ObRawExpr *cond = subquery->get_condition_expr(i); + ObSEArray where_table_ids; + if (OB_ISNULL(cond)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(ObRawExprUtils::extract_table_ids(cond, + where_table_ids))) { + LOG_WARN("fail to extract table ids", K(ret)); + } else if (OB_FAIL(connect_tables(where_table_ids, from_tables, uf))) { + LOG_WARN("fail to connect tables", K(ret)); + } + } + // connect tables according to subquery semi info + for (int64_t i = 0; OB_SUCC(ret) && i < subquery->get_semi_infos().count(); ++i) { + SemiInfo *semi = subquery->get_semi_infos().at(i); + if (OB_ISNULL(semi)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(connect_tables(semi->left_table_ids_, from_tables, uf))) { + LOG_WARN("fail to connect tables", K(ret)); + } + } + // connect tables according to outer conditions + for (int64_t i = 0; OB_SUCC(ret) && i < outer_conditions.count(); ++i) { + ObRawExpr *cond = outer_conditions.at(i); + ObSEArray cond_table_ids; + if (OB_ISNULL(cond)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(ObRawExprUtils::extract_table_ids(cond, + cond_table_ids))) { + LOG_WARN("fail to extract table ids", K(ret)); + } else if (OB_FAIL(connect_tables(cond_table_ids, from_tables, uf))) { + LOG_WARN("fail to connect tables", K(ret)); + } + } + + // collect connected tables + ObSqlBitSet<> visited; + all_connected_tables.reuse(); + for (int64_t i = 0; OB_SUCC(ret) && i < N; ++i) + { + TableItem *table1, *table2; + ObSEArray connected_tables; + if (visited.has_member(i)) { + // do nothing + } else if (OB_FAIL(visited.add_member(i))) { + LOG_WARN("add bit set member failed", K(ret), K(i)); + } else if (OB_ISNULL(table1 = subquery->get_table_item(subquery->get_from_item(i)))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to get table item", K(ret), K(i)); + } else if (OB_FAIL(connected_tables.push_back(table1))) { + LOG_WARN("fail to push back table item", K(ret)); + } else { + for (int64_t j = i + 1; OB_SUCC(ret) && j < N; ++j) { + bool is_connected = false; + if (visited.has_member(j)) { + // do nothing + } else if (OB_FAIL(uf.is_connected(i, j, is_connected))) { + LOG_WARN("failed to check is connected", K(ret), K(i), K(j)); + } else if (!is_connected) { + // do nothing + } else if (OB_FAIL(visited.add_member(j))) { + LOG_WARN("add bit set member failed", K(ret), K(j)); + } else if (OB_ISNULL(table2 = subquery->get_table_item(subquery->get_from_item(j)))) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("fail to get table item", K(ret), K(j)); + } else if (OB_FAIL(connected_tables.push_back(table2))) { + LOG_WARN("fail to push back table item", K(ret)); + } + } + if (OB_SUCC(ret) && OB_FAIL(all_connected_tables.push_back(connected_tables))) { + LOG_WARN("fail to push back connected tables", K(ret)); + } + } + } + } + return ret; +} + + +int ObTransformUtils::do_split_cartesian_tables(ObTransformerCtx *ctx, + ObDMLStmt *stmt, + ObSelectStmt *subquery, + ObSEArray &outer_conditions, + ObIArray> &all_connected_tables, + ObIArray &right_tables, + ObIArray> &new_outer_conds) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(stmt) || OB_ISNULL(subquery)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret), K(stmt), K(subquery)); + } else if (OB_FAIL(collect_cartesian_tables(ctx, stmt, subquery, outer_conditions, + all_connected_tables, right_tables, new_outer_conds))) { + LOG_WARN("fail to collect cartesian tables", K(ret)); + } else if (OB_FAIL(create_columns_for_view_tables(ctx, stmt, right_tables, new_outer_conds))) { + LOG_WARN("fail to create columns", K(ret)); + } + return ret; +} + +int ObTransformUtils::collect_cartesian_tables(ObTransformerCtx *ctx, + ObDMLStmt *stmt, + ObSelectStmt *subquery, + ObIArray &outer_conditions, + ObIArray> &all_connected_tables, + ObIArray &right_tables, + ObIArray> &new_outer_conds) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(ctx) || OB_ISNULL(subquery)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret), K(ctx), K(subquery)); + } else { + OPT_TRACE("split cartesian tables into", all_connected_tables.count(), "view tables"); + for (int64_t i = 0; OB_SUCC(ret) && i < all_connected_tables.count(); ++i) { + TableItem *new_table_item = NULL; + ObSEArray split_outer_conds; + ObIArray &connected_tables = all_connected_tables.at(i); + if (OB_FAIL(collect_split_outer_conds(subquery, outer_conditions, + connected_tables, split_outer_conds))) { + LOG_WARN("fail to collect split semi conditions", K(ret)); + } else if (OB_FAIL(collect_split_exprs_for_view(ctx, stmt, subquery, + new_table_item, connected_tables))) { + LOG_WARN("fail to collect split exprs", K(ret)); + } else if (OB_FAIL(right_tables.push_back(new_table_item))) { + LOG_WARN("fail to push back table item", K(ret)); + } else if (OB_FAIL(new_outer_conds.push_back(split_outer_conds))) { + LOG_WARN("fail to push back semi conditions", K(ret)); + } + } + if (OB_SUCC(ret) && OB_FAIL(collect_common_conditions(ctx, + subquery, + outer_conditions, + right_tables, + new_outer_conds))) { + LOG_WARN("fail to collect common conditions", K(ret)); + } + } + return ret; +} + +int ObTransformUtils::collect_split_exprs_for_view(ObTransformerCtx *ctx, + ObDMLStmt *stmt, + ObSelectStmt *origin_subquery, + TableItem *&view_table, + ObIArray &connected_tables) { + int ret = OB_SUCCESS; + ObSqlBitSet<> table_rel_ids; + if (OB_ISNULL(stmt) || OB_ISNULL(origin_subquery)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret), K(stmt), K(origin_subquery)); + } else if (OB_FAIL(origin_subquery->get_table_rel_ids(connected_tables, + table_rel_ids))) { + LOG_WARN("failed to get table rel ids", K(ret), K(connected_tables), + K(origin_subquery->get_table_items())); + } + // collect condition expr + ObSEArray view_conds; + for (int64_t i = 0; OB_SUCC(ret) && i < origin_subquery->get_condition_size(); ++i) { + ObRawExpr *cond = origin_subquery->get_condition_expr(i); + if (OB_ISNULL(cond)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(cond->pull_relation_id())) { + LOG_WARN("fail to pull rel_id", K(ret), KPC(cond)); + } else if (!cond->get_relation_ids().overlap(table_rel_ids)) { + // do nothing + } else if (OB_FAIL(view_conds.push_back(cond))) { + LOG_WARN("failed to push back conditions", K(ret)); + } + } + if (OB_SUCC(ret) && OB_FAIL(ObOptimizerUtil::remove_item(origin_subquery->get_condition_exprs(), + view_conds))) { + LOG_WARN("fail to remove conditions from origin subquery", K(ret)); + } + // collect semi info + ObSEArray semi_infos; + ObSqlBitSet<> left_table_ids; + for (int64_t i = 0; OB_SUCC(ret) && i < origin_subquery->get_semi_info_size(); ++i) { + SemiInfo *semi_info = origin_subquery->get_semi_infos().at(i); + left_table_ids.reuse(); + if (OB_ISNULL(semi_info)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(ObTransformUtils::get_left_rel_ids_from_semi_info(origin_subquery, + semi_info, + left_table_ids))) { + LOG_WARN("failed to get left table rel ids", K(ret)); + } else if (!left_table_ids.overlap(table_rel_ids)) { + // do nothing + } else if (OB_FAIL(semi_infos.push_back(semi_info))) { + LOG_WARN("failed to push back semi info", K(ret)); + } + } + if (OB_SUCC(ret) && OB_FAIL(ObOptimizerUtil::remove_item(origin_subquery->get_semi_infos(), + semi_infos))) { + LOG_WARN("fail to remove conditions from origin subquery", K(ret)); + } + // create new view table + if (OB_FAIL(ret)) { + } else if (OB_FAIL(ObTransformUtils::add_new_table_item(ctx, + origin_subquery, + NULL, + view_table))) { + LOG_WARN("fail to generate table item", K(ret)); + } else if (OB_FAIL(ObTransformUtils::create_inline_view(ctx, + origin_subquery, + view_table, + connected_tables, + &view_conds, + &semi_infos, + NULL, + NULL, + NULL, + NULL, + NULL))) { + LOG_WARN("failed to create inline view", K(ret)); + } else if (OB_FAIL(stmt->get_table_items().push_back(view_table))) { + LOG_WARN("add table item failed", K(ret)); + } else if (OB_FAIL(stmt->rebuild_tables_hash())) { + LOG_WARN("fail to rebuild table hash", K(ret)); + } + return ret; +} + +int ObTransformUtils::collect_split_outer_conds(ObSelectStmt *origin_subquery, + ObIArray &outer_conditions, + ObIArray &connected_tables, + ObIArray &split_outer_conds) { + int ret = OB_SUCCESS; + ObSqlBitSet<> table_rel_ids; + ObSEArray table_ids; + // collect connect table ids + for (int64_t i = 0; OB_SUCC(ret) && i < connected_tables.count(); ++i) { + TableItem *table_item = connected_tables.at(i); + if (OB_ISNULL(table_item)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("table item is NULL", K(ret)); + } else if (table_item->is_joined_table()) { + const JoinedTable *join_table = static_cast(table_item); + if (OB_FAIL(append(table_ids, join_table->single_table_ids_))) { + LOG_WARN("fail to push back table id", K(ret)); + } + } else if (OB_FAIL(table_ids.push_back(table_item->table_id_))) { + LOG_WARN("fail to push back table id", K(ret)); + } + } + // collect outer conditions + for (int64_t i = 0; OB_SUCC(ret) && i < outer_conditions.count(); ++i) { + ObRawExpr *cond = outer_conditions.at(i); + ObSEArray cond_table_ids; + bool is_overlapped = false; + if (OB_ISNULL(cond)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(ObRawExprUtils::extract_table_ids(cond, + cond_table_ids))) { + LOG_WARN("fail to extract table ids", K(ret)); + } + for (int64_t j = 0; OB_SUCC(ret) && !is_overlapped && j < table_ids.count(); ++j) { + for (int64_t k = 0; OB_SUCC(ret) && k < cond_table_ids.count(); ++k) { + if (table_ids.at(j) == cond_table_ids.at(k)) { + is_overlapped = true; + break; + } + } + } + if (OB_SUCC(ret) && is_overlapped && OB_FAIL(split_outer_conds.push_back(cond))) { + LOG_WARN("fail to push back conditions", K(ret)); + } + } + if (OB_SUCC(ret) && OB_FAIL(ObOptimizerUtil::remove_item(outer_conditions, + split_outer_conds))) { + LOG_WARN("fail to remove conditions from origin subquery", K(ret)); + } + return ret; +} + +int ObTransformUtils::create_columns_for_view_tables(ObTransformerCtx *ctx, + ObDMLStmt *stmt, + ObIArray &right_tables, + ObIArray> &outer_conds) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(stmt) || OB_ISNULL(ctx) || OB_ISNULL(ctx->expr_factory_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret), K(stmt), K(ctx)); + } else if (OB_UNLIKELY(right_tables.count() != outer_conds.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unequal tables and semi conds count", K(ret), K(right_tables.count()), K(outer_conds.count())); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < right_tables.count(); ++i) { + TableItem *split_right_table = right_tables.at(i); + ObIArray &split_outer_conditions = outer_conds.at(i); + ObSelectStmt *split_subquery; + ObSEArray column_exprs; + ObSEArray upper_column_exprs; + ObSEArray new_outer_conditions; + ObSEArray subquery_table_ids; + ObRawExprCopier copier(*ctx->expr_factory_); + if (OB_ISNULL(split_right_table)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("split right table is NULL", K(ret)); + } else if (OB_ISNULL(split_subquery = split_right_table->ref_query_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("split ref query is NULL", K(ret)); + } else if (OB_FAIL(split_subquery->get_table_items(subquery_table_ids))) { + LOG_WARN("fail to get from tables", K(ret)); + } else if (OB_FAIL(ObRawExprUtils::extract_column_exprs(split_outer_conditions, + subquery_table_ids, + column_exprs))) { + LOG_WARN("fail to extract column exprs", K(ret)); + } else if (OB_FALSE_IT(split_subquery->get_select_items().reset())) { + } else if (column_exprs.empty()) { + if (OB_FAIL(ObTransformUtils::create_dummy_select_item(*split_subquery, ctx))) { + LOG_WARN("fail to create dummy select item", K(ret)); + } + } else if (OB_FAIL(ObTransformUtils::create_select_item(*ctx->allocator_, column_exprs, + split_subquery))) { + LOG_WARN("failed to create select item", K(ret)); + } else if (OB_FAIL(ObTransformUtils::create_columns_for_view(ctx, *split_right_table, stmt, + upper_column_exprs))) { + LOG_WARN("failed to create columns for view", K(ret)); + } else if (OB_FAIL(copier.add_replaced_expr(column_exprs, upper_column_exprs))) { + LOG_WARN("failed to add replace pair", K(ret)); + } else if (OB_FAIL(copier.copy_on_replace(split_outer_conditions, new_outer_conditions))) { + LOG_WARN("failed to copy on replace expr", K(ret)); + } else if (OB_FAIL(split_outer_conditions.assign(new_outer_conditions))) { + LOG_WARN("fail to assign semi conditions", K(ret)); + } + } + } + return ret; +} + +int ObTransformUtils::collect_common_conditions(ObTransformerCtx *ctx, + ObSelectStmt *origin_subquery, + ObIArray &outer_conditions, + ObIArray &right_tables, + ObIArray> &new_outer_conds) { + int ret = OB_SUCCESS; + if (OB_ISNULL(ctx) || OB_ISNULL(ctx->expr_factory_) || OB_ISNULL(origin_subquery)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (right_tables.count() != new_outer_conds.count()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unequal tables and semi conds count", K(ret), K(right_tables.count()), K(new_outer_conds.count())); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < right_tables.count(); ++i) { + ObSelectStmt *subquery; + ObIArray &outer_cond = new_outer_conds.at(i); + if (OB_ISNULL(right_tables.at(i)) || OB_ISNULL(subquery = right_tables.at(i)->ref_query_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret), K(right_tables.at(i))); + } else { + // collect common subquery condition + for (int64_t j = 0; OB_SUCC(ret) && j < origin_subquery->get_condition_size(); ++j) { + ObRawExpr *cond = origin_subquery->get_condition_expr(j); + ObRawExpr *new_cond; + if (OB_ISNULL(cond)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(ObRawExprCopier::copy_expr_node(*ctx->expr_factory_, cond, new_cond))) { + LOG_WARN("failed to copy expr", K(ret)); + } else if (OB_FAIL(subquery->get_condition_exprs().push_back(new_cond))) { + LOG_WARN("failed to push back conditions", K(ret)); + } + } + // ckeck there is no common semi info + if (OB_SUCC(ret) && OB_UNLIKELY(origin_subquery->get_semi_info_size() > 0)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("origin subquery should not have semi info", K(ret), K(origin_subquery->get_semi_info_size())); + } + // collect common outer condition + for (int64_t j = 0; OB_SUCC(ret) && j < outer_conditions.count(); ++j) { + ObRawExpr *cond = outer_conditions.at(j); + ObRawExpr *new_cond; + if (OB_ISNULL(cond)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } else if (OB_FAIL(ObRawExprCopier::copy_expr_node(*ctx->expr_factory_, cond, new_cond))) { + LOG_WARN("failed to copy expr", K(ret)); + } else if (OB_FAIL(outer_cond.push_back(new_cond))) { + LOG_WARN("fail to push back conditions", K(ret)); + } + } + } + } + } + return ret; +} + +int ObTransformUtils::connect_tables(const ObIArray &table_ids, + const ObIArray &from_tables, + UnionFind &uf) +{ + int ret = OB_SUCCESS; + ObSEArray indices; + if (OB_FAIL(get_idx_from_table_ids(table_ids, + from_tables, + indices))) { + LOG_WARN("failed to get indices of from table ids", K(ret)); + } else if (indices.count() <= 1) { + // do nothing + } else { + // connect tables appeared in a condition + // we store table indices instead of table item to leverage the consumption + for (int64_t i = 1; OB_SUCC(ret) && i < indices.count(); ++i) { + if (OB_FAIL(uf.connect(indices.at(0), indices.at(i)))) { + LOG_WARN("failed to connect nodes", K(ret), K(i)); + } + } + } + return ret; +} + +int ObTransformUtils::check_contain_correlated_function_table(const 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 { + for (int64_t i = 0; OB_SUCC(ret) && !is_contain && i < stmt->get_table_items().count(); ++i) { + const 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_function_table()) { + // do nothing + } else if (OB_ISNULL(table->function_table_expr_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpect null expr", K(ret)); + } else if (!table->function_table_expr_->get_relation_ids().is_empty()) { + is_contain = true; + } + } + } + return ret; +} + +int ObTransformUtils::check_contain_correlated_json_table(const 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 { + for (int64_t i = 0; OB_SUCC(ret) && !is_contain && i < stmt->get_table_items().count(); ++i) { + const 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_json_table()) { + // do nothing + } else if (OB_ISNULL(table->json_table_def_) || OB_ISNULL(table->json_table_def_->doc_expr_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpect null expr", K(ret), K(table->json_table_def_)); + } else if (!table->json_table_def_->doc_expr_->get_relation_ids().is_empty()) { + is_contain = true; + } + } + } + return ret; +} + +int ObTransformUtils::check_contain_cannot_duplicate_expr(const ObIArray &exprs, bool &is_contain) { + int ret = OB_SUCCESS; + is_contain = false; + for (int64_t i = 0; OB_SUCC(ret) && !is_contain && i < exprs.count(); ++i) { + if (OB_FAIL(recursive_check_cannot_duplicate_expr(exprs.at(i), is_contain))) { + LOG_WARN("fail to check cannot duplicate expr", K(ret), K(i), K(exprs)); + } + } + return ret; +} + +int ObTransformUtils::recursive_check_cannot_duplicate_expr(const ObRawExpr *expr, bool &is_contain) { + int ret = OB_SUCCESS; + if (OB_ISNULL(expr)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("expr is null", K(ret)); + } else if (expr->has_flag(CNT_DYNAMIC_USER_VARIABLE) + || expr->has_flag(CNT_STATE_FUNC) + || expr->has_flag(CNT_RAND_FUNC)) { + is_contain = true; + } else if (!expr->is_deterministic()) { + is_contain = true; + } else { + for (int64_t i = 0; OB_SUCC(ret) && !is_contain && i < expr->get_param_count(); ++i) { + if (OB_FAIL(SMART_CALL(recursive_check_cannot_duplicate_expr(expr->get_param_expr(i), is_contain)))) { + LOG_WARN("fail to check cannot duplicate expr", K(ret), K(i), KPC(expr)); + } + } + } + return ret; +} + +int ObTransformUtils::get_idx_from_table_ids(const ObIArray &src_table_ids, + const ObIArray &target_tables, + ObIArray &indices) +{ + int ret = OB_SUCCESS; + for (int64_t i = 0; OB_SUCC(ret) && i < target_tables.count(); ++i) { + TableItem *target_table = target_tables.at(i); + if (OB_ISNULL(target_table)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("get unexpected null", K(ret)); + } + for (int64_t j = 0; OB_SUCC(ret) && j < src_table_ids.count(); ++j) { + if (src_table_ids.at(j) == target_table->table_id_) { + if (OB_FAIL(indices.push_back(i))) { + LOG_WARN("failed to push back index", K(ret)); + } + break; + } else if (target_table->is_joined_table()) { + if (is_contain(static_cast(target_table)->single_table_ids_, + src_table_ids.at(j))) { + if (!is_contain(indices, i) && OB_FAIL(indices.push_back(i))) { + LOG_WARN("failed to push back index", K(ret)); + } + break; + } + } + } + } + return ret; +} + int ObTransformUtils::check_contain_correlated_lateral_table(const TableItem *table_item, bool &is_contain) { int ret = OB_SUCCESS; @@ -15491,6 +16178,12 @@ bool ObTransformUtils::is_full_group_by(ObSelectStmt& stmt, ObSQLMode mode) return !stmt.has_order_by() && is_only_full_group_by_on(mode); } +bool ObTransformUtils::is_enable_values_table_rewrite(const uint64_t compat_version) +{ + return compat_version >= COMPAT_VERSION_4_3_2 || + (compat_version >= COMPAT_VERSION_4_2_2 && compat_version < COMPAT_VERSION_4_3_0); +} + int ObTransformUtils::check_need_calc_match_score(ObExecContext *exec_ctx, const ObDMLStmt* stmt, ObRawExpr* match_expr, diff --git a/src/sql/rewrite/ob_transform_utils.h b/src/sql/rewrite/ob_transform_utils.h index 188b693886..4d858cc72e 100644 --- a/src/sql/rewrite/ob_transform_utils.h +++ b/src/sql/rewrite/ob_transform_utils.h @@ -19,6 +19,7 @@ #include "sql/resolver/dml/ob_del_upd_stmt.h" #include "sql/rewrite/ob_transform_rule.h" #include "sql/optimizer/ob_fd_item.h" +#include "sql/rewrite/ob_union_find.h" namespace oceanbase { namespace share { @@ -595,6 +596,18 @@ public: const TableItem *table_item, ObIArray &index_ids); + static int get_range_column_items_by_ids(const ObDMLStmt *stmt, + uint64_t table_id, + const ObIArray &column_ids, + ObIArray &column_items); + + static int check_index_extract_query_range(const ObDMLStmt *stmt, + uint64_t table_id, + const ObIArray &index_cols, + const ObIArray &predicate_exprs, + ObTransformerCtx *ctx, + bool &is_match); + static int is_match_index(ObSqlSchemaGuard *schema_guard, const ObDMLStmt *stmt, const ObColumnRefRawExpr *col_expr, @@ -602,7 +615,9 @@ public: EqualSets *equal_sets = NULL, ObIArray *const_exprs = NULL, ObIArray *col_exprs = NULL, - const bool need_match_col_exprs = false); + const bool need_match_col_exprs = false, + const bool need_check_query_range = false, + ObTransformerCtx *ctx = NULL); static int is_match_index(const ObDMLStmt *stmt, const ObIArray &index_cols, @@ -1268,6 +1283,7 @@ public: * Please use with care. */ static int check_error_free_expr(ObRawExpr *expr, bool &is_error_free); + static int check_error_free_exprs(ObIArray &exprs, bool &is_error_free); static int build_row_expr(ObRawExprFactory& expr_factory, common::ObIArray& param_exprs, ObOpRawExpr*& row_expr); @@ -1805,7 +1821,7 @@ public: ObRawExpr *&output, bool is_count_star = false); static int refresh_select_items_name(ObIAllocator &allocator, ObSelectStmt *select_stmt); - static int refresh_column_items_name(ObSelectStmt *stmt, int64_t table_id); + static int refresh_column_items_name(ObDMLStmt *stmt, int64_t table_id); static int get_real_alias_name(ObSelectStmt *stmt, int64_t sel_idx, ObString& alias_name); @@ -1945,6 +1961,28 @@ public: bool &is_ref); static int check_contain_correlated_lateral_table(ObDMLStmt *stmt, bool &is_contain); + + static int cartesian_tables_pre_split(ObSelectStmt *subquery, + ObIArray &outer_conditions, + ObIArray> &all_connected_tables); + static int do_split_cartesian_tables(ObTransformerCtx *ctx, + ObDMLStmt *stmt, + ObSelectStmt *subquery, + ObSEArray &outer_conditions, + ObIArray> &all_connected_tables, + ObIArray &right_tables, + ObIArray> &new_outer_conds); + static int create_columns_for_view_tables(ObTransformerCtx *ctx, + ObDMLStmt *stmt, + ObIArray &right_tables, + ObIArray> &outer_conds); + static int connect_tables(const ObIArray &table_ids, + const ObIArray &from_tables, + UnionFind &uf); + static int check_contain_correlated_function_table(const ObDMLStmt *stmt, bool &is_contain); + static int check_contain_correlated_json_table(const ObDMLStmt *stmt, bool &is_contain); + static int check_contain_cannot_duplicate_expr(const ObIArray &exprs, bool &is_contain); + static bool is_enable_values_table_rewrite(const uint64_t compat_version); // check whether the score calculated by match expr is actually utilized static int check_need_calc_match_score(ObExecContext *exec_ctx, const ObDMLStmt* stmt, @@ -2008,6 +2046,33 @@ private: static int check_convert_string_safely(const ObRawExpr *expr, const ObRawExpr *src_expr, bool &is_safe); + + static int get_idx_from_table_ids(const ObIArray &src_table_ids, + const ObIArray &target_tables, + ObIArray &indices); + + static int collect_cartesian_tables(ObTransformerCtx *ctx, + ObDMLStmt *stmt, + ObSelectStmt *subquery, + ObIArray &outer_conditions, + ObIArray> &all_connected_tables, + ObIArray &right_tables, + ObIArray> &new_outer_conds); + static int collect_split_exprs_for_view(ObTransformerCtx *ctx, + ObDMLStmt *stmt, + ObSelectStmt *origin_subquery, + TableItem *&view_table, + ObIArray &connected_tables); + static int collect_split_outer_conds(ObSelectStmt *origin_subquery, + ObIArray &outer_conditions, + ObIArray &connected_tables, + ObIArray &split_outer_conds); + static int collect_common_conditions(ObTransformerCtx *ctx, + ObSelectStmt *origin_subquery, + ObIArray &outer_conditions, + ObIArray &right_tables, + ObIArray> &new_outer_conds); + static int recursive_check_cannot_duplicate_expr(const ObRawExpr *expr, bool &is_contain); static int inner_check_need_calc_match_score(ObExecContext *exec_ctx, ObRawExpr* expr, ObRawExpr* match_expr, diff --git a/src/sql/rewrite/ob_transform_view_merge.cpp b/src/sql/rewrite/ob_transform_view_merge.cpp index e272615197..1a878a2b53 100644 --- a/src/sql/rewrite/ob_transform_view_merge.cpp +++ b/src/sql/rewrite/ob_transform_view_merge.cpp @@ -349,7 +349,7 @@ int ObTransformViewMerge::check_semi_right_table_can_be_merged(ObDMLStmt *stmt, bool has_rownum = false; bool force_merge = false; bool force_no_merge = false; - if (OB_ISNULL(stmt) || OB_ISNULL(ref_query)) { + if (OB_ISNULL(stmt) || OB_ISNULL(ref_query) || OB_ISNULL(stmt->get_query_ctx())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret), K(stmt), K(ref_query)); } else if (OB_FAIL(check_hint_allowed_merge(*stmt, @@ -372,7 +372,8 @@ int ObTransformViewMerge::check_semi_right_table_can_be_merged(ObDMLStmt *stmt, || ref_query->is_hierarchical_query() || ref_query->has_ora_rowscn() || (lib::is_mysql_mode() && ref_query->has_for_update()) - || ref_query->is_values_table_query()) { + || (ref_query->is_values_table_query() && + !ObTransformUtils::is_enable_values_table_rewrite(stmt->get_query_ctx()->optimizer_features_enable_version_))) { can_be = false; } else if (OB_FAIL(ref_query->has_rownum(has_rownum))) { LOG_WARN("failed to check has rownum expr", K(ret)); @@ -513,7 +514,7 @@ int ObTransformViewMerge::check_basic_validity(ObDMLStmt *parent_stmt, bool force_no_merge = false; bool is_select_expr_valid = false; ObSEArray select_exprs; - if (OB_ISNULL(parent_stmt) || OB_ISNULL(child_stmt)) { + if (OB_ISNULL(parent_stmt) || OB_ISNULL(child_stmt) || OB_ISNULL(parent_stmt->get_query_ctx())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); } else if (OB_FAIL(check_hint_allowed_merge(*parent_stmt, @@ -535,7 +536,8 @@ int ObTransformViewMerge::check_basic_validity(ObDMLStmt *parent_stmt, || child_stmt->has_window_function() || child_stmt->has_sequence() || child_stmt->has_ora_rowscn() - || child_stmt->is_values_table_query()) { + || (child_stmt->is_values_table_query() && + !ObTransformUtils::is_enable_values_table_rewrite(parent_stmt->get_query_ctx()->optimizer_features_enable_version_))) { can_be = false; OPT_TRACE("not a valid view"); } else if (!force_merge && parent_stmt->get_table_size() > 1 && child_stmt->get_table_size() > 1 && diff --git a/src/sql/rewrite/ob_transform_where_subquery_pullup.cpp b/src/sql/rewrite/ob_transform_where_subquery_pullup.cpp index c1cb65cdaf..20fb988743 100644 --- a/src/sql/rewrite/ob_transform_where_subquery_pullup.cpp +++ b/src/sql/rewrite/ob_transform_where_subquery_pullup.cpp @@ -522,18 +522,10 @@ int ObWhereSubQueryPullup::pullup_correlated_subquery_as_view(ObDMLStmt *stmt, ObQueryRefRawExpr *query_ref) { int ret = OB_SUCCESS; - TableItem *right_table = NULL; - SemiInfo *info = NULL; - ObSEArray semi_conds; if (OB_ISNULL(stmt) || OB_ISNULL(subquery) || OB_ISNULL(expr) || OB_ISNULL(ctx_) || OB_ISNULL(ctx_->allocator_) || OB_ISNULL(ctx_->expr_factory_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected null", K(ret), K(stmt), K(subquery), K(expr)); - } else if (OB_FAIL(ObTransformUtils::add_new_table_item(ctx_, stmt, subquery, right_table))) { - LOG_WARN("failed to add new table_item", K(ret)); - } else if (OB_ISNULL(right_table)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("right_table should not be null", K(ret), K(right_table)); } else if (OB_FAIL(ObOptimizerUtil::remove_item(stmt->get_condition_exprs(), expr))) { LOG_WARN("failed to remove condition expr", K(ret)); } else { @@ -551,8 +543,12 @@ int ObWhereSubQueryPullup::pullup_correlated_subquery_as_view(ObDMLStmt *stmt, ObSEArray right_hand_exprs; ObSEArray candi_semi_conds; ObSEArray column_exprs; - ObSEArray upper_column_exprs; - ObRawExprCopier copier(*ctx_->expr_factory_); + ObSEArray right_tables; + ObSEArray, 4> semi_conds; + ObJoinType join_type = UNKNOWN_JOIN; + SemiInfoSplitHelper split_helper; + bool can_split = false; + if (OB_FAIL(ObTransformUtils::get_correlated_conditions(query_ref->get_exec_params(), subquery->get_condition_exprs(), candi_semi_conds))) { @@ -574,51 +570,108 @@ int ObWhereSubQueryPullup::pullup_correlated_subquery_as_view(ObDMLStmt *stmt, candi_semi_conds))) { LOG_WARN("failed to remove condition expr", K(ret)); } else if (OB_FALSE_IT(subquery->get_select_items().reset())) { - } else if (column_exprs.empty()) { - if (OB_FAIL(ObTransformUtils::create_dummy_select_item(*subquery, ctx_))) { - LOG_WARN("failed to create dummy select item", K(ret)); - } else if (OB_FAIL(semi_conds.assign(candi_semi_conds))) { - LOG_WARN("failed to assign semi conditions", K(ret)); + } else if (T_OP_EXISTS == expr->get_expr_type() || expr->has_flag(IS_WITH_ANY)) { + join_type = LEFT_SEMI_JOIN; + // add as LEFT_SEMI/ANTI_JOIN, + // RIGHT_SEMI/ANTI_JOIN path is added in ObJoinOrder::generate_join_paths() + } else if (T_OP_NOT_EXISTS == expr->get_expr_type() || expr->has_flag(IS_WITH_ALL)) { + join_type = LEFT_ANTI_JOIN; + } else { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected expr type", K(ret), K(expr->get_expr_type())); + } + + if (OB_FAIL(ret)) { + // do nothing + } else if (OB_FAIL(check_can_split(subquery, candi_semi_conds, join_type, split_helper))) { + LOG_WARN("fail to check can split", K(ret)); + } else if (!split_helper.can_split_) { + TableItem *table_item = NULL; + ObSEArray upper_column_exprs; + ObSEArray final_semi_conds; + ObRawExprCopier copier(*ctx_->expr_factory_); + if (OB_FAIL(ObTransformUtils::add_new_table_item(ctx_, stmt, subquery, table_item))) { + LOG_WARN("failed to add new table_item", K(ret)); + } else if (OB_ISNULL(table_item)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("table item should not be null", K(ret)); + } else if (column_exprs.empty()) { + if (OB_FAIL(ObTransformUtils::create_dummy_select_item(*subquery, ctx_))) { + LOG_WARN("failed to create dummy select item", K(ret)); + } else if (OB_FAIL(final_semi_conds.assign(candi_semi_conds))) { + LOG_WARN("failed to assign semi conditions", K(ret)); + } + } else if (OB_FAIL(ObTransformUtils::create_select_item(*ctx_->allocator_, column_exprs, + subquery))) { + LOG_WARN("failed to create select item", K(ret)); + } else if (OB_FAIL(ObTransformUtils::create_columns_for_view(ctx_, *table_item, stmt, + upper_column_exprs))) { + LOG_WARN("failed to create columns for view", K(ret)); + } else if (OB_FAIL(copier.add_replaced_expr(column_exprs, upper_column_exprs))) { + LOG_WARN("failed to add replace pair", K(ret)); + } else if (OB_FAIL(copier.copy_on_replace(candi_semi_conds, final_semi_conds))) { + LOG_WARN("failed to copy on replace expr", K(ret)); + } + if (OB_FAIL(ret)) { + } else if (OB_FAIL(right_tables.push_back(table_item))) { + LOG_WARN("fail to push back table item", K(ret)); + } else if (OB_FAIL(semi_conds.push_back(final_semi_conds))) { + LOG_WARN("fail to push back semi conditions", K(ret)); + } + } else if (OB_FAIL(ObTransformUtils::do_split_cartesian_tables(ctx_, stmt, subquery, candi_semi_conds, + split_helper.connected_tables_, right_tables, semi_conds))) { + LOG_WARN("fail to split cartesian tables", K(ret)); + } + if (OB_FAIL(ret)) { + // do nothing + } else if (OB_UNLIKELY(right_tables.count() != semi_conds.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unequal tables and semi conds count", K(ret), K(right_tables.count()), K(semi_conds.count())); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < right_tables.count(); ++i) { + SemiInfo *info = NULL; + TableItem *split_right_table = right_tables.at(i); + ObIArray &split_semi_conditions = semi_conds.at(i); + ObSelectStmt *split_subquery = NULL; + if (OB_ISNULL(split_right_table)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("split right table is NULL", K(ret)); + } else if (OB_ISNULL(split_subquery = split_right_table->ref_query_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("split ref query is NULL", K(ret)); + } else if (OB_FAIL(ObTransformUtils::decorrelate(split_semi_conditions, query_ref->get_exec_params()))) { + LOG_WARN("failed to decorrelate semi conditions", K(ret)); + } else if (OB_FAIL(generate_semi_info(stmt, split_right_table, split_semi_conditions, join_type, info))) { + LOG_WARN("failed to generate semi info", K(ret)); + } else if (OB_FAIL(split_subquery->adjust_subquery_list())) { + LOG_WARN("failed to adjust subquery list", K(ret)); + } else if (OB_FAIL(split_subquery->formalize_stmt(ctx_->session_info_))) { + LOG_WARN("formalize child stmt failed", K(ret)); + } } - } else if (OB_FAIL(ObTransformUtils::create_select_item(*ctx_->allocator_, column_exprs, - subquery))) { - LOG_WARN("failed to create select item", K(ret)); - } else if (OB_FAIL(ObTransformUtils::create_columns_for_view(ctx_, *right_table, stmt, - upper_column_exprs))) { - LOG_WARN("failed to create columns for view", K(ret)); - } else if (OB_FAIL(copier.add_replaced_expr(column_exprs, upper_column_exprs))) { - LOG_WARN("failed to add replace pair", K(ret)); - } else if (OB_FAIL(copier.copy_on_replace(candi_semi_conds, semi_conds))) { - LOG_WARN("failed to copy on replace expr", K(ret)); } } - if (OB_SUCC(ret)) { - if (OB_FAIL(ObTransformUtils::decorrelate(semi_conds, query_ref->get_exec_params()))) { - LOG_WARN("failed to decorrelate semi conditions", K(ret)); - } else if (OB_FAIL(generate_semi_info(stmt, expr, right_table, semi_conds, info))) { - LOG_WARN("failed to generate semi info", K(ret)); - } else if (OB_FAIL(stmt->formalize_stmt(ctx_->session_info_))) { - LOG_WARN("formalize child stmt failed", K(ret)); - } + if (OB_SUCC(ret) && OB_FAIL(stmt->formalize_stmt(ctx_->session_info_))) { + LOG_WARN("failed to formalize stmt", K(ret)); } return ret; } // create semi info with a generate table int ObWhereSubQueryPullup::generate_semi_info(ObDMLStmt *stmt, - ObRawExpr *expr, TableItem *right_table, ObIArray &semi_conditions, + ObJoinType join_type, SemiInfo *&semi_info) { int ret = OB_SUCCESS; semi_info = NULL; SemiInfo *info = NULL; ObIAllocator *alloc = NULL; - if (OB_ISNULL(stmt) || OB_ISNULL(expr) || OB_ISNULL(right_table) || + if (OB_ISNULL(stmt) || OB_ISNULL(right_table) || OB_ISNULL(ctx_) || OB_ISNULL(alloc = ctx_->allocator_)) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("NULL pointer error", K(stmt), K(expr), K(right_table), K(alloc), K(ret)); + LOG_WARN("NULL pointer error", K(stmt), K(right_table), K(alloc), K(ret)); } else if (OB_UNLIKELY(!right_table->is_generated_table())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected right table type in semi info", K(ret), K(right_table->type_)); @@ -633,15 +686,8 @@ int ObWhereSubQueryPullup::generate_semi_info(ObDMLStmt *stmt, LOG_WARN("failed to fill semi left table ids", K(ret)); } else if (OB_FAIL(stmt->add_semi_info(info))) { LOG_WARN("failed to add semi info", K(ret)); - } else if (T_OP_EXISTS == expr->get_expr_type() || expr->has_flag(IS_WITH_ANY)) { - info->join_type_ = LEFT_SEMI_JOIN; - // add as LEFT_SEMI/ANTI_JOIN, - // RIGHT_SEMI/ANTI_JOIN path is added in ObJoinOrder::generate_join_paths() - } else if (T_OP_NOT_EXISTS == expr->get_expr_type() || expr->has_flag(IS_WITH_ALL)) { - info->join_type_ = LEFT_ANTI_JOIN; } else { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("unexpected expr type", K(ret), K(expr->get_expr_type())); + info->join_type_ = join_type; } if (OB_SUCC(ret)) { @@ -995,10 +1041,13 @@ int ObWhereSubQueryPullup::pullup_non_correlated_subquery_as_view(ObDMLStmt *stm { int ret = OB_SUCCESS; ObRawExprFactory *expr_factory = NULL; - TableItem *table_item = NULL; - ObSEArray column_exprs; + ObSEArray right_exprs; ObSEArray new_conditions; - SemiInfo *info = NULL; + ObSEArray right_tables; + ObSEArray, 4> semi_conds; + ObJoinType join_type = UNKNOWN_JOIN; + SemiInfoSplitHelper split_helper; + bool can_split; if (OB_ISNULL(stmt) || OB_ISNULL(subquery) || OB_ISNULL(expr)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("stmt or subquery, query ctx should not be null", @@ -1006,26 +1055,81 @@ int ObWhereSubQueryPullup::pullup_non_correlated_subquery_as_view(ObDMLStmt *stm } else if (OB_ISNULL(ctx_) || OB_ISNULL(expr_factory = ctx_->expr_factory_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("ctx or expr_factory is null", K(ret), K(ctx_), K(expr_factory)); - } else if (!expr->has_flag(IS_WITH_ANY) && !expr->has_flag(IS_WITH_ALL)) { + } else if (expr->has_flag(IS_WITH_ANY)) { + join_type = LEFT_SEMI_JOIN; + // add as LEFT_SEMI/ANTI_JOIN, + // RIGHT_SEMI/ANTI_JOIN path is added in ObJoinOrder::generate_join_paths() + } else if (expr->has_flag(IS_WITH_ALL)) { + join_type = LEFT_ANTI_JOIN; + } else { ret = OB_ERR_UNEXPECTED; - LOG_WARN("expression is invalid", K(ret)); - } else if (OB_FAIL(ObTransformUtils::add_new_table_item(ctx_, stmt, subquery, table_item))) { - LOG_WARN("failed to add new table_item", K(ret)); - } else if (OB_ISNULL(table_item)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("table_item should not be null", K(ret)); + LOG_WARN("unexpected expr type", K(ret), K(expr->get_expr_type())); + } + + if (OB_FAIL(ret)) { + // do nothing } else if (OB_FAIL(ObOptimizerUtil::remove_item(stmt->get_condition_exprs(), expr))) { LOG_WARN("failed to remove condition expr", K(ret)); - } else if (OB_FAIL(ObTransformUtils::create_columns_for_view(ctx_, *table_item, - stmt, column_exprs))) { - LOG_WARN("failed to create columns for view", K(ret)); - } else if (OB_FAIL(generate_conditions(stmt, column_exprs, subquery, expr, new_conditions, new_conditions))) { - //首先根据left_hand和subquery的targetlist生成新的条件 + } else if (OB_FAIL(subquery->get_select_exprs(right_exprs))) { + LOG_WARN("failed to get select exprs", K(ret)); + } else if (OB_FAIL(generate_conditions(stmt, right_exprs, subquery, expr, + new_conditions, new_conditions))) { LOG_WARN("failed to generate new condition exprs", K(ret)); - } else if (OB_FAIL(generate_semi_info(stmt, expr, table_item, new_conditions, info))) { - LOG_WARN("generate semi info failed", K(ret)); - } else if (OB_FAIL(stmt->formalize_stmt(ctx_->session_info_))) { - LOG_WARN("formalize child stmt failed", K(ret)); + } else if (OB_FAIL(check_can_split(subquery, new_conditions, join_type, split_helper))) { + LOG_WARN("fail to check can split", K(ret)); + } else if (!split_helper.can_split_) { + TableItem *table_item = NULL; + right_exprs.reuse(); + new_conditions.reuse(); + if (OB_FAIL(ObTransformUtils::add_new_table_item(ctx_, stmt, subquery, table_item))) { + LOG_WARN("failed to add new table_item", K(ret)); + } else if (OB_ISNULL(table_item)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("table_item should not be null", K(ret)); + } else if (OB_FAIL(ObTransformUtils::create_columns_for_view(ctx_, *table_item, + stmt, right_exprs))) { + LOG_WARN("failed to create columns for view", K(ret)); + } else if (OB_FAIL(generate_conditions(stmt, right_exprs, subquery, expr, + new_conditions, new_conditions))) { + LOG_WARN("failed to generate new condition exprs", K(ret)); + } else if (OB_FAIL(right_tables.push_back(table_item))) { + LOG_WARN("fail to push back table item", K(ret)); + } else if (OB_FAIL(semi_conds.push_back(new_conditions))) { + LOG_WARN("fail to push back semi conditions", K(ret)); + } + } else if (OB_FALSE_IT(subquery->get_select_items().reset())) { + } else if (OB_FAIL(ObTransformUtils::do_split_cartesian_tables(ctx_, stmt, subquery, new_conditions, + split_helper.connected_tables_, right_tables, semi_conds))) { + LOG_WARN("fail to split cartesian tables", K(ret)); + } + if (OB_FAIL(ret)) { + // do nothing + } else if (OB_UNLIKELY(right_tables.count() != semi_conds.count())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unequal tables and semi conds count", K(ret), K(right_tables.count()), K(semi_conds.count())); + } else { + for (int64_t i = 0; OB_SUCC(ret) && i < right_tables.count(); ++i) { + SemiInfo *info = NULL; + TableItem *split_right_table = right_tables.at(i); + ObIArray &split_semi_conditions = semi_conds.at(i); + ObSelectStmt *split_subquery = NULL; + if (OB_ISNULL(split_right_table)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("split right table is NULL", K(ret)); + } else if (OB_ISNULL(split_subquery = split_right_table->ref_query_)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("split ref query is NULL", K(ret)); + } else if (OB_FAIL(generate_semi_info(stmt, split_right_table, split_semi_conditions, join_type, info))) { + LOG_WARN("failed to generate semi info", K(ret)); + } else if (OB_FAIL(split_subquery->adjust_subquery_list())) { + LOG_WARN("failed to adjust subquery list", K(ret)); + } else if (OB_FAIL(split_subquery->formalize_stmt(ctx_->session_info_))) { + LOG_WARN("formalize child stmt failed", K(ret)); + } + } + } + if (OB_SUCC(ret) && OB_FAIL(stmt->formalize_stmt(ctx_->session_info_))) { + LOG_WARN("failed to formalize stmt", K(ret)); } return ret; } @@ -1223,7 +1327,8 @@ int ObWhereSubQueryPullup::check_subquery_validity(ObDMLStmt &stmt, bool is_valid = true; OPT_TRACE("try to pullup single set subquery:", query_ref); if (OB_ISNULL(root_expr) || OB_ISNULL(query_ref) || - OB_ISNULL(subquery = query_ref->get_ref_stmt())) { + OB_ISNULL(subquery = query_ref->get_ref_stmt()) || + OB_ISNULL(stmt.get_query_ctx())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("subquery is null", K(ret), K(subquery)); } else if (OB_FAIL(check_hint_allowed_unnest(stmt, *subquery, is_valid))) { @@ -1231,7 +1336,9 @@ int ObWhereSubQueryPullup::check_subquery_validity(ObDMLStmt &stmt, } else if (!is_valid) { // do nothing OPT_TRACE("hint reject transform"); - } else if (!subquery->is_spj() || subquery->has_subquery() || subquery->is_values_table_query()) { + } else if (!subquery->is_spj() || subquery->has_subquery() || + (subquery->is_values_table_query() && + !ObTransformUtils::is_enable_values_table_rewrite(stmt.get_query_ctx()->optimizer_features_enable_version_))) { is_valid = false; OPT_TRACE("subquery is not spj or has subquery"); } else if (OB_FAIL(subquery->get_column_exprs(columns))) { @@ -1718,5 +1825,83 @@ int ObWhereSubQueryPullup::construct_transform_hint(ObDMLStmt &stmt, void *trans return ret; } +/** + * @brief ObWhereSubQueryPullup::check_can_split + * Check whether the semi join can be split. + * + * Check Rules: + * 1. subquery is a spj query; + * 2. subquery should not contains having; + * 3. subquery should not contains subquery; + * 4. join type is SEMI JOIN (ANTI JOIN can be support later). + * + * Besides, pre split subquery and check the subquery can be split into + * more than ONE view tables. + */ +int ObWhereSubQueryPullup::check_can_split(ObSelectStmt *subquery, + ObIArray &semi_conditions, + ObJoinType join_type, + SemiInfoSplitHelper &helper) +{ + int ret = OB_SUCCESS; + bool can_split = true; + bool is_contain = false; + if (OB_ISNULL(subquery) || OB_ISNULL(subquery->get_query_ctx())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null", K(ret), K(subquery)); + } else if (subquery->get_query_ctx()->optimizer_features_enable_version_ < COMPAT_VERSION_4_2_3 + || (subquery->get_query_ctx()->optimizer_features_enable_version_ >= COMPAT_VERSION_4_3_0 + && subquery->get_query_ctx()->optimizer_features_enable_version_ < COMPAT_VERSION_4_3_2)) { + can_split = false; + OPT_TRACE("can not split cartesian tables, optimizer features version lower than 4.3.2"); + } else if (join_type != LEFT_SEMI_JOIN) { + can_split = false; + OPT_TRACE("can not split cartesian tables, not a semi join"); + } else if (!subquery->is_spj()) { + can_split = false; + OPT_TRACE("can not split cartesian tables, it is not a spj"); + } else if (subquery->has_having()) { + can_split = false; + OPT_TRACE("can not split cartesian tables, contain having"); + } else if (subquery->has_subquery()) { + can_split = false; + OPT_TRACE("can not split cartesian tables, contain subquery"); + } else if (OB_FAIL(ObTransformUtils::check_contain_cannot_duplicate_expr(semi_conditions, + is_contain))) { + LOG_WARN("failed to check contain can not duplicate expr", K(ret)); + } else if (is_contain) { + can_split = false; + OPT_TRACE("can not split cartesian tables, contain can not duplicate function"); + } else if (OB_FAIL(ObTransformUtils::check_contain_correlated_function_table(subquery, + is_contain))) { + LOG_WARN("failed to check contain correlated function table", K(ret)); + } else if (is_contain) { + can_split = false; + OPT_TRACE("can not split cartesian tables, contain correlated function table"); + } else if (OB_FAIL(ObTransformUtils::check_contain_correlated_json_table(subquery, + is_contain))) { + LOG_WARN("failed to check contain correlated json table", K(ret)); + } else if (is_contain) { + can_split = false; + OPT_TRACE("can not split cartesian tables, contain correlated json table"); + } else if (OB_FAIL(ObTransformUtils::check_contain_correlated_lateral_table(subquery, + is_contain))) { + LOG_WARN("failed to check contain correlated lateral table", K(ret)); + } else if (is_contain) { + can_split = false; + OPT_TRACE("can not split cartesian tables, contain correlated lateral derived table"); + } else if (OB_FAIL(ObTransformUtils::cartesian_tables_pre_split(subquery, + semi_conditions, helper.connected_tables_))){ + LOG_WARN("fail to pre split cartesian tables", K(ret)); + } else if (helper.connected_tables_.count() <= 1) { + can_split = false; + OPT_TRACE("can not split cartesian tables, all tables releated"); + } + if (OB_SUCC(ret)) { + helper.can_split_ = can_split; + } + return ret; +} + } } diff --git a/src/sql/rewrite/ob_transform_where_subquery_pullup.h b/src/sql/rewrite/ob_transform_where_subquery_pullup.h index 67bbc0b29d..0016abd948 100644 --- a/src/sql/rewrite/ob_transform_where_subquery_pullup.h +++ b/src/sql/rewrite/ob_transform_where_subquery_pullup.h @@ -15,6 +15,7 @@ #include "sql/rewrite/ob_transform_rule.h" #include "sql/resolver/dml/ob_select_stmt.h" +#include "sql/rewrite/ob_union_find.h" namespace oceanbase { @@ -123,6 +124,17 @@ struct SingleSetParam { K_(need_add_limit_constraint)); }; + struct SemiInfoSplitHelper { + SemiInfoSplitHelper() : can_split_(false) + {} + + bool can_split_; + ObSEArray, 4> connected_tables_; + + TO_STRING_KV(K_(can_split), + K_(connected_tables)); + }; + int transform_anyall_query(ObDMLStmt *stmt, ObIArray &unnest_stmts, bool &trans_happened); @@ -224,9 +236,9 @@ struct SingleSetParam { int make_null_test(ObDMLStmt *stmt, ObRawExpr *in_expr, ObRawExpr *&out_expr); int generate_semi_info(ObDMLStmt* stmt, - ObRawExpr* expr, TableItem *right_table, ObIArray &semi_conditions, + ObJoinType join_type, SemiInfo *&semi_info); int fill_semi_left_table_ids(ObDMLStmt *stmt, SemiInfo *info); @@ -243,6 +255,11 @@ struct SingleSetParam { int check_hint_allowed_unnest(const ObDMLStmt &stmt, const ObSelectStmt &subquery, bool &allowed); + int check_can_split(ObSelectStmt *subquery, + ObIArray &semi_conditions, + ObJoinType join_type, + SemiInfoSplitHelper &helper); + private: DISALLOW_COPY_AND_ASSIGN(ObWhereSubQueryPullup); }; diff --git a/src/sql/rewrite/ob_transformer_impl.cpp b/src/sql/rewrite/ob_transformer_impl.cpp index 1dfb1ff2b0..d130d6f2cc 100644 --- a/src/sql/rewrite/ob_transformer_impl.cpp +++ b/src/sql/rewrite/ob_transformer_impl.cpp @@ -47,6 +47,7 @@ #include "sql/rewrite/ob_transform_count_to_exists.h" #include "sql/rewrite/ob_transform_expr_pullup.h" #include "sql/rewrite/ob_transform_dblink.h" +#include "sql/rewrite/ob_transform_conditional_aggr_coalesce.h" #include "sql/rewrite/ob_transform_mv_rewrite.h" #include "sql/rewrite/ob_transform_decorrelate.h" #include "common/ob_smart_call.h" @@ -116,12 +117,80 @@ int ObTransformerImpl::get_stmt_trans_info(ObDMLStmt *stmt, bool is_root) return ret; } +int ObTransformerImpl::get_random_order_array(uint64_t need_types, + ObQueryCtx *query_ctx, + ObArray &need_types_array) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(query_ctx)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid argument", K(query_ctx)); + } else { + common::ObArray index_array; + for (int64_t i = POST_PROCESS + 1; i < TRANSFORM_TYPE_COUNT_PLUS_ONE && OB_SUCC(ret); ++i) { + if ((need_types & (1 << i)) != 0) { + if (OB_FAIL(index_array.push_back(i))) { + LOG_WARN("failed to push back index", K(ret)); + } + } + } + if (OB_SUCC(ret)) { + for (int64_t idx = 1; idx < index_array.count(); ++idx) { + int64_t rand_pos = query_ctx->rand_gen_.get(0, idx); + if (rand_pos != idx) { + std::swap(index_array.at(rand_pos), index_array.at(idx)); + } + } + + LOG_TRACE("the random order will be ",K(index_array)); + for (int64_t i = 0; i < index_array.count() && OB_SUCC(ret); ++i) { + int need_types_pos = index_array.at(i); + uint64_t need_types_local = 1u << need_types_pos; + if (OB_FAIL(need_types_array.push_back(need_types_local))) { + LOG_WARN("failed to push back need types array", K(ret)); + } + } + } + } + return ret; +} + +int ObTransformerImpl::transform_random_order(ObDMLStmt *&stmt, ObQueryCtx *query_ctx, uint64_t need_types, int iter_count) +{ + int ret = OB_SUCCESS; + if (OB_ISNULL(query_ctx)) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("invalid argument", K(query_ctx)); + } else { + ObArray need_types_array; + if (OB_FAIL(get_random_order_array(need_types, query_ctx, need_types_array))) { + LOG_WARN("failed to get random order array", K(ret)); + } else { + bool any_trans_happened = true; + for (int64_t i = 0; i < iter_count && OB_SUCC(ret) && any_trans_happened; ++i) { + any_trans_happened = false; + for (int64_t j = 0; j < need_types_array.count() && OB_SUCC(ret); ++j) { + bool trans_happened = false; + if (OB_FAIL(transform_rule_set(stmt, need_types_array.at(j), 1, trans_happened))) { + LOG_WARN("failed to transform one rule set", K(ret)); + } else { + any_trans_happened = any_trans_happened || trans_happened; + } + } + } + } + } + return ret; +} + int ObTransformerImpl::do_transform(ObDMLStmt *&stmt) { int ret = OB_SUCCESS; uint64_t need_types = ObTransformRule::ALL_TRANSFORM_RULES; bool transformation_enabled = true; - const ObQueryCtx *query_ctx = NULL; + ObQueryCtx *query_ctx = NULL; + int iter_count = max_iteration_count_; + bool is_outline = false; if (OB_ISNULL(stmt) || OB_ISNULL(query_ctx = stmt->get_query_ctx()) || OB_ISNULL(ctx_) || OB_ISNULL(ctx_->session_info_)) { ret = OB_ERR_UNEXPECTED; @@ -135,9 +204,23 @@ int ObTransformerImpl::do_transform(ObDMLStmt *&stmt) LOG_WARN("failed choose rewrite rules for stmt", K(ret)); } else if (need_types == 0) { // do nothing - } else if (OB_FAIL(transform_rule_set(stmt, need_types, max_iteration_count_))) { - LOG_WARN("failed to transform one rule set", K(ret)); - } else { /*do nothing*/ } + } else { + if (!query_ctx->get_injected_random_status()) { + is_outline = query_ctx->get_query_hint().has_outline_data(); + if (is_outline) { + iter_count = max(iter_count, query_ctx->get_query_hint().trans_list_.count() + 1); + } + bool trans_happened = false; + if (OB_FAIL(transform_rule_set(stmt, need_types, iter_count, trans_happened))) { + LOG_WARN("failed to transform one rule set", K(ret)); + } + } else { + //unlikely need_types 特殊处理, 把所有位置取出来重新排序。 + if (OB_FAIL(transform_random_order(stmt, query_ctx, need_types, iter_count))) { + LOG_WARN("failed to transform random order", K(ret)); + } + } + } return ret; } @@ -275,12 +358,14 @@ int ObTransformerImpl::do_transform_dblink_read(ObDMLStmt *&stmt) int ObTransformerImpl::transform_heuristic_rule(ObDMLStmt *&stmt) { int ret = OB_SUCCESS; + bool trans_happened = false; if (OB_ISNULL(stmt)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("stmt is NULL", K(ret)); } else if (OB_FAIL(transform_rule_set(stmt, ObTransformRule::ALL_HEURISTICS_RULES, - max_iteration_count_))) { + max_iteration_count_, + trans_happened))) { LOG_WARN("failed to transform one rule set", K(ret)); } else { /*do nothing*/ } return ret; @@ -288,9 +373,11 @@ int ObTransformerImpl::transform_heuristic_rule(ObDMLStmt *&stmt) int ObTransformerImpl::transform_rule_set(ObDMLStmt *&stmt, uint64_t needed_types, - int64_t iteration_count) + int64_t iteration_count, + bool& trans_happened) { int ret = OB_SUCCESS; + trans_happened = false; if (0 != (needed_types & needed_transform_types_)) { bool need_next_iteration = true; int64_t i = 0; @@ -313,6 +400,7 @@ int ObTransformerImpl::transform_rule_set(ObDMLStmt *&stmt, LOG_WARN("failed to formalize stmt", K(ret)); } else { need_next_iteration = true; + trans_happened = true; } LOG_TRACE("succeed to transform one iteration", K(i), K(need_next_iteration), K(ret)); OPT_TRACE("-- end ", i, " iteration"); @@ -366,6 +454,7 @@ int ObTransformerImpl::transform_rule_set_in_one_iteration(ObDMLStmt *&stmt, APPLY_RULE_IF_NEEDED(WIN_MAGIC, ObTransformWinMagic); APPLY_RULE_IF_NEEDED(GROUPBY_PUSHDOWN, ObTransformGroupByPushdown); APPLY_RULE_IF_NEEDED(GROUPBY_PULLUP, ObTransformGroupByPullup); + APPLY_RULE_IF_NEEDED(CONDITIONAL_AGGR_COALESCE, ObTransformConditionalAggrCoalesce); APPLY_RULE_IF_NEEDED(FASTMINMAX, ObTransformMinMax); APPLY_RULE_IF_NEEDED(PREDICATE_MOVE_AROUND, ObTransformPredicateMoveAround); APPLY_RULE_IF_NEEDED(OR_EXPANSION, ObTransformOrExpansion); diff --git a/src/sql/rewrite/ob_transformer_impl.h b/src/sql/rewrite/ob_transformer_impl.h index 2bed210ff7..1b011d36fb 100644 --- a/src/sql/rewrite/ob_transformer_impl.h +++ b/src/sql/rewrite/ob_transformer_impl.h @@ -61,7 +61,8 @@ public: int transform_heuristic_rule(ObDMLStmt *&stmt); int transform_rule_set(ObDMLStmt *&stmt, uint64_t needed_types, - int64_t iteration_count); + int64_t iteration_count, + bool &trans_happened); int transform_rule_set_in_one_iteration(ObDMLStmt *&stmt, uint64_t needed_types, bool &trans_happened); @@ -176,6 +177,15 @@ private: const char *rule_name, bool &trans_happened); + int transform_random_order(ObDMLStmt *&stmt, + ObQueryCtx *query_ctx, + uint64_t need_types, + int iter_count); + + static int get_random_order_array(uint64_t need_types, + ObQueryCtx *query_ctx, + ObArray &need_types_array); + private: ObTransformerCtx *ctx_; uint64_t needed_transform_types_; diff --git a/src/sql/session/ob_sql_session_info.cpp b/src/sql/session/ob_sql_session_info.cpp index bc0b915d6a..d20572a003 100644 --- a/src/sql/session/ob_sql_session_info.cpp +++ b/src/sql/session/ob_sql_session_info.cpp @@ -508,6 +508,17 @@ bool ObSQLSessionInfo::is_in_range_optimization_enabled() const return bret; } +int64_t ObSQLSessionInfo::get_inlist_rewrite_threshold() const +{ + int64_t thredhold = INT64_MAX; + int64_t tenant_id = get_effective_tenant_id(); + omt::ObTenantConfigGuard tenant_config(TENANT_CONF(tenant_id)); + if (tenant_config.is_valid()) { + thredhold = tenant_config->_inlist_rewrite_threshold; + } + return thredhold; +} + int ObSQLSessionInfo::is_better_inlist_enabled(bool &enabled) const { int ret = OB_SUCCESS; diff --git a/src/sql/session/ob_sql_session_info.h b/src/sql/session/ob_sql_session_info.h index f8e7399f30..2401a0904c 100644 --- a/src/sql/session/ob_sql_session_info.h +++ b/src/sql/session/ob_sql_session_info.h @@ -1226,6 +1226,7 @@ public: int is_temp_table_transformation_enabled(bool &transformation_enabled) const; int is_groupby_placement_transformation_enabled(bool &transformation_enabled) const; bool is_in_range_optimization_enabled() const; + int64_t get_inlist_rewrite_threshold() const; int is_better_inlist_enabled(bool &enabled) const; bool is_index_skip_scan_enabled() const; bool is_qualify_filter_enabled() const; diff --git a/tools/deploy/mysql_test/r/mysql/dist_nest_loop_simple.result b/tools/deploy/mysql_test/r/mysql/dist_nest_loop_simple.result index 4d0e312db0..3efeb740e7 100644 --- a/tools/deploy/mysql_test/r/mysql/dist_nest_loop_simple.result +++ b/tools/deploy/mysql_test/r/mysql/dist_nest_loop_simple.result @@ -43,7 +43,7 @@ Outputs & filters: 2 - output([t1.c2], [t2.c2], [t1.c3], [t2.c3], [t1.c1], [t2.c1]), filter(nil), rowset=16 conds([t1.c2 = t2.c2], [t1.c3 = t2.c3]), nl_params_(nil), use_batch=false 3 - output([t1.c2], [t1.c3], [t1.c1]), filter(nil), rowset=16 - affinitize, force partition granule, asc + affinitize, force partition granule 4 - output([t1.c2], [t1.c3], [t1.c1]), filter(nil), rowset=16 access([t1.c2], [t1.c3], [t1.c1]), partitions(p[0-1]) is_index_back=false, is_global_index=false, @@ -134,7 +134,7 @@ Outputs & filters: is_index_back=false, is_global_index=false, range_key([t2.c1], [t2.c2], [t2.c3]), range(MIN,MIN,MIN ; MAX,MAX,MAX)always true 7 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 - affinitize, force partition granule, asc + affinitize, force partition granule 8 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 access([t1.c2], [t1.c3], [t1.c1], [t1.c4]), partitions(p[0-1]) is_index_back=false, is_global_index=false, @@ -163,7 +163,7 @@ Outputs & filters: 2 - output([t1.c1], [t1.c2], [t1.c3], [t1.c4]), filter(nil), rowset=16 dop=1 3 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 4 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 access([t1.c2], [t1.c3], [t1.c1], [t1.c4]), partitions(p[0-1]) is_index_back=false, is_global_index=false, @@ -210,7 +210,7 @@ Outputs & filters: 6 - output([t1.c1], [t1.c2], [t1.c3], [t1.c4]), filter(nil), rowset=16 dop=1 7 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 access([t1.c2], [t1.c3], [t1.c1], [t1.c4]), partitions(p[0-1]) is_index_back=false, is_global_index=false, @@ -239,7 +239,7 @@ Outputs & filters: 2 - output([t1.c2], [t2.c2], [t1.c3], [t1.c1], [t1.c4], [t2.c1], [t2.c3]), filter(nil), rowset=16 conds([t1.c2 = t2.c2]), nl_params_(nil), use_batch=false 3 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 - affinitize, force partition granule, asc + affinitize, force partition granule 4 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 access([t1.c2], [t1.c3], [t1.c1], [t1.c4]), partitions(p[0-1]) is_index_back=false, is_global_index=false, @@ -280,7 +280,7 @@ Outputs & filters: 4 - output([PARTITION_ID], [t1.c1], [t1.c2], [t1.c3], [t1.c4]), filter(nil), rowset=16 (#keys=1, [t1.c1]), dop=1 5 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 6 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 access([t1.c2], [t1.c3], [t1.c1], [t1.c4]), partitions(p[0-1]) is_index_back=false, is_global_index=false, @@ -316,7 +316,7 @@ Outputs & filters: 2 - output([t1.c2], [t2.c2], [t1.c3], [t2.c3], [t1.c1], [t1.c4], [t2.c1]), filter(nil), rowset=16 conds([t1.c2 = t2.c2], [t1.c3 = t2.c3]), nl_params_(nil), use_batch=false 3 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 - affinitize, force partition granule, asc + affinitize, force partition granule 4 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 access([t1.c2], [t1.c3], [t1.c1], [t1.c4]), partitions(p[0-1]) is_index_back=false, is_global_index=false, @@ -357,7 +357,7 @@ Outputs & filters: 4 - output([PARTITION_ID], [t1.c1], [t1.c3], [t1.c2], [t1.c4]), filter(nil), rowset=16 (#keys=1, [t1.c1]), dop=1 5 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 6 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 access([t1.c2], [t1.c3], [t1.c1], [t1.c4]), partitions(p[0-1]) is_index_back=false, is_global_index=false, @@ -393,7 +393,7 @@ Outputs & filters: 2 - output([t1.c2], [t2.c2], [t1.c3], [t1.c1], [t1.c4], [t2.c1], [t2.c3]), filter(nil), rowset=16 conds([t1.c2 = t2.c2]), nl_params_(nil), use_batch=false 3 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 - affinitize, force partition granule, asc + affinitize, force partition granule 4 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 access([t1.c2], [t1.c3], [t1.c1], [t1.c4]), partitions(p[0-1]) is_index_back=false, is_global_index=false, @@ -442,7 +442,7 @@ Outputs & filters: 7 - output([t1.c1], [t1.c2], [t1.c3], [t1.c4]), filter(nil), rowset=16 (#keys=1, [t1.c1]), dop=1 8 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 9 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 access([t1.c2], [t1.c3], [t1.c1], [t1.c4]), partitions(p[0-1]) is_index_back=false, is_global_index=false, @@ -471,7 +471,7 @@ Outputs & filters: 2 - output([t1.c2], [t2.c2], [t1.c3], [t2.c3], [t1.c1], [t1.c4], [t2.c1]), filter(nil), rowset=16 conds([t1.c2 = t2.c2], [t1.c3 = t2.c3]), nl_params_(nil), use_batch=false 3 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 - affinitize, force partition granule, asc + affinitize, force partition granule 4 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 access([t1.c2], [t1.c3], [t1.c1], [t1.c4]), partitions(p[0-1]) is_index_back=false, is_global_index=false, @@ -916,7 +916,7 @@ Outputs & filters: 7 - output([t1.c1], [t1.c4], [t1.c2], [t1.c3]), filter(nil), rowset=16 (#keys=1, [t1.c4]), dop=1 8 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 9 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 access([t1.c2], [t1.c3], [t1.c1], [t1.c4]), partitions(p[0-1]) is_index_back=false, is_global_index=false, @@ -977,7 +977,7 @@ Outputs & filters: 4 - output([PARTITION_ID], [t1.c1], [t1.c4], [t1.c2], [t1.c3]), filter(nil), rowset=16 (#keys=1, [t1.c4]), dop=1 5 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 6 - output([t1.c2], [t1.c3], [t1.c1], [t1.c4]), filter(nil), rowset=16 access([t1.c2], [t1.c3], [t1.c1], [t1.c4]), partitions(p[0-1]) is_index_back=false, is_global_index=false, diff --git a/tools/deploy/mysql_test/test_suite/datatype/r/mysql/div.result b/tools/deploy/mysql_test/test_suite/datatype/r/mysql/div.result index 7e9ba9254d..88eabcee6e 100644 --- a/tools/deploy/mysql_test/test_suite/datatype/r/mysql/div.result +++ b/tools/deploy/mysql_test/test_suite/datatype/r/mysql/div.result @@ -198561,10 +198561,7 @@ Query Plan |5 | ├─PX PARTITION ITERATOR| |14 |5 | |6 | │ └─TABLE FULL SCAN |table1000_key_pk_parts_2|14 |5 | |7 | └─MATERIAL | |1 |1 | -|8 | └─SUBPLAN SCAN |VIEW2 |1 |1 | -|9 | └─SUBPLAN FILTER | |1 |1 | -|10| ├─EXPRESSION |VALUES_TABLE1 |1 |1 | -|11| └─EXPRESSION | |1 |1 | +|8 | └─VALUES TABLE ACCESS|VALUES_TABLE1 |1 |1 | =================================================================================== Outputs & filters: ------------------------------------- @@ -198576,23 +198573,17 @@ Outputs & filters: 3 - output([T_FUN_COUNT(*)]), filter(nil), rowset=16 group(nil), agg_func([T_FUN_COUNT(*)]) 4 - output(nil), filter(nil), rowset=16 - conds([(T_OP_OR, table1000_key_pk_parts_2.col_double <= VIEW2.column_0, table1000_key_pk_parts_2.col_double IS NULL, VIEW2.column_0 IS NULL)]), nl_params_(nil), - use_batch=false + conds([(T_OP_OR, table1000_key_pk_parts_2.col_double <= cast(VALUES_TABLE1.column_0, DOUBLE(-1, -1)), table1000_key_pk_parts_2.col_double IS NULL, + cast(VALUES_TABLE1.column_0, DOUBLE(-1, -1)) IS NULL)]), nl_params_(nil), use_batch=false 5 - output([table1000_key_pk_parts_2.col_double]), filter(nil), rowset=16 force partition granule 6 - output([table1000_key_pk_parts_2.col_double]), filter(nil), rowset=16 access([table1000_key_pk_parts_2.col_double]), partitions(p[0-1]) is_index_back=false, is_global_index=false, range_key([table1000_key_pk_parts_2.__pk_increment]), range(MIN ; MAX)always true - 7 - output([VIEW2.column_0]), filter(nil), rowset=16 - 8 - output([VIEW2.column_0]), filter(nil), rowset=16 - access([VIEW2.column_0]) - 9 - output([cast(VALUES_TABLE1.column_0, DOUBLE(-1, -1))]), filter(nil), rowset=16 - exec_params_(nil), onetime_exprs_([subquery(1)(:1)]), init_plan_idxs_(nil), use_batch=false - 10 - output([VALUES_TABLE1.column_0]), filter(nil) - values({:1}) - 11 - output([Floor(cast(-360448000, DECIMAL(20, 0)) / cast(1458241536, DECIMAL(20, 0)))]), filter(nil) - values({Floor(cast(-360448000, DECIMAL(20, 0)) / cast(1458241536, DECIMAL(20, 0)))}) + 7 - output([VALUES_TABLE1.column_0]), filter(nil), rowset=16 + 8 - output([VALUES_TABLE1.column_0]), filter(nil) + access([VALUES_TABLE1.column_0]) SELECT count(*) FROM table1000_key_pk_parts_2 WHERE col_double > ALL (VALUES ROW((SELECT Floor(-360448000 / 1458241536) FROM dual))); count(*) 12 diff --git a/tools/deploy/mysql_test/test_suite/executor/r/mysql/basic.result b/tools/deploy/mysql_test/test_suite/executor/r/mysql/basic.result index 9f2e768fb1..5fe19c057c 100644 --- a/tools/deploy/mysql_test/test_suite/executor/r/mysql/basic.result +++ b/tools/deploy/mysql_test/test_suite/executor/r/mysql/basic.result @@ -489,7 +489,7 @@ Outputs & filters: 6 - output([t_r4_01_20.b], [t_r4_01_20.a]), filter(nil), rowset=256 sort_keys([t_r4_01_20.b, ASC]) 7 - output([t_r4_01_20.a], [t_r4_01_20.b]), filter(nil), rowset=256 - force partition granule, asc + force partition granule 8 - output([t_r4_01_20.a], [t_r4_01_20.b]), filter(nil), rowset=256 access([t_r4_01_20.a], [t_r4_01_20.b]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -544,7 +544,7 @@ Outputs & filters: 6 - output([t_r5_09_28.b], [t_r5_09_28.a]), filter(nil), rowset=256 sort_keys([t_r5_09_28.b, DESC]) 7 - output([t_r5_09_28.a], [t_r5_09_28.b]), filter(nil), rowset=256 - force partition granule, asc + force partition granule 8 - output([t_r5_09_28.a], [t_r5_09_28.b]), filter(nil), rowset=256 access([t_r5_09_28.a], [t_r5_09_28.b]), partitions(p[0-4]) is_index_back=false, is_global_index=false, @@ -1168,7 +1168,7 @@ Outputs & filters: 2 - output([t_r4_01_20.b], [t_r4_01_20.a]), filter(nil), rowset=16 sort_keys([t_r4_01_20.b, ASC]) 3 - output([t_r4_01_20.a], [t_r4_01_20.b]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 4 - output([t_r4_01_20.a], [t_r4_01_20.b]), filter(nil), rowset=16 access([t_r4_01_20.a], [t_r4_01_20.b]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -1211,7 +1211,7 @@ Outputs & filters: 2 - output([t_r4_09_28.b], [t_r4_09_28.a]), filter(nil), rowset=16 sort_keys([t_r4_09_28.b, DESC]) 3 - output([t_r4_09_28.a], [t_r4_09_28.b]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 4 - output([t_r4_09_28.a], [t_r4_09_28.b]), filter(nil), rowset=16 access([t_r4_09_28.a], [t_r4_09_28.b]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -1696,7 +1696,7 @@ Outputs & filters: 2 - output([t1.a], [t2.aa], [t1.b], [t1.c], [t2.a], [t2.b], [t2.bb], [t2.c], [t2.d], [t2.e]), filter(nil), rowset=16 equal_conds([t1.a = t2.aa]), other_conds(nil) 3 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - affinitize, force partition granule, asc + affinitize, force partition granule 4 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -1745,7 +1745,7 @@ Outputs & filters: 4 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 (#keys=1, [t1.a]), dop=1 5 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 6 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -1788,7 +1788,7 @@ Outputs & filters: 2 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 dop=1 3 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 4 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -1834,7 +1834,7 @@ Outputs & filters: 2 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 dop=1 3 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 4 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -1876,7 +1876,7 @@ Outputs & filters: 2 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 dop=1 3 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 4 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -1918,7 +1918,7 @@ Outputs & filters: 2 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 dop=1 3 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 4 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -2706,7 +2706,7 @@ Outputs & filters: 1 - output([INTERNAL_FUNCTION(t1.a, t1.b, t1.c, t2.a, t2.b, t2.c)]), filter(nil), rowset=16 dop=1 2 - output([t1.a], [t2.a], [t1.b], [t1.c], [t2.b], [t2.c]), filter(nil), rowset=16 - partition wise, force partition granule, asc + partition wise, force partition granule 3 - output([t1.a], [t2.a], [t1.b], [t1.c], [t2.b], [t2.c]), filter(nil), rowset=16 equal_conds([t1.a = t2.a]), other_conds(nil) merge_directions([ASC]) @@ -2760,7 +2760,7 @@ Outputs & filters: 6 - output([t1.a], [INTERNAL_FUNCTION(t1.a, 110, 19)], [t1.b], [t1.c]), filter(nil), rowset=16 (#keys=1, [INTERNAL_FUNCTION(t1.a, 110, 19)]), dop=1 7 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -2802,7 +2802,7 @@ Outputs & filters: 6 - output([t1.a], [INTERNAL_FUNCTION(t1.a, 110, 17)], [t1.b], [t1.c]), filter(nil), rowset=16 (#keys=1, [INTERNAL_FUNCTION(t1.a, 110, 17)]), dop=1 7 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -2835,7 +2835,7 @@ Outputs & filters: 2 - output([t2.a], [t2.b], [t2.c]), filter(nil), rowset=16 dop=1 3 - output([t2.a], [t2.c], [t2.b]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 4 - output([t2.a], [t2.c], [t2.b]), filter(nil), rowset=16 access([t2.a], [t2.c], [t2.b]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -2844,7 +2844,7 @@ Outputs & filters: 6 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 dop=1 7 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -2887,7 +2887,7 @@ Outputs & filters: 6 - output([t2.a], [INTERNAL_FUNCTION(t2.a, 110, 19)], [t2.b], [t2.c]), filter(nil), rowset=16 (#keys=1, [INTERNAL_FUNCTION(t2.a, 110, 19)]), dop=1 7 - output([t2.a], [t2.b], [t2.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t2.a], [t2.b], [t2.c]), filter(nil), rowset=16 access([t2.a], [t2.b], [t2.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -3013,7 +3013,7 @@ Outputs & filters: 6 - output([t2.a], [t2.b], [t2.c]), filter(nil), rowset=16 dop=1 7 - output([t2.a], [t2.c], [t2.b]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t2.a], [t2.c], [t2.b]), filter(nil), rowset=16 access([t2.a], [t2.c], [t2.b]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -3060,7 +3060,7 @@ Outputs & filters: 6 - output([t2.a], [INTERNAL_FUNCTION(t2.a, 110, 17)], [t2.b], [t2.c]), filter(nil), rowset=16 (#keys=1, [INTERNAL_FUNCTION(t2.a, 110, 17)]), dop=1 7 - output([t2.a], [t2.b], [t2.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t2.a], [t2.b], [t2.c]), filter(nil), rowset=16 access([t2.a], [t2.b], [t2.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -3186,7 +3186,7 @@ Outputs & filters: 6 - output([t2.c], [cast(t2.c, DATETIME(19, 0))], [t2.a], [t2.b]), filter(nil), rowset=16 (#keys=1, [cast(t2.c, DATETIME(19, 0))]), dop=1 7 - output([t2.a], [t2.c], [t2.b]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t2.a], [t2.c], [t2.b]), filter(nil), rowset=16 access([t2.a], [t2.c], [t2.b]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -3489,7 +3489,7 @@ Outputs & filters: 2 - output([t1.a], [t2.aa], [t1.b], [t1.c], [t2.a], [t2.b], [t2.bb], [t2.c], [t2.d], [t2.e]), filter(nil), rowset=16 equal_conds([t1.a = t2.aa]), other_conds(nil) 3 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - affinitize, force partition granule, asc + affinitize, force partition granule 4 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -3538,7 +3538,7 @@ Outputs & filters: 4 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 (#keys=1, [t1.a]), dop=1 5 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 6 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -3581,7 +3581,7 @@ Outputs & filters: 2 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 dop=1 3 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 4 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -3627,7 +3627,7 @@ Outputs & filters: 2 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 dop=1 3 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 4 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -3669,7 +3669,7 @@ Outputs & filters: 2 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 dop=1 3 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 4 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -3711,7 +3711,7 @@ Outputs & filters: 2 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 dop=1 3 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 4 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -4764,7 +4764,7 @@ Outputs & filters: 1 - output([INTERNAL_FUNCTION(t1.a, t1.b, t1.c, t2.a, t2.b, t2.c)]), filter(nil), rowset=16 dop=1 2 - output([t1.a], [t2.a], [t1.b], [t1.c], [t2.b], [t2.c]), filter(nil), rowset=16 - partition wise, force partition granule, asc + partition wise, force partition granule 3 - output([t1.a], [t2.a], [t1.b], [t1.c], [t2.b], [t2.c]), filter(nil), rowset=16 equal_conds([t1.a = t2.a]), other_conds(nil) merge_directions([ASC]) @@ -4818,7 +4818,7 @@ Outputs & filters: 6 - output([t1.a], [INTERNAL_FUNCTION(t1.a, 110, 19)], [t1.b], [t1.c]), filter(nil), rowset=16 (#keys=1, [INTERNAL_FUNCTION(t1.a, 110, 19)]), dop=1 7 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -4860,7 +4860,7 @@ Outputs & filters: 6 - output([t1.a], [INTERNAL_FUNCTION(t1.a, 110, 17)], [t1.b], [t1.c]), filter(nil), rowset=16 (#keys=1, [INTERNAL_FUNCTION(t1.a, 110, 17)]), dop=1 7 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -4893,7 +4893,7 @@ Outputs & filters: 2 - output([t2.a], [t2.b], [t2.c]), filter(nil), rowset=16 dop=1 3 - output([t2.a], [t2.c], [t2.b]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 4 - output([t2.a], [t2.c], [t2.b]), filter(nil), rowset=16 access([t2.a], [t2.c], [t2.b]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -4902,7 +4902,7 @@ Outputs & filters: 6 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 dop=1 7 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t1.a], [t1.b], [t1.c]), filter(nil), rowset=16 access([t1.a], [t1.b], [t1.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -4945,7 +4945,7 @@ Outputs & filters: 6 - output([t2.a], [INTERNAL_FUNCTION(t2.a, 110, 19)], [t2.b], [t2.c]), filter(nil), rowset=16 (#keys=1, [INTERNAL_FUNCTION(t2.a, 110, 19)]), dop=1 7 - output([t2.a], [t2.b], [t2.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t2.a], [t2.b], [t2.c]), filter(nil), rowset=16 access([t2.a], [t2.b], [t2.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -5071,7 +5071,7 @@ Outputs & filters: 6 - output([t2.a], [t2.b], [t2.c]), filter(nil), rowset=16 dop=1 7 - output([t2.a], [t2.c], [t2.b]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t2.a], [t2.c], [t2.b]), filter(nil), rowset=16 access([t2.a], [t2.c], [t2.b]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -5118,7 +5118,7 @@ Outputs & filters: 6 - output([t2.a], [INTERNAL_FUNCTION(t2.a, 110, 17)], [t2.b], [t2.c]), filter(nil), rowset=16 (#keys=1, [INTERNAL_FUNCTION(t2.a, 110, 17)]), dop=1 7 - output([t2.a], [t2.b], [t2.c]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t2.a], [t2.b], [t2.c]), filter(nil), rowset=16 access([t2.a], [t2.b], [t2.c]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -5244,7 +5244,7 @@ Outputs & filters: 6 - output([t2.c], [cast(t2.c, DATETIME(19, 0))], [t2.a], [t2.b]), filter(nil), rowset=16 (#keys=1, [cast(t2.c, DATETIME(19, 0))]), dop=1 7 - output([t2.a], [t2.c], [t2.b]), filter(nil), rowset=16 - force partition granule, asc + force partition granule 8 - output([t2.a], [t2.c], [t2.b]), filter(nil), rowset=16 access([t2.a], [t2.c], [t2.b]), partitions(p[0-3]) is_index_back=false, is_global_index=false, diff --git a/tools/deploy/mysql_test/test_suite/executor/r/mysql/full_join.result b/tools/deploy/mysql_test/test_suite/executor/r/mysql/full_join.result index c89b4e1ef3..5908dd52dd 100644 --- a/tools/deploy/mysql_test/test_suite/executor/r/mysql/full_join.result +++ b/tools/deploy/mysql_test/test_suite/executor/r/mysql/full_join.result @@ -1206,13 +1206,13 @@ Outputs & filters: 5 - output([t1.a]), filter(nil), rowset=256 (#keys=1, [t1.a]), dop=1 6 - output([t1.a]), filter(nil), rowset=256 - force partition granule, asc + force partition granule 7 - output([t1.a]), filter(nil), rowset=256 access([t1.a]), partitions(p[0-3]) is_index_back=false, is_global_index=false, range_key([t1.a], [t1.b]), range(MIN,MIN ; MAX,MAX)always true 8 - output([t2.a]), filter(nil), rowset=256 - affinitize, force partition granule, asc + affinitize, force partition granule 9 - output([t2.a]), filter(nil), rowset=256 access([t2.a]), partitions(p[0-4]) is_index_back=false, is_global_index=false, @@ -1280,13 +1280,13 @@ Outputs & filters: 5 - output([t1.a]), filter(nil), rowset=256 (#keys=1, [t1.a]), dop=1 6 - output([t1.a]), filter(nil), rowset=256 - force partition granule, asc + force partition granule 7 - output([t1.a]), filter(nil), rowset=256 access([t1.a]), partitions(p[0-4]) is_index_back=false, is_global_index=false, range_key([t1.a]), range(MIN ; MAX)always true 8 - output([t2.a]), filter(nil), rowset=256 - affinitize, force partition granule, asc + affinitize, force partition granule 9 - output([t2.a]), filter(nil), rowset=256 access([t2.a]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -1348,7 +1348,7 @@ Outputs & filters: is_index_back=false, is_global_index=false, range_key([t1.a]), range(MIN ; MAX)always true 8 - output([t2.a]), filter(nil), rowset=256 - affinitize, force partition granule, asc + affinitize, force partition granule 9 - output([t2.a]), filter(nil), rowset=256 access([t2.a]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -1422,7 +1422,7 @@ Outputs & filters: is_index_back=false, is_global_index=false, range_key([t1.a]), range(MIN ; MAX)always true 8 - output([t2.a]), filter(nil), rowset=256 - affinitize, force partition granule, asc + affinitize, force partition granule 9 - output([t2.a]), filter(nil), rowset=256 access([t2.a]), partitions(p[0-4]) is_index_back=false, is_global_index=false, @@ -1478,7 +1478,7 @@ Outputs & filters: 5 - output([t1.a]), filter(nil), rowset=256 (#keys=1, [t1.a]), dop=1 6 - output([t1.a]), filter(nil), rowset=256 - force partition granule, asc + force partition granule 7 - output([t1.a]), filter(nil), rowset=256 access([t1.a]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -1552,7 +1552,7 @@ Outputs & filters: 5 - output([t1.a]), filter(nil), rowset=256 (#keys=1, [t1.a]), dop=1 6 - output([t1.a]), filter(nil), rowset=256 - force partition granule, asc + force partition granule 7 - output([t1.a]), filter(nil), rowset=256 access([t1.a]), partitions(p[0-4]) is_index_back=false, is_global_index=false, @@ -1750,13 +1750,13 @@ Outputs & filters: 5 - output([t1.a]), filter(nil), rowset=256 (#keys=1, [t1.a]), dop=1 6 - output([t1.a]), filter(nil), rowset=256 - force partition granule, asc + force partition granule 7 - output([t1.a]), filter(nil), rowset=256 access([t1.a]), partitions(p[0-3]) is_index_back=false, is_global_index=false, range_key([t1.a], [t1.b]), range(MIN,MIN ; MAX,MAX)always true 8 - output([t2.a]), filter(nil), rowset=256 - affinitize, force partition granule, asc + affinitize, force partition granule 9 - output([t2.a]), filter(nil), rowset=256 access([t2.a]), partitions(p[0-4]) is_index_back=false, is_global_index=false, @@ -1824,13 +1824,13 @@ Outputs & filters: 5 - output([t1.a]), filter(nil), rowset=256 (#keys=1, [t1.a]), dop=1 6 - output([t1.a]), filter(nil), rowset=256 - force partition granule, asc + force partition granule 7 - output([t1.a]), filter(nil), rowset=256 access([t1.a]), partitions(p[0-4]) is_index_back=false, is_global_index=false, range_key([t1.a]), range(MIN ; MAX)always true 8 - output([t2.a]), filter(nil), rowset=256 - affinitize, force partition granule, asc + affinitize, force partition granule 9 - output([t2.a]), filter(nil), rowset=256 access([t2.a]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -1892,7 +1892,7 @@ Outputs & filters: is_index_back=false, is_global_index=false, range_key([t1.a]), range(MIN ; MAX)always true 8 - output([t2.a]), filter(nil), rowset=256 - affinitize, force partition granule, asc + affinitize, force partition granule 9 - output([t2.a]), filter(nil), rowset=256 access([t2.a]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -1966,7 +1966,7 @@ Outputs & filters: is_index_back=false, is_global_index=false, range_key([t1.a]), range(MIN ; MAX)always true 8 - output([t2.a]), filter(nil), rowset=256 - affinitize, force partition granule, asc + affinitize, force partition granule 9 - output([t2.a]), filter(nil), rowset=256 access([t2.a]), partitions(p[0-4]) is_index_back=false, is_global_index=false, @@ -2022,7 +2022,7 @@ Outputs & filters: 5 - output([t1.a]), filter(nil), rowset=256 (#keys=1, [t1.a]), dop=1 6 - output([t1.a]), filter(nil), rowset=256 - force partition granule, asc + force partition granule 7 - output([t1.a]), filter(nil), rowset=256 access([t1.a]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -2096,7 +2096,7 @@ Outputs & filters: 5 - output([t1.a]), filter(nil), rowset=256 (#keys=1, [t1.a]), dop=1 6 - output([t1.a]), filter(nil), rowset=256 - force partition granule, asc + force partition granule 7 - output([t1.a]), filter(nil), rowset=256 access([t1.a]), partitions(p[0-4]) is_index_back=false, is_global_index=false, @@ -2294,13 +2294,13 @@ Outputs & filters: 5 - output([t1.a]), filter(nil), rowset=256 (#keys=1, [t1.a]), dop=1 6 - output([t1.a]), filter(nil), rowset=256 - force partition granule, asc + force partition granule 7 - output([t1.a]), filter(nil), rowset=256 access([t1.a]), partitions(p[0-3]) is_index_back=false, is_global_index=false, range_key([t1.a], [t1.b]), range(MIN,MIN ; MAX,MAX)always true 8 - output([t2.a]), filter(nil), rowset=256 - affinitize, force partition granule, asc + affinitize, force partition granule 9 - output([t2.a]), filter(nil), rowset=256 access([t2.a]), partitions(p[0-4]) is_index_back=false, is_global_index=false, @@ -2368,13 +2368,13 @@ Outputs & filters: 5 - output([t1.a]), filter(nil), rowset=256 (#keys=1, [t1.a]), dop=1 6 - output([t1.a]), filter(nil), rowset=256 - force partition granule, asc + force partition granule 7 - output([t1.a]), filter(nil), rowset=256 access([t1.a]), partitions(p[0-4]) is_index_back=false, is_global_index=false, range_key([t1.a]), range(MIN ; MAX)always true 8 - output([t2.a]), filter(nil), rowset=256 - affinitize, force partition granule, asc + affinitize, force partition granule 9 - output([t2.a]), filter(nil), rowset=256 access([t2.a]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -2436,7 +2436,7 @@ Outputs & filters: is_index_back=false, is_global_index=false, range_key([t1.a]), range(MIN ; MAX)always true 8 - output([t2.a]), filter(nil), rowset=256 - affinitize, force partition granule, asc + affinitize, force partition granule 9 - output([t2.a]), filter(nil), rowset=256 access([t2.a]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -2510,7 +2510,7 @@ Outputs & filters: is_index_back=false, is_global_index=false, range_key([t1.a]), range(MIN ; MAX)always true 8 - output([t2.a]), filter(nil), rowset=256 - affinitize, force partition granule, asc + affinitize, force partition granule 9 - output([t2.a]), filter(nil), rowset=256 access([t2.a]), partitions(p[0-4]) is_index_back=false, is_global_index=false, @@ -2566,7 +2566,7 @@ Outputs & filters: 5 - output([t1.a]), filter(nil), rowset=256 (#keys=1, [t1.a]), dop=1 6 - output([t1.a]), filter(nil), rowset=256 - force partition granule, asc + force partition granule 7 - output([t1.a]), filter(nil), rowset=256 access([t1.a]), partitions(p[0-3]) is_index_back=false, is_global_index=false, @@ -2640,7 +2640,7 @@ Outputs & filters: 5 - output([t1.a]), filter(nil), rowset=256 (#keys=1, [t1.a]), dop=1 6 - output([t1.a]), filter(nil), rowset=256 - force partition granule, asc + force partition granule 7 - output([t1.a]), filter(nil), rowset=256 access([t1.a]), partitions(p[0-4]) is_index_back=false, is_global_index=false, diff --git a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result index aa4072c8d2..02418a40d4 100644 --- a/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result +++ b/tools/deploy/mysql_test/test_suite/inner_table/r/mysql/all_virtual_sys_parameter_stat.result @@ -352,6 +352,7 @@ _ha_rpc_timeout _ha_tablet_info_batch_count _hidden_sys_tenant_memory _ignore_system_memory_over_limit_error +_inlist_rewrite_threshold _io_callback_thread_count _iut_enable _iut_max_entries diff --git a/tools/deploy/mysql_test/test_suite/px/r/mysql/tsc.result b/tools/deploy/mysql_test/test_suite/px/r/mysql/tsc.result index 564002e397..3c17e4614e 100644 --- a/tools/deploy/mysql_test/test_suite/px/r/mysql/tsc.result +++ b/tools/deploy/mysql_test/test_suite/px/r/mysql/tsc.result @@ -425,10 +425,10 @@ Query Plan ================================================================ |ID|OPERATOR |NAME |EST.ROWS|EST.TIME(us)| ---------------------------------------------------------------- -|0 |LIMIT | |3 |12 | -|1 |└─PX COORDINATOR MERGE SORT | |3 |12 | -|2 | └─EXCHANGE OUT DISTR |:EX10000|3 |10 | -|3 | └─TOP-N SORT | |3 |8 | +|0 |LIMIT | |3 |15 | +|1 |└─PX COORDINATOR MERGE SORT | |6 |15 | +|2 | └─EXCHANGE OUT DISTR |:EX10000|6 |12 | +|3 | └─TOP-N SORT | |6 |8 | |4 | └─PX BLOCK ITERATOR | |14 |7 | |5 | └─TABLE FULL SCAN |stu |14 |7 | ================================================================ diff --git a/tools/deploy/mysql_test/test_suite/static_engine/r/mysql/subplan_filter.result b/tools/deploy/mysql_test/test_suite/static_engine/r/mysql/subplan_filter.result index f0410b3883..f494696f0a 100644 --- a/tools/deploy/mysql_test/test_suite/static_engine/r/mysql/subplan_filter.result +++ b/tools/deploy/mysql_test/test_suite/static_engine/r/mysql/subplan_filter.result @@ -799,10 +799,10 @@ Query Plan |2 |│ └─EXCHANGE OUT DISTR |:EX10000|5 |6 | |3 |│ └─PX BLOCK ITERATOR | |5 |6 | |4 |│ └─TABLE FULL SCAN |t2 |5 |6 | -|5 |└─LIMIT | |1 |6 | -|6 | └─PX COORDINATOR MERGE SORT | |1 |6 | -|7 | └─EXCHANGE OUT DISTR |:EX20000|1 |6 | -|8 | └─TOP-N SORT | |1 |6 | +|5 |└─LIMIT | |1 |7 | +|6 | └─PX COORDINATOR MERGE SORT | |2 |7 | +|7 | └─EXCHANGE OUT DISTR |:EX20000|2 |6 | +|8 | └─TOP-N SORT | |2 |6 | |9 | └─PX BLOCK ITERATOR | |5 |6 | |10| └─TABLE FULL SCAN |t1 |5 |6 | ================================================================== diff --git a/tools/deploy/mysql_test/test_suite/static_engine/r/mysql/table_scan.result b/tools/deploy/mysql_test/test_suite/static_engine/r/mysql/table_scan.result index 305a3cc3f9..5441ff4701 100644 --- a/tools/deploy/mysql_test/test_suite/static_engine/r/mysql/table_scan.result +++ b/tools/deploy/mysql_test/test_suite/static_engine/r/mysql/table_scan.result @@ -296,7 +296,7 @@ Query Plan ================================================== |ID|OPERATOR |NAME |EST.ROWS|EST.TIME(us)| -------------------------------------------------- -|0 |TABLE FULL SCAN|t1(idx)|2 |11 | +|0 |TABLE FULL SCAN|t1(idx)|2 |23 | ================================================== Outputs & filters: ------------------------------------- diff --git a/tools/deploy/mysql_test/test_suite/subquery/r/mysql/optimizer_subquery_bug.result b/tools/deploy/mysql_test/test_suite/subquery/r/mysql/optimizer_subquery_bug.result index d2b7e4039d..c62fca359a 100644 --- a/tools/deploy/mysql_test/test_suite/subquery/r/mysql/optimizer_subquery_bug.result +++ b/tools/deploy/mysql_test/test_suite/subquery/r/mysql/optimizer_subquery_bug.result @@ -873,19 +873,19 @@ Query Plan ============================================================================== |ID|OPERATOR |NAME |EST.ROWS|EST.TIME(us)| ------------------------------------------------------------------------------ -|0 |SCALAR GROUP BY | |1 |15 | -|1 |└─MERGE ANTI JOIN | |9 |15 | +|0 |SCALAR GROUP BY | |1 |17 | +|1 |└─MERGE ANTI JOIN | |9 |17 | |2 | ├─TABLE FULL SCAN |table10_bigint |10 |3 | -|3 | └─SUBPLAN SCAN |VIEW3 |1 |12 | -|4 | └─MERGE ANTI JOIN | |1 |12 | +|3 | └─SUBPLAN SCAN |VIEW3 |1 |14 | +|4 | └─MERGE ANTI JOIN | |1 |14 | |5 | ├─TABLE FULL SCAN |table2_bigint |2 |3 | -|6 | └─SORT | |2 |9 | -|7 | └─SUBPLAN SCAN |VIEW2 |2 |9 | -|8 | └─NESTED-LOOP JOIN | |2 |9 | -|9 | ├─SUBPLAN SCAN |VIEW4 |1 |3 | -|10| │ └─SCALAR GROUP BY | |1 |3 | -|11| │ └─SUBPLAN SCAN |VIEW1 |1 |3 | -|12| │ └─TABLE FULL SCAN|table100_bigint|1 |3 | +|6 | └─SORT | |2 |12 | +|7 | └─SUBPLAN SCAN |VIEW2 |2 |12 | +|8 | └─NESTED-LOOP JOIN | |2 |12 | +|9 | ├─SUBPLAN SCAN |VIEW4 |1 |5 | +|10| │ └─SCALAR GROUP BY | |1 |5 | +|11| │ └─SUBPLAN SCAN |VIEW1 |1 |5 | +|12| │ └─TABLE FULL SCAN|table100_bigint|1 |5 | |13| └─TABLE FULL SCAN |table100_bigint|100 |4 | ============================================================================== Outputs & filters: @@ -939,20 +939,20 @@ Query Plan ================================================================================== |ID|OPERATOR |NAME |EST.ROWS|EST.TIME(us)| ---------------------------------------------------------------------------------- -|0 |SUBPLAN FILTER | |1 |17 | -|1 |├─SUBPLAN SCAN |VIEW5 |1 |15 | -|2 |│ └─SCALAR GROUP BY | |1 |15 | -|3 |│ └─MERGE ANTI JOIN | |9 |15 | +|0 |SUBPLAN FILTER | |1 |22 | +|1 |├─SUBPLAN SCAN |VIEW5 |1 |20 | +|2 |│ └─SCALAR GROUP BY | |1 |20 | +|3 |│ └─MERGE ANTI JOIN | |9 |19 | |4 |│ ├─TABLE FULL SCAN |table10_bigint |10 |3 | -|5 |│ └─SORT | |1 |12 | -|6 |│ └─SUBPLAN SCAN |VIEW3 |1 |12 | -|7 |│ └─HASH RIGHT ANTI JOIN NA | |1 |12 | -|8 |│ ├─SUBPLAN SCAN |VIEW2 |1 |9 | -|9 |│ │ └─NESTED-LOOP JOIN | |1 |9 | -|10|│ │ ├─SUBPLAN SCAN |VIEW4 |1 |3 | -|11|│ │ │ └─SCALAR GROUP BY | |1 |3 | -|12|│ │ │ └─SUBPLAN SCAN |VIEW1 |1 |3 | -|13|│ │ │ └─TABLE FULL SCAN|table100_bigint|1 |3 | +|5 |│ └─SORT | |1 |16 | +|6 |│ └─SUBPLAN SCAN |VIEW3 |1 |16 | +|7 |│ └─HASH RIGHT ANTI JOIN NA | |1 |16 | +|8 |│ ├─SUBPLAN SCAN |VIEW2 |1 |14 | +|9 |│ │ └─NESTED-LOOP JOIN | |1 |14 | +|10|│ │ ├─SUBPLAN SCAN |VIEW4 |1 |7 | +|11|│ │ │ └─SCALAR GROUP BY | |1 |7 | +|12|│ │ │ └─SUBPLAN SCAN |VIEW1 |1 |7 | +|13|│ │ │ └─TABLE FULL SCAN|table100_bigint|1 |7 | |14|│ │ └─TABLE FULL SCAN |table100_bigint|100 |4 | |15|│ └─TABLE FULL SCAN |table2_bigint |2 |3 | |16|└─TOP-N SORT | |1 |3 | diff --git a/tools/deploy/mysql_test/test_suite/subquery/r/mysql/subquery_sj_innodb.result b/tools/deploy/mysql_test/test_suite/subquery/r/mysql/subquery_sj_innodb.result index d97c41691f..8479c4c111 100644 --- a/tools/deploy/mysql_test/test_suite/subquery/r/mysql/subquery_sj_innodb.result +++ b/tools/deploy/mysql_test/test_suite/subquery/r/mysql/subquery_sj_innodb.result @@ -197,35 +197,48 @@ explain select 1 from t2 where c2 in (select 1 from t3, t2) and c1 in (select convert(c6,char(1)) from t2); Query Plan -============================================================== -|ID|OPERATOR |NAME |EST.ROWS|EST.TIME(us)| --------------------------------------------------------------- -|0 |HASH SEMI JOIN | |1 |5 | -|1 |├─NESTED-LOOP JOIN CARTESIAN | |1 |3 | -|2 |│ ├─TABLE FULL SCAN |t2 |1 |3 | -|3 |│ └─MATERIAL | |1 |3 | -|4 |│ └─SUBPLAN SCAN |VIEW1|1 |3 | -|5 |│ └─TABLE FULL SCAN |t3 |1 |3 | -|6 |└─TABLE FULL SCAN |t2 |1 |3 | -============================================================== +================================================================ +|ID|OPERATOR |NAME |EST.ROWS|EST.TIME(us)| +---------------------------------------------------------------- +|0 |HASH SEMI JOIN | |1 |5 | +|1 |├─NESTED-LOOP JOIN CARTESIAN | |1 |3 | +|2 |│ ├─NESTED-LOOP JOIN CARTESIAN | |1 |3 | +|3 |│ │ ├─TABLE FULL SCAN |t2 |1 |3 | +|4 |│ │ └─MATERIAL | |1 |3 | +|5 |│ │ └─SUBPLAN SCAN |VIEW1|1 |3 | +|6 |│ │ └─TABLE FULL SCAN |t3 |1 |3 | +|7 |│ └─MATERIAL | |1 |3 | +|8 |│ └─SUBPLAN SCAN |VIEW2|1 |3 | +|9 |│ └─TABLE FULL SCAN |t2 |1 |3 | +|10|└─TABLE FULL SCAN |t2 |1 |3 | +================================================================ Outputs & filters: ------------------------------------- 0 - output([1]), filter(nil), rowset=16 equal_conds([cast(t2.c1, VARCHAR(1048576)) = cast(cast(t2.c6, CHAR(1)), VARCHAR(1048576))]), other_conds(nil) 1 - output([t2.c1]), filter(nil), rowset=16 conds(nil), nl_params_(nil), use_batch=false - 2 - output([t2.c1]), filter([cast(1, DECIMAL(1, 0)) = cast(t2.c2, DECIMAL(-1, -1))]), rowset=16 + 2 - output([t2.c1]), filter(nil), rowset=16 + conds(nil), nl_params_(nil), use_batch=false + 3 - output([t2.c1]), filter([cast(t2.c2, DECIMAL(-1, -1)) = cast(1, DECIMAL(1, 0))]), rowset=16 access([t2.c2], [t2.c1]), partitions(p0) is_index_back=false, is_global_index=false, filter_before_indexback[false], range_key([t2.__pk_increment]), range(MIN ; MAX)always true - 3 - output(nil), filter(nil), rowset=16 4 - output(nil), filter(nil), rowset=16 + 5 - output(nil), filter(nil), rowset=16 access(nil) - 5 - output([1]), filter(nil), rowset=16 + 6 - output([1]), filter(nil), rowset=16 access(nil), partitions(p0) limit(1), offset(nil), is_index_back=false, is_global_index=false, range_key([t3.__pk_increment]), range(MIN ; MAX)always true - 6 - output([t2.c6]), filter(nil), rowset=16 + 7 - output(nil), filter(nil), rowset=16 + 8 - output(nil), filter(nil), rowset=16 + access(nil) + 9 - output([1]), filter(nil), rowset=16 + access(nil), partitions(p0) + limit(1), offset(nil), is_index_back=false, is_global_index=false, + range_key([t2.__pk_increment]), range(MIN ; MAX)always true + 10 - output([t2.c6]), filter(nil), rowset=16 access([t2.c6]), partitions(p0) is_index_back=false, is_global_index=false, range_key([t2.__pk_increment]), range(MIN ; MAX)always true