/** * 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/resolver/expr/ob_raw_expr.h" #include "sql/optimizer/ob_del_upd_log_plan.h" #include "sql/optimizer/ob_log_del_upd.h" #include "sql/optimizer/ob_log_plan.h" #include "sql/optimizer/ob_log_table_scan.h" #include "sql/optimizer/ob_log_exchange.h" #include "share/schema/ob_schema_getter_guard.h" #include "common/ob_smart_call.h" #include "sql/rewrite/ob_transform_utils.h" #include "sql/optimizer/ob_log_join.h" using namespace oceanbase; using namespace oceanbase::sql; using namespace oceanbase::share; using namespace oceanbase::share::schema; using namespace common; int IndexDMLInfo::deep_copy(ObIRawExprCopier &expr_copier, const IndexDMLInfo &other) { int ret = OB_SUCCESS; table_id_ = other.table_id_; loc_table_id_ = other.loc_table_id_; ref_table_id_ = other.ref_table_id_; index_name_ = other.index_name_; spk_cnt_ = other.spk_cnt_; rowkey_cnt_ = other.rowkey_cnt_; need_filter_null_ = other.need_filter_null_; is_primary_index_ = other.is_primary_index_; is_update_unique_key_ = other.is_update_unique_key_; is_update_part_key_ = other.is_update_part_key_; assignments_.reset(); if (OB_FAIL(expr_copier.copy(other.column_exprs_, column_exprs_))) { LOG_WARN("failed to assign column exprs", K(ret)); } else if (OB_FAIL(expr_copier.copy(other.column_convert_exprs_, column_convert_exprs_))) { LOG_WARN("failed to copy exprs", K(ret)); } else if (OB_FAIL(assignments_.prepare_allocate(other.assignments_.count()))) { LOG_WARN("failed to prepare allocate assignment array", K(ret)); } else if (OB_FAIL(expr_copier.copy(other.ck_cst_exprs_, ck_cst_exprs_))) { LOG_WARN("failed to copy exprs", K(ret)); } else if (OB_FAIL(part_ids_.assign(other.part_ids_))) { LOG_WARN("failed to assign part ids", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < other.assignments_.count(); ++i) { if (OB_FAIL(assignments_.at(i).deep_copy(expr_copier, other.assignments_.at(i)))) { LOG_WARN("failed to deep copy assignment", K(ret)); } } return ret; } int IndexDMLInfo::assign_basic(const IndexDMLInfo &other) { int ret = OB_SUCCESS; table_id_ = other.table_id_; loc_table_id_ = other.loc_table_id_; ref_table_id_ = other.ref_table_id_; index_name_ = other.index_name_; spk_cnt_ = other.spk_cnt_; rowkey_cnt_ = other.rowkey_cnt_; need_filter_null_ = other.need_filter_null_; is_primary_index_ = other.is_primary_index_; is_update_unique_key_ = other.is_update_unique_key_; is_update_part_key_ = other.is_update_part_key_; if (OB_FAIL(column_exprs_.assign(other.column_exprs_))) { LOG_WARN("failed to assign column exprs", K(ret)); } else if (OB_FAIL(column_convert_exprs_.assign(other.column_convert_exprs_))) { LOG_WARN("failed to assign column conver array", K(ret)); } else if (OB_FAIL(assignments_.assign(other.assignments_))) { LOG_WARN("failed to assign assignments array", K(ret)); } else if (OB_FAIL(ck_cst_exprs_.assign(other.ck_cst_exprs_))) { LOG_WARN("failed to assign check constraint exprs", K(ret)); } else if (OB_FAIL(part_ids_.assign(other.part_ids_))) { LOG_WARN("failed to assign part ids", K(ret)); } return ret; } int IndexDMLInfo::assign(const ObDmlTableInfo &info) { int ret = OB_SUCCESS; table_id_ = info.table_id_; loc_table_id_ = info.loc_table_id_; ref_table_id_ = info.ref_table_id_; index_name_ = info.table_name_; need_filter_null_ = info.need_filter_null_; if (OB_FAIL(column_exprs_.assign(info.column_exprs_))) { LOG_WARN("failed to assign column exprs", K(ret)); } else if (OB_FAIL(ck_cst_exprs_.assign(info.check_constraint_exprs_))) { LOG_WARN("failed to assign expr", K(ret)); } else if (OB_FAIL(part_ids_.assign(info.part_ids_))) { LOG_WARN("failed to assign part ids", K(ret)); } return ret; } int64_t IndexDMLInfo::to_explain_string(char *buf, int64_t buf_len, ExplainType type) const { int64_t pos = 0; int ret = OB_SUCCESS; BUF_PRINTF("{"); if (index_name_.empty()) { BUF_PRINTF("%lu: ", ref_table_id_); } else { pos += index_name_.to_string(buf + pos, buf_len - pos); } int64_t N = column_exprs_.count(); BUF_PRINTF(": ("); for (int64_t i = 0; OB_SUCC(ret) && i < N; i++) { if (NULL != column_exprs_.at(i)) { if (OB_SUCC(ret)) { if (OB_FAIL(column_exprs_.at(i)->get_name(buf, buf_len, pos, type))) { LOG_WARN("failed to get_name", K(ret)); } } if (i < N - 1) { BUF_PRINTF(", "); } } } BUF_PRINTF(")"); BUF_PRINTF("}"); return pos; } int IndexDMLInfo::init_assignment_info(const ObAssignments &assignments, ObRawExprFactory &expr_factory) { int ret = OB_SUCCESS; assignments_.reset(); for (int64_t i = 0; OB_SUCC(ret) && i < assignments.count(); ++i) { if (has_exist_in_array(column_exprs_, assignments.at(i).column_expr_)) { //将更新表达式的index添加到index info中,表示该index跟assignment有关 if (OB_FAIL(assignments_.push_back(assignments.at(i)))) { LOG_WARN("add assignment index to assign info failed", K(ret)); } } } return ret; } int IndexDMLInfo::get_rowkey_exprs(ObIArray &rowkey, bool need_spk) const { int ret = OB_SUCCESS; rowkey.reuse(); for (int64_t i = 0; OB_SUCC(ret) && i < get_real_uk_cnt(); ++i) { if (OB_FAIL(rowkey.push_back(column_exprs_.at(i)))) { LOG_WARN("failed to push back rowkey expr", K(ret)); } } if (OB_SUCC(ret) && need_spk) { for (int64_t i = get_real_uk_cnt(); OB_SUCC(ret) && i < rowkey_cnt_; ++i) { if (i >= column_exprs_.count() || !is_shadow_column(column_exprs_.at(i)->get_column_id())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("column_exprs_ does not contain shadow primary key", K(ret), K(i), K(column_exprs_)); } else if (OB_FAIL(rowkey.push_back(column_exprs_.at(i)))) { LOG_WARN("failed to push back rowkey expr", K(ret)); } } } return ret; } int IndexDMLInfo::get_rowkey_exprs(ObIArray &rowkey, bool need_spk) const { int ret = OB_SUCCESS; rowkey.reuse(); for (int64_t i = 0; OB_SUCC(ret) && i < get_real_uk_cnt(); ++i) { if (OB_FAIL(rowkey.push_back(column_exprs_.at(i)))) { LOG_WARN("failed to push back rowkey expr", K(ret)); } } if (OB_SUCC(ret) && need_spk) { for (int64_t i = get_real_uk_cnt(); OB_SUCC(ret) && i < rowkey_cnt_; ++i) { if (i >= column_exprs_.count() || !is_shadow_column(column_exprs_.at(i)->get_column_id())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("column_exprs_ does not contain shadow primary key", K(ret), K(i), K(column_exprs_)); } else if (OB_FAIL(rowkey.push_back(column_exprs_.at(i)))) { LOG_WARN("failed to push back rowkey expr", K(ret)); } } } return ret; } int IndexDMLInfo::init_column_convert_expr(const ObAssignments &assignments) { // 将 col1 = expr1 这种表达式中的 expr1 放到 column_convert_exprs // 中,用于后面的插入操作 int ret = OB_SUCCESS; int found = 0; // 在 assignment 中找到匹配表达式的个数 column_convert_exprs_.reset(); for (int i = 0; OB_SUCC(ret) && i < column_exprs_.count(); ++i) { ObRawExpr *insert_expr = column_exprs_.at(i); // find_replacement_in_assignment for (int j = 0; OB_SUCC(ret) && j < assignments.count(); ++j) { if (insert_expr == assignments.at(j).column_expr_) { insert_expr = const_cast(assignments.at(j).expr_); found++; break; } } if (OB_ISNULL(insert_expr)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("null unexpected", K(ret)); } else if (OB_FAIL(column_convert_exprs_.push_back(insert_expr))) { LOG_WARN("fail push back data", K(ret)); } } if (OB_SUCC(ret) && found != assignments.count()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("not all update asssigment found in insert target exprs", K(ret), K(found), K(assignments.count()), K(assignments)); } return ret; } ObLogDelUpd::ObLogDelUpd(ObDelUpdLogPlan &plan) : ObLogicalOperator(plan), my_dml_plan_(plan), view_check_exprs_(NULL), table_partition_info_(NULL), stmt_id_expr_(nullptr), lock_row_flag_expr_(NULL), ignore_(false), is_returning_(false), is_multi_part_dml_(false), is_pdml_(false), gi_charged_(false), is_index_maintenance_(false), need_barrier_(false), is_first_dml_op_(false), table_location_uncertain_(false), pdml_partition_id_expr_(NULL), pdml_is_returning_(false), err_log_define_(), need_alloc_part_id_expr_(false), has_instead_of_trigger_(false) { } int ObLogDelUpd::print_my_plan_annotation(char *buf, int64_t &buf_len, int64_t &pos, ExplainType type) { int ret = OB_SUCCESS; ret = BUF_PRINTF(", "); if (OB_FAIL(ret)) { // do nothing } else if (OB_FAIL(print_table_infos(ObString::make_string("table_columns"), buf, buf_len, pos, type))) { LOG_WARN("failed to print table infos", K(ret)); } else if (EXPLAIN_EXTENDED == type && need_barrier()) { ret = BUF_PRINTF(", "); ret = BUF_PRINTF("with_barrier"); } return ret; } uint64_t ObLogDelUpd::hash(uint64_t seed) const { seed = do_hash(is_multi_part_dml_, seed); seed = ObLogicalOperator::hash(seed); return seed; } int ObLogDelUpd::extract_err_log_info() { int ret = OB_SUCCESS; if (OB_ISNULL(get_stmt()) || OB_UNLIKELY(!get_stmt()->is_update_stmt() && !get_stmt()->is_delete_stmt() && !get_stmt()->is_insert_stmt())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); } else if (OB_FAIL(generate_errlog_info( static_cast(*get_stmt()), get_err_log_define()))) { LOG_WARN("failed to generate errlog info", K(ret)); } else { /*do nothing*/ } return ret; } int ObLogDelUpd::generate_errlog_info(const ObDelUpdStmt &stmt, ObErrLogDefine &errlog_define) { int ret = OB_SUCCESS; const ObIArray &error_log_exprs = stmt.get_error_log_info().error_log_exprs_; ObSEArray dml_columns; ObSEArray dml_values; if (stmt.is_insert_stmt()) { const ObInsertTableInfo& insert_info = static_cast(stmt).get_insert_table_info(); if (OB_FAIL(append(dml_columns, insert_info.column_exprs_))) { LOG_WARN("failed to append column expr", K(ret)); } else if (OB_FAIL(append(dml_values, insert_info.column_conv_exprs_))) { LOG_WARN("failed to append column convert expr", K(ret)); } } else if (stmt.is_update_stmt()) { const ObUpdateTableInfo* update_info = static_cast(stmt).get_update_table_info().at(0); if (OB_ISNULL(update_info)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get null update", K(ret)); } else { const ObAssignments &assigns = update_info->assignments_; for (int64_t i = 0; OB_SUCC(ret) && i < assigns.count(); ++i) { if (OB_FAIL(dml_columns.push_back(assigns.at(i).column_expr_))) { LOG_WARN("failed to push back assign column", K(ret)); } else if (OB_FAIL(dml_values.push_back(assigns.at(i).expr_))) { LOG_WARN("failed to push back assign value", K(ret)); } } } } for (int64_t i = 0; OB_SUCC(ret) && i < error_log_exprs.count(); ++i) { ObColumnRefRawExpr *col_expr = NULL; ObRawExpr *val_expr = NULL; int64_t idx = OB_INVALID_INDEX; if (OB_ISNULL(col_expr = error_log_exprs.at(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); } else if (!ObOptimizerUtil::find_item(dml_columns, col_expr, &idx)) { val_expr = col_expr; } else if (OB_UNLIKELY(idx < 0 || idx >= dml_values.count())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid index", K(ret), K(idx), K(dml_columns.count()), K(dml_values.count())); } else { val_expr = dml_values.at(idx); } if (OB_SUCC(ret)) { if (OB_FAIL(errlog_define.err_log_value_exprs_.push_back(val_expr))) { LOG_WARN("failed to push back errlog expr value", K(ret)); } else if (OB_FAIL(errlog_define.err_log_column_names_.push_back(col_expr->get_column_name()))) { LOG_WARN("failed to push back column name", K(ret)); } } } if (OB_SUCC(ret)) { errlog_define.is_err_log_ = stmt.is_error_logging(); errlog_define.err_log_database_name_ = stmt.get_err_log_database_name(); errlog_define.err_log_table_name_ = stmt.get_err_log_table_name(); errlog_define.reject_limit_ = stmt.get_err_log_reject_limit(); } return ret; } int ObLogDelUpd::allocate_granule_pre(AllocGIContext &ctx) { return pw_allocate_granule_pre(ctx); } int ObLogDelUpd::allocate_granule_post(AllocGIContext &ctx) { int ret = OB_SUCCESS; bool is_partition_wise_state = ctx.is_in_partition_wise_state(); if (OB_ISNULL(get_plan())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); } else if (OB_FAIL(pw_allocate_granule_post(ctx))){ // 分配完GI以后,会对ctx的状态进行清理 LOG_WARN("failed to allocate pw gi post", K(ret)); } else { if (is_partition_wise_state && ctx.is_op_set_pw(this)) { if (get_type() == log_op_def::LOG_UPDATE || // UPDATE: UPDATE, UPDATE RETURNING get_type() == log_op_def::LOG_DELETE || // DELETE: DELETE, DELETE RETURNING get_type() == log_op_def::LOG_INSERT || // INSERT: INSERT, INSERT UPDATE, REPLACE get_type() == log_op_def::LOG_MERGE) { // FOR UPDATE: FOR UPDATE ObSEArray tsc_ops; if (OB_FAIL(find_all_tsc(tsc_ops, this))) { LOG_WARN("failed to find all tsc", K(ret)); } else if (tsc_ops.count() < 1){ // do nothing set_gi_above(true); } else { // tsc op与当前dml算子都需要set gi above ARRAY_FOREACH(tsc_ops, idx) { ObLogTableScan *tsc_op = static_cast(tsc_ops.at(idx)); tsc_op->set_gi_above(true); } set_gi_above(true); } } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("op type doesn't support gi", K(ret), K(get_name())); } } else { LOG_TRACE("not allocate dml gi for px", K(ret), K(ctx), K(ctx.is_op_set_pw(this))); } } return ret; } int ObLogDelUpd::generate_pdml_partition_id_expr() { int ret = OB_SUCCESS; // pdml 分配 partition id expr // 1. 如果当前pdml op对应的表是非分区表,就不分配partition id expr // 2. 如果当前pdml op对应的表是分区表,就分配partition id expr uint64_t table_id = OB_INVALID_ID; ObOpPseudoColumnRawExpr *partition_id_expr = nullptr; ObLogicalOperator *producer = NULL; if (OB_UNLIKELY(!is_pdml()) || OB_UNLIKELY(index_dml_infos_.count() != 1) || OB_ISNULL(index_dml_infos_.at(0))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("index info array is empty", K(ret)); } else if (OB_FAIL(ObLogicalOperator::generate_pseudo_partition_id_expr(partition_id_expr))) { LOG_WARN("fail allocate part id expr", K(table_id), K(ret)); } else if (OB_FAIL(find_pdml_part_id_producer(*this, index_dml_infos_.at(0)->ref_table_id_, producer))) { LOG_WARN("find pdml partition id expr producer failed", K(ret)); } else if (NULL == producer) { ret = OB_ERR_UNEXPECTED; LOG_WARN("can not found pdml partition id expr producer", K(ret)); } else { pdml_partition_id_expr_ = partition_id_expr; if (producer->get_type() == log_op_def::LOG_EXCHANGE) { static_cast(producer)->set_partition_id_expr(partition_id_expr); } else if (producer->get_type() == log_op_def::LOG_TABLE_SCAN) { static_cast(producer)->set_tablet_id_expr(partition_id_expr); } else { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected type of pdml partition id producer", K(ret), K(producer)); } } return ret; } int ObLogDelUpd::find_pdml_part_id_producer(ObLogicalOperator &op, const uint64_t tid, ObLogicalOperator *&producer) { int ret = OB_SUCCESS; producer = NULL; OZ(check_stack_overflow()); if (OB_SUCC(ret)) { if (op.get_type() == log_op_def::LOG_EXCHANGE) { ObLogExchange &log_ex = static_cast(op); if (log_ex.is_producer() && log_ex.get_repartition_ref_table_id() == tid) { producer = &op; } } else if (op.get_type() == log_op_def::LOG_TABLE_SCAN) { // PDML partition id expr在table scan分配的逻辑 // pdml table scan分配partition id expr的producer // table scan中分配partition id expr的producer的逻辑比较特殊: // 分配partition id的时候,需要保证partition id expr对应的table id与table // scan的table id是相同的 // 对应insert的dml操作,例如:insert into t1 select from t1, // 产生的计划如下: // insert // subplan // GI // TSC // .... // // 这种情况下,如果给TSC算子分配partition idexpr,那么根据表达式分配的框架, // 其会被裁剪掉,因此目前insert与subplan之间会添加一个EX算子. // 后期会进行优化,如果insert与subplan是一个full partition wise // join,那么就在insert算子上分配一个GI算子,目前先使用在subplan上分配EX算子的方式实现 ObLogTableScan &tsc = static_cast(op); if (tid == (tsc.get_is_index_global() ? tsc.get_index_table_id() : tsc.get_ref_table_id())) { producer = &op; } } for (int64_t i = 0; OB_SUCC(ret) && NULL == producer && i < op.get_num_of_child(); i++) { if (OB_ISNULL(op.get_child(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("null child", K(ret)); } else if (log_op_def::LOG_JOIN == op.get_type()) { ObLogJoin &join_op = static_cast(op); if (IS_LEFT_SEMI_ANTI_JOIN(join_op.get_join_type()) && second_child == i) { continue; } else if (IS_RIGHT_SEMI_ANTI_JOIN(join_op.get_join_type()) && first_child == i) { continue; } if (OB_FAIL(find_pdml_part_id_producer(*op.get_child(i), tid, producer))) { LOG_WARN("find pdml part id producer failed", K(ret)); } } else if (OB_FAIL(find_pdml_part_id_producer(*op.get_child(i), tid, producer))) { LOG_WARN("find pdml part id producer failed", K(ret)); } } } return ret; } int ObLogDelUpd::inner_get_op_exprs(ObIArray &all_exprs, bool need_column_expr) { int ret = OB_SUCCESS; if (is_multi_part_dml() && OB_FAIL(generate_multi_part_partition_id_expr())) { LOG_WARN("failed to generate update expr", K(ret)); } else if (is_pdml() && need_alloc_part_id_expr_ && OB_FAIL(generate_pdml_partition_id_expr())) { LOG_WARN("failed to allocate partition id expr", K(ret)); } else if (OB_FAIL(generate_rowid_expr_for_trigger())) { LOG_WARN("failed to try add rowid col expr for trigger", K(ret)); } else if (NULL != lock_row_flag_expr_ && OB_FAIL(all_exprs.push_back(lock_row_flag_expr_))) { LOG_WARN("failed to push back expr", K(ret)); } else if (OB_FAIL(append(all_exprs, view_check_exprs_))) { LOG_WARN("failed to append exprs", K(ret)); } else if (NULL != pdml_partition_id_expr_ && OB_FAIL(all_exprs.push_back(pdml_partition_id_expr_))) { LOG_WARN("failed to push back exprs", K(ret)); } else if (OB_FAIL(get_table_columns_exprs(get_index_dml_infos(), all_exprs, need_column_expr))) { LOG_WARN("failed to add table columns to ctx", K(ret)); } else if (OB_FAIL(ObLogicalOperator::get_op_exprs(all_exprs))) { LOG_WARN("failed to get op exprs", K(ret)); } else if (get_plan()->get_optimizer_context().is_batched_multi_stmt()) { const ObDelUpdStmt *upd_stmt = static_cast(get_stmt()); if (OB_ISNULL(upd_stmt)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); } else if (OB_ISNULL(stmt_id_expr_ = upd_stmt->get_ab_stmt_id_expr())) { // new engine, stmt_id_expr_ come from update_stmt/insert_stmt, allocated by transformer ret = OB_ERR_UNEXPECTED; LOG_WARN("stmt id expr is null", K(ret)); } else if (OB_FAIL(all_exprs.push_back(upd_stmt->get_ab_stmt_id_expr()))) { LOG_WARN("add stmt id expr to all exprs failed", K(ret)); } } return ret; } int ObLogDelUpd::get_table_columns_exprs(const ObIArray &index_dml_infos, ObIArray &all_exprs, bool need_column_expr) { int ret = OB_SUCCESS; ObSEArray dml_columns; ObSEArray dml_values; if (OB_ISNULL(get_plan())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpect null plan", K(ret)); } else { need_column_expr |= get_plan()->get_optimizer_context().is_online_ddl(); } for (int64_t i = 0; OB_SUCC(ret) && i < index_dml_infos.count(); ++i) { const IndexDMLInfo *index_dml_info = index_dml_infos.at(i); dml_columns.reuse(); dml_values.reuse(); if (OB_ISNULL(index_dml_info)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); //for insert index info, no need add column exprs. } else if (need_column_expr && OB_FAIL(append(all_exprs, index_dml_info->column_exprs_))) { LOG_WARN("failed to add all_exprs to ctx", K(ret)); } else if (OB_FAIL(append_array_no_dup(all_exprs, index_dml_info->column_convert_exprs_))) { LOG_WARN("failed to append all_exprs", K(ret)); } else if (OB_FAIL(get_update_exprs(*index_dml_info, dml_columns, dml_values))) { LOG_WARN("failed to get update all_exprs", K(ret)); } else if (OB_FAIL(append_array_no_dup(all_exprs, dml_columns))) { LOG_WARN("failed to append array no dup", K(ret)); } else if (OB_FAIL(append_array_no_dup(all_exprs, dml_values))) { LOG_WARN("failed to add update all_exprs to context", K(ret)); } else if (OB_FAIL(append_array_no_dup(all_exprs, index_dml_info->ck_cst_exprs_))) { LOG_WARN("failed to append check constraint all_exprs", K(ret)); } else if (NULL != index_dml_infos.at(i)->old_part_id_expr_ && OB_FAIL(all_exprs.push_back(index_dml_info->old_part_id_expr_))) { LOG_WARN("failed to push back old partition id expr", K(ret)); } else if (NULL != index_dml_infos.at(i)->new_part_id_expr_ && OB_FAIL(all_exprs.push_back(index_dml_info->new_part_id_expr_))) { LOG_WARN("failed to push back new parititon id expr", K(ret)); } else if (NULL != index_dml_infos.at(i)->old_rowid_expr_ && OB_FAIL(all_exprs.push_back(index_dml_info->old_rowid_expr_))) { LOG_WARN("failed to push back old rowid expr", K(ret)); } else if (NULL != index_dml_infos.at(i)->new_rowid_expr_ && OB_FAIL(all_exprs.push_back(index_dml_info->new_rowid_expr_))) { LOG_WARN("failed to push back new rowid expr", K(ret)); } else if (need_column_expr) { ObColumnRefRawExpr *column_expr = NULL; for (int64_t k = 0; OB_SUCC(ret) && k < index_dml_info->column_exprs_.count(); k++) { if (OB_ISNULL(column_expr = index_dml_info->column_exprs_.at(k))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(column_expr), K(ret)); } else if (is_shadow_column(column_expr->get_column_id()) && NULL != column_expr->get_dependant_expr() && OB_FAIL(add_var_to_array_no_dup(all_exprs, column_expr->get_dependant_expr()))) { LOG_WARN("failed to add column expr", K(ret)); } else { /*do nothing*/ } } } } return ret; } int ObLogDelUpd::allocate_expr_post(ObAllocExprContext &ctx) { int ret = OB_SUCCESS; for (int64_t i = 0; OB_SUCC(ret) && i < ctx.expr_producers_.count(); i++) { ExprProducer expr_producer = ctx.expr_producers_.at(i); if (expr_producer.producer_id_ == id_ && expr_producer.expr_->is_column_ref_expr() && expr_producer.producer_branch_ == OB_INVALID_ID) { if (OB_FAIL(mark_expr_produced(const_cast(expr_producer.expr_), branch_id_, id_, ctx))) { LOG_WARN("failed to mark expr produced", K(ret)); } else { /*do nothing*/ } } } if (OB_FAIL(ret)) { /*do nothing*/ } else if (OB_FAIL(ObLogicalOperator::allocate_expr_post(ctx))) { LOG_WARN("failed to allocate expr post", K(ret)); } else { /*do nothing*/} return ret; } // table id for location lookup. uint64_t ObLogDelUpd::get_loc_table_id() const { uint64_t tid = common::OB_INVALID_ID; if (!get_index_dml_infos().empty() && NULL != get_index_dml_infos().at(0)) { tid = get_index_dml_infos().at(0)->loc_table_id_; } return tid; } uint64_t ObLogDelUpd::get_index_tid() const { uint64_t index_tid = common::OB_INVALID_ID; if (!get_index_dml_infos().empty() && NULL != get_index_dml_infos().at(0)) { index_tid = get_index_dml_infos().at(0)->ref_table_id_; } return index_tid; } uint64_t ObLogDelUpd::get_table_id() const { uint64_t table_id = common::OB_INVALID_ID; if (!get_index_dml_infos().empty() && NULL != get_index_dml_infos().at(0)) { table_id = get_index_dml_infos().at(0)->table_id_; } return table_id; } int ObLogDelUpd::est_cost() { int ret = OB_SUCCESS; ObLogicalOperator *child = NULL; if (OB_ISNULL(get_plan()) || OB_ISNULL(child = get_child(ObLogicalOperator::first_child))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(get_plan()), K(child), K(ret)); } else { ObOptimizerContext &opt_ctx = get_plan()->get_optimizer_context(); card_ = child->get_card(); op_cost_ = ObOptEstCost::cost_get_rows(child->get_card(), opt_ctx.get_cost_model_type()); cost_ = op_cost_ + child->get_cost(); } return ret; } int ObLogDelUpd::compute_sharding_info() { int ret = OB_SUCCESS; ObLogicalOperator *child = NULL; if (OB_ISNULL(child = get_child(ObLogicalOperator::first_child))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(child), K(ret)); } else if (OB_FAIL(ObLogicalOperator::compute_sharding_info())) { LOG_WARN("failed to compute sharding info", K(ret)); } else if (OB_ISNULL(get_sharding())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); } else if (!is_pdml()) { is_partition_wise_ = !is_multi_part_dml_ && !child->is_exchange_allocated() && get_sharding()->is_distributed() && NULL != get_sharding()->get_phy_table_location_info(); } return ret; } int ObLogDelUpd::check_has_trigger(uint64_t tid, bool &has_trg) { int ret = OB_SUCCESS; has_trg = false; ObSchemaGetterGuard *schema_guard = NULL; ObSQLSessionInfo *session_info = NULL; const ObTableSchema *tbl_schema = NULL; if (OB_ISNULL(get_plan()) || OB_ISNULL(schema_guard = get_plan()->get_optimizer_context().get_schema_guard()) || OB_ISNULL(session_info = get_plan()->get_optimizer_context().get_session_info())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("schema guard is null", K(ret)); } else if (OB_FAIL(schema_guard->get_table_schema( session_info->get_effective_tenant_id(), tid, tbl_schema))) { LOG_WARN("get table schema failed", K(ret)); } else if (OB_ISNULL(tbl_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema is NULL", K(ret)); } else if (tbl_schema->get_trigger_list().count() > 0) { has_trg = true; } return ret; } int ObLogDelUpd::build_rowid_expr(uint64_t table_id, uint64_t table_ref_id, const ObIArray &rowkeys, ObRawExpr *&rowid_expr) { int ret = OB_SUCCESS; ObSysFunRawExpr *rowid_sysfun_expr = NULL; ObSchemaGetterGuard *schema_guard = NULL; ObSQLSessionInfo *session_info = NULL; const ObTableSchema *tbl_schema = NULL; if (OB_ISNULL(get_plan()) || OB_ISNULL(schema_guard = get_plan()->get_optimizer_context().get_schema_guard()) || OB_ISNULL(get_plan()->get_optimizer_context().get_session_info()) || OB_ISNULL(session_info = get_plan()->get_optimizer_context().get_session_info())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("invalid params", K(ret), KP(schema_guard), KP(get_plan())); } else if (OB_FAIL(schema_guard->get_table_schema( session_info->get_effective_tenant_id(), table_ref_id, tbl_schema))) { LOG_WARN("get_table_schema failed", K(ret), K(table_ref_id)); } else if (OB_ISNULL(tbl_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema is NULL", K(ret)); } else if (OB_FAIL(ObRawExprUtils::build_rowid_expr(get_stmt(), get_plan()->get_optimizer_context().get_expr_factory(), get_plan()->get_optimizer_context().get_allocator(), *(get_plan()->get_optimizer_context().get_session_info()), *tbl_schema, table_id, rowkeys, rowid_sysfun_expr))) { LOG_WARN("failed to build rowid col expr", K(ret)); } else if (OB_ISNULL(rowid_sysfun_expr)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("rowid sysfun_expr is NULL", K(ret)); } else { rowid_expr = rowid_sysfun_expr; } return ret; } int ObLogDelUpd::get_rowid_version(int64_t &rowid_version) { int ret = OB_SUCCESS; ObSchemaGetterGuard *schema_guard = NULL; ObSQLSessionInfo *session_info = NULL; const ObTableSchema *table_schema = NULL; ObSEArray rowkey_exprs; if (OB_ISNULL(get_plan())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema or plan is NULL", K(ret), KP(schema_guard), KP(get_plan())); } else if (OB_ISNULL(schema_guard = get_plan()->get_optimizer_context().get_schema_guard()) || OB_ISNULL(session_info = get_plan()->get_optimizer_context().get_session_info())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get schama guart", K(ret), K(schema_guard), K(session_info)); } else if (OB_FAIL(schema_guard->get_table_schema( session_info->get_effective_tenant_id(), get_index_tid(), table_schema))) { LOG_WARN("failed to get table schema", K(ret), K(get_index_tid())); } else if (OB_ISNULL(table_schema)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table schema is NULL", K(ret)); } else { ObSEArray col_ids; int64_t rowkey_cnt = 0; OZ(table_schema->get_column_ids_serialize_to_rowid(col_ids, rowkey_cnt)); OZ(table_schema->get_rowid_version(rowkey_cnt, col_ids.count(), rowid_version)); LOG_DEBUG("get get_rowid_version is", K(rowid_version)); } return ret; } int ObLogDelUpd::compute_op_ordering() { int ret = OB_SUCCESS; get_op_ordering().reset(); is_local_order_ = false; return ret; } int ObLogDelUpd::compute_plan_type() { int ret = OB_SUCCESS; if (OB_FAIL(ObLogicalOperator::compute_plan_type())) { LOG_WARN("failed to compute plan type", K(ret)); } else if (is_multi_part_dml()) { location_type_ = ObPhyPlanType::OB_PHY_PLAN_UNCERTAIN; } else { /*do nothing*/ } return ret; } int ObLogDelUpd::get_table_location_type(ObTableLocationType &type) { int ret = OB_SUCCESS; ObOptimizerContext& optimizer_context = get_plan()->get_optimizer_context(); ObAddr &server = optimizer_context.get_local_server_addr(); type = OB_TBL_LOCATION_UNINITIALIZED; if (OB_ISNULL(table_partition_info_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get unexpected null", K(ret)); } else if (OB_FAIL(table_partition_info_->get_location_type(server, type))) { LOG_WARN("get location type failed", K(ret)); } else { /*do nothing*/ } return ret; } int ObLogDelUpd::assign_dml_infos(const ObIArray &index_dml_infos) { int ret = OB_SUCCESS; if (OB_FAIL(index_dml_infos_.assign(index_dml_infos))) { LOG_WARN("failed to assign index dml infos", K(ret)); } for (int64_t i = 0; OB_SUCC(ret) && i < index_dml_infos.count(); ++i) { if (OB_ISNULL(index_dml_infos.at(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("index dml info is null", K(ret)); } else if (index_dml_infos.at(i)->is_primary_index_ && OB_FAIL(loc_table_list_.push_back(index_dml_infos.at(i)->loc_table_id_))) { LOG_WARN("failed to add loc table id", K(ret)); } } return ret; } int ObLogDelUpd::get_index_dml_infos(uint64_t loc_table_id, ObIArray &index_infos) const { int ret = OB_SUCCESS; for (int64_t i = 0; OB_SUCC(ret) && i < index_dml_infos_.count(); ++i) { if (OB_ISNULL(index_dml_infos_.at(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("dml index info is null", K(ret)); } else if (index_dml_infos_.at(i)->loc_table_id_ != loc_table_id) { // do nothing } else if (OB_FAIL(index_infos.push_back(index_dml_infos_.at(i)))) { LOG_WARN("failed to push back dml index info", K(ret)); } } return ret; } int ObLogDelUpd::get_index_dml_infos(uint64_t loc_table_id, ObIArray &index_infos) { int ret = OB_SUCCESS; for (int64_t i = 0; OB_SUCC(ret) && i < index_dml_infos_.count(); ++i) { if (OB_ISNULL(index_dml_infos_.at(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("dml index info is null", K(ret)); } else if (index_dml_infos_.at(i)->loc_table_id_ != loc_table_id) { // do nothing } else if (OB_FAIL(index_infos.push_back(index_dml_infos_.at(i)))) { LOG_WARN("failed to push back dml index info", K(ret)); } } return ret; } IndexDMLInfo* ObLogDelUpd::get_primary_dml_info() { IndexDMLInfo *ret = NULL; if (1 == loc_table_list_.count()) { ret = get_primary_dml_info(loc_table_list_.at(0)); } return ret; } const IndexDMLInfo* ObLogDelUpd::get_primary_dml_info() const { const IndexDMLInfo *ret = NULL; if (1 == loc_table_list_.count()) { ret = get_primary_dml_info(loc_table_list_.at(0)); } return ret; } IndexDMLInfo* ObLogDelUpd::get_primary_dml_info(uint64_t loc_table_id) { IndexDMLInfo *ret = NULL; for (int64_t i = 0; i < index_dml_infos_.count(); ++i) { if (NULL == index_dml_infos_.at(i)) { // do nothing } else if (index_dml_infos_.at(i)->loc_table_id_ == loc_table_id && index_dml_infos_.at(i)->is_primary_index_) { ret = index_dml_infos_.at(i); break; } } return ret; } const IndexDMLInfo* ObLogDelUpd::get_primary_dml_info(uint64_t loc_table_id) const { const IndexDMLInfo *ret = NULL; for (int64_t i = 0; i < index_dml_infos_.count(); ++i) { if (NULL == index_dml_infos_.at(i)) { // do nothing } else if (index_dml_infos_.at(i)->loc_table_id_ == loc_table_id && index_dml_infos_.at(i)->is_primary_index_) { ret = index_dml_infos_.at(i); break; } } return ret; } const ObIArray& ObLogDelUpd::get_table_list() const { return loc_table_list_; } int ObLogDelUpd::get_table_index_name(const IndexDMLInfo &index_info, ObString &table_name, ObString &index_name) { int ret = OB_SUCCESS; const ObDelUpdStmt *stmt = NULL; const TableItem* table_item = nullptr; ObSEArray table_infos; if (OB_ISNULL(stmt = static_cast(get_stmt()))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("failed to get primary dml info", K(ret), K(stmt)); } else if (OB_FAIL(stmt->get_dml_table_infos(table_infos))) { LOG_WARN("failed to get dml table infos", K(ret)); } else { index_name = index_info.index_name_; bool find = false; for (int64_t i = 0; OB_SUCC(ret) && !find && i < table_infos.count(); ++i) { const ObDmlTableInfo* table_info = table_infos.at(i); if (OB_ISNULL(table_info)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get null table info", K(ret)); } else if (table_info->loc_table_id_ != index_info.loc_table_id_) { // do nothing } else if (OB_ISNULL(table_item = stmt->get_table_item_by_id(table_info->table_id_))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("get null table item", K(ret)); } else { table_name = table_item->get_table_name(); find = true; } } } return ret; } int ObLogDelUpd::print_table_infos(const ObString &prefix, char *buf, int64_t &buf_len, int64_t &pos, ExplainType type) { int ret = OB_SUCCESS; const ObIArray &index_dml_infos = get_index_dml_infos(); // pre check validity for (int64_t i = 0; OB_SUCC(ret) && i < index_dml_infos.count(); ++i) { if (OB_ISNULL(index_dml_infos.at(i))) { ret = OB_ERR_UNEXPECTED; LOG_WARN("index dml info is null", K(ret)); } } if (OB_SUCC(ret)) { pos += prefix.to_string(buf + pos, buf_len - pos); BUF_PRINTF("("); if (index_dml_infos.count() == 0) { BUF_PRINTF("nil"); } else { for (int64_t i = 0; i < index_dml_infos.count(); ++i) { const IndexDMLInfo *dml_info = index_dml_infos.at(i); ObString table_name; ObString index_name; ret = get_table_index_name(*dml_info, table_name, index_name); BUF_PRINTF("["); BUF_PRINTF("{"); pos += table_name.to_string(buf + pos, buf_len - pos); BUF_PRINTF(": ("); int64_t j = i + 1; for (; j < index_dml_infos.count() && index_dml_infos.at(j)->loc_table_id_ == dml_info->loc_table_id_; ++j); for (; i < j; ++i) { pos += index_dml_infos.at(i)->to_explain_string(buf + pos, buf_len - pos, type); BUF_PRINTF(", "); } --i; pos -= 2; BUF_PRINTF(")"); if (T_MERGE_DISTINCT == dml_info->distinct_algo_) { BUF_PRINTF(", merge_distinct"); } else if (T_HASH_DISTINCT == dml_info->distinct_algo_) { BUF_PRINTF(", hash_distinct"); } BUF_PRINTF("}"); BUF_PRINTF("]"); if (i < index_dml_infos.count() - 1) { BUF_PRINTF(", "); } } } BUF_PRINTF(")"); } return ret; } int ObLogDelUpd::print_assigns(const ObAssignments &assigns, char *buf, int64_t &buf_len, int64_t &pos, ExplainType type) { int ret = OB_SUCCESS; int64_t N = assigns.count(); if (N == 0) { if (OB_FAIL(BUF_PRINTF("nil"))) { LOG_WARN("BUG_PRINTF fails", K(ret)); } else { /* Do nothing */ } } else { for (int64_t i = 0; OB_SUCC(ret) && i < N; i++) { if (OB_FAIL(BUF_PRINTF("["))) { LOG_WARN("BUG_PRINTF fails", K(ret)); } else if (OB_ISNULL(assigns.at(i).column_expr_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("variable_ or variable_->expr_ is NULL", K(ret)); } else if (OB_FAIL(assigns.at(i).column_expr_ ->get_name(buf, buf_len, pos, type))) { LOG_WARN("get_name fails", K(ret)); } else if(OB_FAIL(BUF_PRINTF("="))) { LOG_WARN("BUG_PRINTF fails", K(ret)); } else if (OB_ISNULL(assigns.at(i).expr_)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("expr_ is NULL", K(ret)); } else if (OB_FAIL(assigns.at(i).expr_ ->get_name(buf, buf_len, pos, type))) { LOG_WARN("get_name fails", K(ret)); } else if (OB_FAIL(BUF_PRINTF("]"))) { LOG_WARN("BUG_PRINTF fails", K(ret)); } else { /* Do nothing */ } if (OB_SUCCESS == ret && i < N - 1) { if (OB_FAIL(BUF_PRINTF(", "))) { LOG_WARN("BUG_PRINTF fails", K(ret)); } else { /* Do nothing */ } } else { /* Do nothing */ } } } return ret; } const common::ObIArray* ObLogDelUpd::get_table_columns() const { const IndexDMLInfo *index_info = get_primary_dml_info(); const common::ObIArray *ret = NULL; if (NULL != index_info) { ret = &index_info->column_exprs_; } return ret; } int ObLogDelUpd::generate_old_rowid_expr(IndexDMLInfo &table_dml_info) { int ret = OB_SUCCESS; ObSEArray rowkey_exprs; if (OB_FAIL(table_dml_info.get_rowkey_exprs(rowkey_exprs))) { LOG_WARN("failed to get rowkey exprs", K(ret)); } else if (OB_FAIL(build_rowid_expr(table_dml_info.loc_table_id_, table_dml_info.ref_table_id_, rowkey_exprs, table_dml_info.old_rowid_expr_))) { LOG_WARN("failed to build rowid expr", K(ret)); } return ret; } int ObLogDelUpd::generate_update_new_rowid_expr(IndexDMLInfo &table_dml_info) { int ret = OB_SUCCESS; ObSEArray update_columns; ObSEArray update_values; if (OB_FAIL(get_update_exprs(table_dml_info, update_columns, update_values))) { LOG_WARN("failed to get update exprs", K(ret)); } else if (OB_FAIL(convert_expr_by_dml_operation(update_columns, update_values, table_dml_info.old_rowid_expr_, table_dml_info.new_rowid_expr_))) { LOG_WARN("failed to convert expr by dml operation", K(ret)); } return ret; } int ObLogDelUpd::generate_insert_new_rowid_expr(IndexDMLInfo &table_dml_info) { int ret = OB_SUCCESS; ObSEArray insert_columns; ObSEArray insert_values; if (OB_FAIL(get_insert_exprs(table_dml_info, insert_columns, insert_values))) { LOG_WARN("failed to get insert exprs", K(ret)); } else if (OB_FAIL(convert_expr_by_dml_operation(insert_columns, insert_values, table_dml_info.old_rowid_expr_, table_dml_info.new_rowid_expr_))) { LOG_WARN("failed to convert expr by dml operation", K(ret)); } else { // insert does not have old rowid expr table_dml_info.old_rowid_expr_ = NULL; } return ret; } int ObLogDelUpd::generate_old_calc_partid_expr(IndexDMLInfo &index_info) { int ret = OB_SUCCESS; if (OB_ISNULL(get_plan())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("plan is null", K(ret)); } else if (OB_FAIL(get_plan()->gen_calc_part_id_expr(index_info.loc_table_id_, index_info.ref_table_id_, CALC_PARTITION_TABLET_ID, index_info.old_part_id_expr_))) { LOG_WARN("failed to gen calc part id expr", K(ret)); } return ret; } int ObLogDelUpd::generate_insert_new_calc_partid_expr(IndexDMLInfo &index_dml_info) { int ret = OB_SUCCESS; ObSEArray dml_columns; ObSEArray dml_values; if (OB_FAIL(get_insert_exprs(index_dml_info, dml_columns, dml_values))) { LOG_WARN("failed to get insert exprs", K(ret)); } else if (OB_FAIL(convert_expr_by_dml_operation( dml_columns, dml_values, index_dml_info.old_part_id_expr_, index_dml_info.new_part_id_expr_))) { LOG_WARN("failed to convert expr by dml operation", K(ret)); } else { // insert does not have old partition id expr index_dml_info.old_part_id_expr_ = NULL; } return ret; } int ObLogDelUpd::generate_update_new_calc_partid_expr(IndexDMLInfo &index_dml_info) { int ret = OB_SUCCESS; ObSEArray update_columns; ObSEArray update_values; if (OB_FAIL(get_update_exprs(index_dml_info, update_columns, update_values))) { LOG_WARN("failed to get update exprs", K(ret)); } else if (OB_FAIL(convert_expr_by_dml_operation(update_columns, update_values, index_dml_info.old_part_id_expr_, index_dml_info.new_part_id_expr_))) { LOG_WARN("failed to convert expr by dml operation", K(ret)); } return ret; } int ObLogDelUpd::convert_expr_by_dml_operation(const ObIArray &dml_columns, const ObIArray &dml_new_values, ObRawExpr *cur_value, ObRawExpr *&new_value) { int ret = OB_SUCCESS; if (OB_ISNULL(cur_value) || OB_ISNULL(get_plan())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("current value expr is null", K(ret), K(get_plan()), K(cur_value)); } else { ObRawExprCopier copier(get_plan()->get_optimizer_context().get_expr_factory()); if (OB_FAIL(copier.add_replaced_expr(dml_columns, dml_new_values))) { LOG_WARN("failed to add replace pair", K(ret)); } else if (OB_FAIL(copier.copy_on_replace(cur_value, new_value))) { LOG_WARN("failed to copy on replace expr", K(ret)); } } return ret; } int ObLogDelUpd::get_update_exprs(const IndexDMLInfo &dml_info, ObIArray &dml_columns, ObIArray &dml_values) { int ret = OB_SUCCESS; const ObAssignments &assigns = dml_info.assignments_; for (int64_t i = 0; OB_SUCC(ret) && i < assigns.count(); ++i) { if (OB_FAIL(dml_columns.push_back(assigns.at(i).column_expr_))) { LOG_WARN("failed to push back assign column", K(ret)); } else if (OB_FAIL(dml_values.push_back(assigns.at(i).expr_))) { LOG_WARN("failed to push back assign value", K(ret)); } } return ret; } int ObLogDelUpd::get_insert_exprs(const IndexDMLInfo &dml_info, ObIArray &dml_columns, ObIArray &dml_values) { int ret = OB_SUCCESS; if (OB_UNLIKELY(dml_info.column_exprs_.count() != dml_info.column_convert_exprs_.count())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("column and values size do not match", K(ret)); } else if (OB_FAIL(append(dml_columns, dml_info.column_exprs_))) { LOG_WARN("failed to append column exprs", K(ret)); } else if (OB_FAIL(append(dml_values, dml_info.column_convert_exprs_))) { LOG_WARN("failed to append value exprs", K(ret)); } return ret; } int ObLogDelUpd::print_outline(planText &plan_text) { int ret = OB_SUCCESS; if (OB_ISNULL(get_plan())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected NULL", K(ret), K(get_plan())); } else if (USED_HINT == plan_text.outline_type_) { const ObHint *hint = get_plan()->get_log_plan_hint().get_normal_hint(T_USE_DISTRIBUTED_DML); if (NULL != hint) { bool match_hint = is_multi_part_dml() ? hint->is_enable_hint() : hint->is_disable_hint(); if (match_hint && OB_FAIL(hint->print_hint(plan_text))) { LOG_WARN("failed to print use multi part dml hint", K(ret), K(*hint)); } } } else if (OUTLINE_DATA == plan_text.outline_type_ && is_multi_part_dml()) { char *buf = plan_text.buf; int64_t &buf_len = plan_text.buf_len; int64_t &pos = plan_text.pos; const ObDMLStmt *stmt = NULL; ObString qb_name; if (OB_ISNULL(stmt = get_plan()->get_stmt())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("unexpected NULL", K(ret), K(stmt)); } else if (OB_FAIL(stmt->get_qb_name(qb_name))) { LOG_WARN("fail to get qb_name", K(ret), K(stmt->get_stmt_id())); } else if (OB_FAIL(BUF_PRINTF("%s%s(@\"%.*s\")", ObQueryHint::get_outline_indent(plan_text.is_oneline_), ObHint::get_hint_name(T_USE_DISTRIBUTED_DML), qb_name.length(), qb_name.ptr()))) { LOG_WARN("fail to print buffer", K(ret), K(buf), K(buf_len), K(pos)); } } return ret; } int ObLogDelUpd::copy_part_expr_pre(CopyPartExprCtx &ctx) { int ret = OB_SUCCESS; for (int64_t i = 0; OB_SUCC(ret) && i < get_index_dml_infos().count(); ++i) { IndexDMLInfo *dml_info = get_index_dml_infos().at(i); if (OB_ISNULL(dml_info)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("index dml info is null", K(ret)); } else if (NULL != dml_info->old_part_id_expr_ && OB_FAIL(copy_part_expr(ctx, dml_info->old_part_id_expr_))) { LOG_WARN("failed to copy part expr", K(ret)); } } return ret; }