patch 4.0

This commit is contained in:
wangzelin.wzl
2022-10-24 10:34:53 +08:00
parent 4ad6e00ec3
commit 93a1074b0c
10533 changed files with 2588271 additions and 2299373 deletions

View File

@ -13,13 +13,9 @@
#define USING_LOG_PREFIX SQL_OPT
#include "sql/resolver/dml/ob_merge_stmt.h"
#include "sql/optimizer/ob_log_merge.h"
#include "lib/container/ob_se_array.h"
#include "lib/container/ob_se_array_iterator.h"
#include "share/partition_table/ob_partition_location_cache.h"
#include "sql/ob_phy_table_location.h"
#include "sql/code_generator/ob_expr_generator_impl.h"
#include "sql/ob_sql_utils.h"
#include "sql/optimizer/ob_log_plan.h"
#include "sql/optimizer/ob_merge_log_plan.h"
using namespace oceanbase;
using namespace sql;
@ -27,41 +23,67 @@ using namespace oceanbase::common;
using namespace share;
using namespace oceanbase::share::schema;
int ObLogMerge::print_my_plan_annotation(char* buf, int64_t& buf_len, int64_t& pos, ExplainType type)
int ObLogMerge::print_my_plan_annotation(char *buf,
int64_t &buf_len,
int64_t &pos,
ExplainType type)
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObLogInsert::print_my_plan_annotation(buf, buf_len, pos, type))) {
LOG_WARN("fail to print plan annotaqqtion", K(ret));
} else if (OB_FAIL(BUF_PRINTF(", "))) {
LOG_WARN("BUF_PRINTF fails", K(ret));
if (OB_FAIL(BUF_PRINTF(", "))) {
LOG_WARN("BUG_PRINTF fails", K(ret));
} else if (OB_FAIL(BUF_PRINTF("\n "))) {
LOG_WARN("BUF_PRINTF fails", K(ret));
} else if (OB_ISNULL(match_condition_exprs_) || OB_ISNULL(update_condition_exprs_) ||
OB_ISNULL(insert_condition_exprs_) || OB_ISNULL(delete_condition_exprs_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("some condition exprs is NULL",
KPC(match_condition_exprs_),
KPC(update_condition_exprs_),
KPC(insert_condition_exprs_),
KPC(delete_condition_exprs_),
K(ret));
} else {
const ObIArray<ObRawExpr*>& match_conds = *match_condition_exprs_;
const ObIArray<ObRawExpr*>& update_conds = *update_condition_exprs_;
const ObIArray<ObRawExpr*>& delete_conds = *delete_condition_exprs_;
const ObIArray<ObRawExpr*>& insert_conds = *insert_condition_exprs_;
EXPLAIN_PRINT_EXPRS(match_conds, type);
if (OB_SUCC(ret) && OB_FAIL(BUF_PRINTF(", "))) {
LOG_WARN("BUF_PRINTF fails", K(ret));
LOG_WARN("BUG_PRINTF fails", K(ret));
} else if (OB_FAIL(print_table_infos(ObString::make_string("columns"),
buf, buf_len, pos, type))) {
LOG_WARN("failed to print table info", K(ret));
} else if (NULL != table_partition_info_) {
if (OB_SUCC(ret)) {
if(OB_FAIL(BUF_PRINTF(", "))) {
LOG_WARN("BUG_PRINTF fails", K(ret));
} else if (OB_FAIL(explain_print_partitions(*table_partition_info_, buf, buf_len, pos))) {
LOG_WARN("Failed to print partitions");
} else { }//do nothing
}
EXPLAIN_PRINT_EXPRS(insert_conds, type);
if (OB_FAIL(ret)) {
} else if (OB_FAIL(BUF_PRINTF(", "))) {
}
if (OB_SUCC(ret) && NULL != get_primary_dml_info() &&
!get_primary_dml_info()->column_convert_exprs_.empty()) {
const ObIArray<ObRawExpr *> &column_values = get_primary_dml_info()->column_convert_exprs_;
if(OB_FAIL(BUF_PRINTF(", "))) {
LOG_WARN("BUG_PRINTF fails", K(ret));
} else if (OB_FAIL(BUF_PRINTF("\n "))) {
LOG_WARN("BUG_PRINTF fails", K(ret));
} else {
EXPLAIN_PRINT_EXPRS(column_values, type);
}
}
if (OB_SUCC(ret) && !get_update_infos().empty() && NULL != get_update_infos().at(0)) {
const ObAssignments &assigns = get_update_infos().at(0)->assignments_;
if (OB_FAIL(BUF_PRINTF(",\n"))) {
LOG_WARN("BUG_PRINTF fails", K(ret));
} else if (OB_FAIL(BUF_PRINTF(" update("))) {
LOG_WARN("BUG_PRINTF fails", K(ret));
} else if (OB_FAIL(print_assigns(assigns,
buf, buf_len, pos, type))) {
LOG_WARN("failed to print assigns", K(ret));
} else { /* Do nothing */ }
BUF_PRINTF(")");
}
if (OB_SUCC(ret)) {
if (OB_FAIL(BUF_PRINTF(", "))) {
LOG_WARN("BUF_PRINTF fails", K(ret));
} else if (OB_FAIL(BUF_PRINTF("\n "))) {
LOG_WARN("BUF_PRINTF fails", K(ret));
}
}
if (OB_SUCC(ret)) {
const ObIArray<ObRawExpr *> &update_conds = get_update_condition();
const ObIArray<ObRawExpr *> &delete_conds = get_delete_condition();
const ObIArray<ObRawExpr *> &insert_conds = get_insert_condition();
EXPLAIN_PRINT_EXPRS(insert_conds, type);
if (OB_SUCC(ret) && OB_FAIL(BUF_PRINTF(", "))) {
LOG_WARN("BUF_PRINTF fails", K(ret));
}
EXPLAIN_PRINT_EXPRS(update_conds, type);
if (OB_SUCC(ret) && OB_FAIL(BUF_PRINTF(", "))) {
@ -72,298 +94,277 @@ int ObLogMerge::print_my_plan_annotation(char* buf, int64_t& buf_len, int64_t& p
return ret;
}
int ObLogMerge::add_all_table_assignments_to_ctx(ObAllocExprContext& ctx)
{
int ret = OB_SUCCESS;
int64_t table_cnt = tables_assignments_->count();
if (table_cnt == 0) {
// do nothing
} else if (table_cnt == 1) {
ObSEArray<ObRawExpr*, 8> exprs;
const ObAssignments& assigns = tables_assignments_->at(0).assignments_;
for (int64_t i = 0; OB_SUCC(ret) && i < assigns.count(); i++) {
if (OB_ISNULL(assigns.at(i).expr_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (OB_FAIL(exprs.push_back(assigns.at(i).expr_))) {
LOG_WARN("failed to push back exprs", K(ret));
} else { /*do nothing*/
}
}
if (OB_SUCC(ret)) {
if (OB_FAIL(add_merge_exprs_to_ctx(ctx, exprs))) {
LOG_WARN("failed to add exprs to ctx", K(ret));
} else { /*do nothing*/
}
}
} else {
LOG_WARN("unexpected assignment table count", K(table_cnt));
}
return ret;
}
int ObLogMerge::allocate_expr_pre(ObAllocExprContext& ctx)
{
int ret = OB_SUCCESS;
if (OB_FAIL(add_all_table_assignments_to_ctx(ctx))) {
LOG_WARN("fail to add all table assignment", K(ret));
} else if (NULL != match_condition_exprs_ && OB_FAIL(add_merge_exprs_to_ctx(ctx, *match_condition_exprs_))) {
LOG_WARN("fail to add expr to ctx", K(ret));
} else if (NULL != insert_condition_exprs_ && OB_FAIL(add_merge_exprs_to_ctx(ctx, *insert_condition_exprs_))) {
LOG_WARN("fail to add expr to ctx", K(ret));
} else if (NULL != update_condition_exprs_ && OB_FAIL(add_merge_exprs_to_ctx(ctx, *update_condition_exprs_))) {
LOG_WARN("fail to add expr to ctx", K(ret));
} else if (OB_FAIL(add_delete_exprs_to_ctx(ctx))) {
LOG_WARN("fail to add expr to ctx", K(ret));
} else if (nullptr != value_vector_ && OB_FAIL(add_merge_exprs_to_ctx(ctx, *value_vector_))) {
LOG_WARN("fail to add expr to ctx", K(ret));
} else if (OB_FAIL(add_all_source_table_columns_to_ctx(ctx))) {
LOG_WARN("fail to add source table columns to ctx", K(ret));
} else if (OB_FAIL(ObLogicalOperator::allocate_expr_pre(ctx))) {
LOG_WARN("failed to add parent need expr", K(ret));
}
return ret;
}
int ObLogMerge::add_merge_exprs_to_ctx(ObAllocExprContext& ctx, const ObIArray<ObRawExpr*>& exprs)
{
int ret = OB_SUCCESS;
ObLogicalOperator* child = NULL;
uint64_t producer_id = OB_INVALID_ID;
ObSEArray<ObRawExpr*, 8> subquery_exprs;
ObSEArray<ObRawExpr*, 8> non_subquery_exprs;
if (OB_FAIL(classify_merge_subquery_expr(exprs, subquery_exprs, non_subquery_exprs))) {
LOG_WARN("failed to classify merge subquery exprs", K(ret));
} else if (!non_subquery_exprs.empty() && OB_FAIL(add_exprs_to_ctx(ctx, non_subquery_exprs))) {
LOG_WARN("failed to add exprs to ctx", K(ret));
} else if (subquery_exprs.empty()) {
/*do nothing*/
} else if (OB_ISNULL(child = get_child(ObLogicalOperator::first_child))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (OB_FAIL(get_next_producer_id(child, producer_id))) {
LOG_WARN("failed to get next producer id", K(ret));
} else if (OB_FAIL(add_exprs_to_ctx(ctx, subquery_exprs, producer_id))) {
LOG_WARN("failed to add exprs to ctx", K(ret));
} else { /*do nothing*/
}
return ret;
}
int ObLogMerge::classify_merge_subquery_expr(
const ObIArray<ObRawExpr*>& exprs, ObIArray<ObRawExpr*>& subquery_exprs, ObIArray<ObRawExpr*>& non_subquery_exprs)
int ObLogMerge::compute_sharding_info()
{
int ret = OB_SUCCESS;
if (OB_ISNULL(get_plan())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < exprs.count(); i++) {
bool has_subquery = false;
if (OB_ISNULL(exprs.at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (OB_FAIL(ObOptimizerUtil::check_expr_contain_subquery(
exprs.at(i), &get_plan()->get_onetime_exprs(), has_subquery))) {
LOG_WARN("failed to check whether contain subquery", K(ret));
} else if (has_subquery) {
ret = subquery_exprs.push_back(exprs.at(i));
} else {
ret = non_subquery_exprs.push_back(exprs.at(i));
}
}
}
} else if (NULL != get_sharding()) {
is_partition_wise_ = true;
} else if (get_plan()->get_optimizer_context().use_pdml() && is_multi_part_dml()) {
// pdml merge
strong_sharding_ = get_plan()->get_optimizer_context().get_distributed_sharding();
} else if (is_multi_part_dml()) {
strong_sharding_ = get_plan()->get_optimizer_context().get_local_sharding();
} else if (OB_FAIL(ObLogDelUpd::compute_sharding_info())) {
LOG_WARN("failed to compute sharding info", K(ret));
} else { /*do nothing*/ }
return ret;
}
int ObLogMerge::add_delete_exprs_to_ctx(ObAllocExprContext& ctx)
int ObLogMerge::get_op_exprs(ObIArray<ObRawExpr*> &all_exprs)
{
int ret = OB_SUCCESS;
ObRawExpr* expr = NULL;
ObSEArray<ObRawExpr*, 8> subquery_exprs;
ObSEArray<ObRawExpr*, 8> non_subquery_exprs;
if (OB_ISNULL(delete_condition_exprs_)) {
if (OB_FAIL(ObLogDelUpd::inner_get_op_exprs(all_exprs, true))) {
LOG_WARN("failed to get op exprs", K(ret));
} else if (OB_FAIL(get_table_columns_exprs(get_update_infos(),
all_exprs,
true))) {
LOG_WARN("failed to add update dml info exprs", K(ret));
} else if (OB_FAIL(get_table_columns_exprs(get_delete_infos(),
all_exprs,
true))) {
LOG_WARN("failed to add delete dml info exprs", K(ret));
} else if (OB_FAIL(append_array_no_dup(all_exprs, get_insert_condition()))) {
LOG_WARN("failed to add exprs to ctx", K(ret));
} else if (OB_FAIL(append_array_no_dup(all_exprs, get_update_condition()))) {
LOG_WARN("failed to add exprs to ctx", K(ret));
} else if (OB_FAIL(append_array_no_dup(all_exprs, get_delete_condition()))) {
LOG_WARN("failed to add exprs to ctx", K(ret));
} else { /*do nothing*/ }
return ret;
}
int ObLogMerge::get_modified_index_id(common::ObIArray<uint64_t> &index_tids)
{
int ret = OB_SUCCESS;
const ObMergeStmt *merge_stmt = NULL;
index_tids.reuse();
if (OB_ISNULL(merge_stmt = static_cast<const ObMergeStmt *>(get_stmt()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (OB_FAIL(classify_merge_subquery_expr(*delete_condition_exprs_, subquery_exprs, non_subquery_exprs))) {
LOG_WARN("failed to classify merge subquery exprs", K(ret));
} else {
if (!non_subquery_exprs.empty()) {
ObSEArray<ObRawExpr*, 8> delete_column_exprs;
if (OB_FAIL(ObRawExprUtils::extract_column_exprs(non_subquery_exprs, delete_column_exprs))) {
LOG_WARN("fail to extract column exprs", K(ret));
} else if (OB_FAIL(add_exprs_to_ctx(ctx, delete_column_exprs))) {
LOG_WARN("fail to add exprs to ctx", K(ret));
} else { /*do nothing*/
}
}
if (OB_SUCC(ret) && !subquery_exprs.empty()) {
ObLogicalOperator* child = NULL;
uint64_t producer_id = OB_INVALID_ID;
if (OB_ISNULL(child = get_child(ObLogicalOperator::first_child))) {
LOG_WARN("merge stmt is null", K(ret));
} else if (merge_stmt->has_insert_clause()) {
// Reminder, the resolver mock a insert dml info even if the merge stmt does not have a insert clause
for (int64_t i = 0; OB_SUCC(ret) && i < get_index_dml_infos().count(); ++i) {
if (OB_ISNULL(get_index_dml_infos().at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (OB_FAIL(get_next_producer_id(child, producer_id))) {
LOG_WARN("failed to get next producer id", K(ret));
} else if (OB_FAIL(add_exprs_to_ctx(ctx, subquery_exprs, producer_id))) {
LOG_WARN("failed to add exprs to ctx", K(ret));
} else { /*do nothing*/
LOG_WARN("index dml info is null", K(ret));
} else if (OB_FAIL(index_tids.push_back(get_index_dml_infos().at(i)->ref_table_id_))) {
LOG_WARN("failed to push back insert index id", K(ret));
}
}
} else if (!get_delete_infos().empty()) {
for (int64_t i = 0; OB_SUCC(ret) && i < get_delete_infos().count(); ++i) {
if (OB_ISNULL(get_delete_infos().at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("delete dml info is null", K(ret));
} else if (OB_FAIL(index_tids.push_back(get_delete_infos().at(i)->ref_table_id_))) {
LOG_WARN("failed to push back delete index id", K(ret));
}
}
}
return ret;
}
int ObLogMerge::check_output_dep_specific(ObRawExprCheckDep& checker)
{
int ret = OB_SUCCESS;
if (OB_FAIL(ObLogInsert::check_output_dep_specific(checker))) {
LOG_WARN("ObLogDelUpd::check_output_dep_specific fails", K(ret));
} else {
if (NULL != match_condition_exprs_) {
for (int64_t i = 0; OB_SUCC(ret) && i < match_condition_exprs_->count(); i++) {
if (OB_ISNULL(match_condition_exprs_->at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (OB_FAIL(checker.check(*match_condition_exprs_->at(i)))) {
LOG_WARN("failed to check expr", K(ret));
} else { /*do nothing*/
}
}
}
if (OB_SUCC(ret) && NULL != insert_condition_exprs_) {
for (int64_t i = 0; OB_SUCC(ret) && i < insert_condition_exprs_->count(); i++) {
if (OB_ISNULL(insert_condition_exprs_->at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (OB_FAIL(checker.check(*insert_condition_exprs_->at(i)))) {
LOG_WARN("failed to check expr", K(ret));
} else { /*do nothing*/
}
}
}
if (OB_SUCC(ret) && NULL != update_condition_exprs_) {
for (int64_t i = 0; OB_SUCC(ret) && i < update_condition_exprs_->count(); i++) {
if (OB_ISNULL(update_condition_exprs_->at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (OB_FAIL(checker.check(*update_condition_exprs_->at(i)))) {
LOG_WARN("failed to check expr", K(ret));
} else { /*do nothing*/
}
}
}
if (OB_SUCC(ret) && NULL != delete_condition_exprs_) {
for (int64_t i = 0; OB_SUCC(ret) && i < delete_condition_exprs_->count(); i++) {
if (OB_ISNULL(delete_condition_exprs_->at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (OB_FAIL(checker.check(*delete_condition_exprs_->at(i)))) {
LOG_WARN("failed to check expr", K(ret));
} else { /*do nothing*/
}
}
}
if (OB_SUCC(ret) && NULL != value_vector_) {
for (int64_t i = 0; OB_SUCC(ret) && i < value_vector_->count(); i++) {
if (OB_ISNULL(value_vector_->at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (OB_FAIL(checker.check(*value_vector_->at(i)))) {
LOG_WARN("failed to check expr", K(ret));
} else { /*do nothing*/
}
}
}
if (OB_SUCC(ret) && NULL != rowkey_exprs_) {
for (int64_t i = 0; OB_SUCC(ret) && i < rowkey_exprs_->count(); i++) {
if (OB_ISNULL(rowkey_exprs_->at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret));
} else if (OB_FAIL(checker.check(*rowkey_exprs_->at(i)))) {
LOG_WARN("failed to check expr", K(ret));
} else { /*do nothing*/
}
for (int64_t i = 0; OB_SUCC(ret) && i < get_update_infos().count(); ++i) {
if (OB_ISNULL(get_update_infos().at(i))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("update dml info is null", K(ret));
} else if (OB_FAIL(index_tids.push_back(get_update_infos().at(i)->ref_table_id_))) {
LOG_WARN("failed to push back update index id", K(ret));
}
}
}
return ret;
}
uint64_t ObLogMerge::hash(uint64_t seed) const
{
if (NULL != match_condition_exprs_) {
HASH_PTR_ARRAY(*match_condition_exprs_, seed);
}
if (NULL != insert_condition_exprs_) {
HASH_PTR_ARRAY(*insert_condition_exprs_, seed);
}
if (NULL != update_condition_exprs_) {
HASH_PTR_ARRAY(*update_condition_exprs_, seed);
}
if (NULL != delete_condition_exprs_) {
HASH_PTR_ARRAY(*delete_condition_exprs_, seed);
}
if (NULL != value_vector_) {
HASH_PTR_ARRAY(*value_vector_, seed);
}
if (NULL != rowkey_exprs_) {
HASH_PTR_ARRAY(*rowkey_exprs_, seed);
}
seed = ObLogicalOperator::hash(seed);
return seed;
}
int ObLogMerge::add_all_source_table_columns_to_ctx(ObAllocExprContext& ctx)
int ObLogMerge::assign_dml_infos(const ObIArray<IndexDMLInfo *> &index_insert_infos,
const ObIArray<IndexDMLInfo *> &index_update_infos,
const ObIArray<IndexDMLInfo *> &index_delete_infos)
{
int ret = OB_SUCCESS;
CK(OB_NOT_NULL(table_columns_), !table_columns_->empty());
OZ(add_exprs_to_ctx(ctx, *table_columns_));
return ret;
}
int ObLogMerge::inner_append_not_produced_exprs(ObRawExprUniqueSet& raw_exprs) const
{
int ret = OB_SUCCESS;
OZ(ObLogDelUpd::inner_append_not_produced_exprs(raw_exprs));
uint64_t loc_table_id;
if (!index_update_infos.empty()) {
loc_table_id = index_update_infos.at(0)->loc_table_id_;
} else if (OB_UNLIKELY(index_insert_infos.empty())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("insert dml info is null", K(ret), K(index_update_infos), K(index_insert_infos));
} else {
loc_table_id = index_insert_infos.at(0)->loc_table_id_;
}
if (OB_SUCC(ret)) {
if (nullptr != match_condition_exprs_ && OB_FAIL(raw_exprs.append(*match_condition_exprs_))) {
LOG_WARN("append_array_no_dup failed", K(ret));
}
if (nullptr != insert_condition_exprs_ && OB_FAIL(raw_exprs.append(*insert_condition_exprs_))) {
LOG_WARN("append_array_no_dup failed", K(ret));
}
if (nullptr != update_condition_exprs_ && OB_FAIL(raw_exprs.append(*update_condition_exprs_))) {
LOG_WARN("append_array_no_dup failed", K(ret));
}
if (nullptr != delete_condition_exprs_ && OB_FAIL(raw_exprs.append(*delete_condition_exprs_))) {
LOG_WARN("append_array_no_dup failed", K(ret));
}
if (nullptr != rowkey_exprs_ && OB_FAIL(raw_exprs.append(*rowkey_exprs_))) {
LOG_WARN("append_array_no_dup failed", K(ret));
if (OB_FAIL(loc_table_list_.push_back(loc_table_id))) {
LOG_WARN("failed to push back loc table id", K(ret));
} else if (OB_FAIL(index_dml_infos_.assign(index_insert_infos))) {
LOG_WARN("failed to assgin index insert infos", K(ret));
} else if (OB_FAIL(index_upd_infos_.assign(index_update_infos))) {
LOG_WARN("failed to assign update index infos", K(ret));
} else if (OB_FAIL(index_del_infos_.assign(index_delete_infos))) {
LOG_WARN("failed to assign delete index infos", K(ret));
}
}
OZ(ObLogInsert::inner_append_not_produced_exprs(raw_exprs));
return ret;
}
const char* ObLogMerge::get_name() const
const char *ObLogMerge::get_name() const
{
const char* ret = "NOT SET";
const char *ret = "NOT SET";
if (is_multi_part_dml()) {
ret = "MULTI PARTITION MERGE";
ret = "DISTRIBUTED MERGE";
} else {
ret = "MERGE";
}
LOG_DEBUG("merge_op get name", K(is_multi_part_dml()));
return ret;
}
const common::ObIArray<ObRawExpr *>& ObLogMerge::get_insert_condition() const
{
return static_cast<const ObMergeLogPlan &>(my_dml_plan_).get_insert_condition();
}
const common::ObIArray<ObRawExpr *>& ObLogMerge::get_update_condition() const
{
return static_cast<const ObMergeLogPlan &>(my_dml_plan_).get_update_condition();
}
const common::ObIArray<ObRawExpr *>& ObLogMerge::get_delete_condition() const
{
return static_cast<const ObMergeLogPlan &>(my_dml_plan_).get_delete_condition();
}
int ObLogMerge::generate_rowid_expr_for_trigger()
{
int ret = OB_SUCCESS;
bool has_trg = false;
const ObMergeStmt *merge_stmt = NULL;
uint64_t loc_table_id = OB_INVALID_ID;
uint64_t ref_table_id = OB_INVALID_ID;
IndexDMLInfo *insert_info =
index_dml_infos_.empty() ? NULL : index_dml_infos_.at(0);
IndexDMLInfo *delete_info =
index_del_infos_.empty() ? NULL : index_del_infos_.at(0);
IndexDMLInfo *update_info =
index_upd_infos_.empty() ? NULL : index_upd_infos_.at(0);
if (NULL != update_info) {
ref_table_id = update_info->ref_table_id_;
loc_table_id = update_info->loc_table_id_;
} else if (NULL != insert_info) {
ref_table_id = insert_info->ref_table_id_;
loc_table_id = insert_info->loc_table_id_;
}
if (OB_ISNULL(merge_stmt = static_cast<const ObMergeStmt*>(get_stmt())) ||
OB_UNLIKELY(OB_INVALID_ID == loc_table_id) ||
OB_UNLIKELY(OB_INVALID_ID == ref_table_id)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("table id is invalid", K(ret), K(loc_table_id), K(ref_table_id));
} else if (has_instead_of_trigger()) {
// do nothing
} else if (OB_FAIL(check_has_trigger(ref_table_id, has_trg))) {
LOG_WARN("failed to check has trg", K(ret));
}
// for insert clause
if (OB_SUCC(ret) && merge_stmt->has_insert_clause() && has_trg) {
if (OB_ISNULL(insert_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("insert info is null", K(ret), K(insert_info));
} else if (OB_FAIL(generate_old_rowid_expr(*insert_info))) {
LOG_WARN("failed to generate rowid expr", K(ret));
} else if (OB_FAIL(generate_insert_new_rowid_expr(*insert_info))) {
LOG_WARN("failed to generate new rowid", K(ret));
}
}
// for update clause
if (OB_SUCC(ret) && merge_stmt->has_update_clause() && has_trg) {
if (OB_ISNULL(update_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("update info is null", K(ret));
} else if (OB_FAIL(generate_old_rowid_expr(*update_info))) {
LOG_WARN("failed to generate rowid expr", K(ret));
} else if (OB_FAIL(generate_update_new_rowid_expr(*update_info))) {
LOG_WARN("failed to generate update rowid expr", K(ret));
} else if (NULL != delete_info) {
delete_info->old_rowid_expr_ = update_info->new_rowid_expr_;
delete_info->new_rowid_expr_ = NULL;
}
}
return ret;
}
int ObLogMerge::generate_multi_part_partition_id_expr()
{
int ret = OB_SUCCESS;
const ObMergeStmt *merge_stmt = static_cast<const ObMergeStmt *>(get_stmt());
if (OB_ISNULL(merge_stmt)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("merge stmt is null", K(ret), K(merge_stmt));
} else if (merge_stmt->has_insert_clause()) {
for (int64_t i = 0; OB_SUCC(ret) && i < get_index_dml_infos().count(); ++i) {
IndexDMLInfo *ins_info = get_index_dml_infos().at(i);
if (OB_ISNULL(ins_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("insert info is null", K(ret), K(ins_info));
} else if (OB_FAIL(generate_old_calc_partid_expr(*ins_info))) {
LOG_WARN("failed to generate calc partition expr", K(ret));
} else if (OB_FAIL(generate_insert_new_calc_partid_expr(*ins_info))) {
LOG_WARN("failed to genearte calc partition expr for insert clause", K(ret));
}
}
}
for (int64_t i = 0; OB_SUCC(ret) && i < get_update_infos().count(); ++i) {
IndexDMLInfo *upd_info = get_update_infos().at(i);
if (OB_ISNULL(upd_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("update info is null", K(ret), K(upd_info));
} else if (OB_FAIL(generate_old_calc_partid_expr(*upd_info))) {
LOG_WARN("failed to generate calc partition expr for update", K(ret));
} else if (OB_FAIL(generate_update_new_calc_partid_expr(*upd_info))) {
LOG_WARN("failed to generate new calc partition id expr", K(ret));
}
}
// Reminder, the calc partition id expr is mocked for the delete clause !
// You may find that it is exactly the same as the old_part_id_expr of the update_dml_info.
// But they are calcuated in the different way by the engine.
// The update_dml_info's old_part_id_expr is calucalted naturally (based on the output of table scan)
// The delete_dml_info's old_part_id_expr is not calucalted with the output of the table scan
// Its inputs (those column exprs) are created (or mocked) by the ObTableMergeOp !
// Hence, the delete old_part_id_expr is different with the update one.
// We must generate the expr individually !
for (int64_t i = 0; OB_SUCC(ret) && i < get_delete_infos().count(); ++i) {
IndexDMLInfo *del_info = get_delete_infos().at(i);
if (OB_ISNULL(del_info)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("delete info is null", K(ret), K(del_info));
} else if (OB_FAIL(generate_old_calc_partid_expr(*del_info))) {
LOG_WARN("failed to generate calc partition expr for delete", K(ret));
}
}
return ret;
}
bool ObLogMerge::is_insert_dml_info(const IndexDMLInfo *dml_info) const
{
return ObOptimizerUtil::find_item(get_index_dml_infos(), dml_info);
}
bool ObLogMerge::is_delete_dml_info(const IndexDMLInfo *dml_info) const
{
return ObOptimizerUtil::find_item(get_delete_infos(), dml_info);
}
int ObLogMerge::gen_location_constraint(void *ctx)
{
int ret = OB_SUCCESS;
// constraints for merge partition pruning
ObQueryCtx *query_ctx = NULL;
if (OB_ISNULL(get_stmt()) || OB_ISNULL(query_ctx = get_stmt()->get_query_ctx())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null", K(ret), K(get_stmt()), K(query_ctx));
} else if (OB_FAIL(append(query_ctx->all_equal_param_constraints_, get_equal_infos()))) {
LOG_WARN("append equal param info failed", K(ret));
} else if (OB_FAIL(ObLogicalOperator::gen_location_constraint(ctx))) {
LOG_WARN("failed to gen location constraint", K(ret));
} else { /* do nothing */ }
return ret;
}