patch 4.0
This commit is contained in:
@ -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;
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user