[FEAT MERGE] 4.2 add trans_debug_info for 4377

Co-authored-by: Handora <qcdsr970209@gmail.com>
This commit is contained in:
yishenglanlingzui
2023-05-06 06:08:35 +00:00
committed by ob-robot
parent 711f4d5cd0
commit 9e328378c8
63 changed files with 1013 additions and 169 deletions

View File

@ -811,7 +811,7 @@ int ObDelUpdLogPlan::compute_hash_dist_exprs_for_pdml_insert(ObExchangeInfo &exc
}
int ObDelUpdLogPlan::replace_assignment_expr_from_dml_info(const IndexDMLInfo &dml_info,
ObRawExpr* &expr)
ObRawExpr *&expr)
{
int ret = OB_SUCCESS;
for (int64_t j = 0; OB_SUCC(ret) && j < dml_info.assignments_.count(); ++j) {
@ -1751,6 +1751,35 @@ int ObDelUpdLogPlan::prepare_table_dml_info_basic(const ObDmlTableInfo& table_in
} else {
table_dml_info->rowkey_cnt_ = index_schema->get_rowkey_column_num();
table_dml_info->is_primary_index_ = true;
ObExecContext *exec_ctx = get_optimizer_context().get_exec_ctx();
if (OB_ISNULL(exec_ctx)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("exec_cts is null", K(ret));
} else if (exec_ctx->get_sql_ctx()->is_strict_defensive_check_ &&
!(optimizer_context_.get_session_info()->is_inner()) &&
(stmt_->is_update_stmt() || stmt_->is_delete_stmt()) &&
GET_MIN_CLUSTER_VERSION() >= CLUSTER_VERSION_4_2_0_0) {
// 1: Is strict defensive check mode
// 2: Not inner_sql
// 3: Now only support delete and update statement
// 4: disable it when upgrade
// Only when the three conditions are met can the defensive_check information be added
TableItem *table_item = nullptr;
ObOpPseudoColumnRawExpr *trans_info_expr = nullptr;
if (OB_ISNULL(table_item = stmt_->get_table_item_by_id(table_info.table_id_))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null", K(ret), K(table_info.table_id_), KPC(stmt_));
} else if (OB_FAIL(ObOptimizerUtil::generate_pseudo_trans_info_expr(get_optimizer_context(),
table_item->get_table_name(),
trans_info_expr))) {
LOG_WARN("fail to generate pseudo trans info expr", K(ret), K(table_item->get_table_name()));
} else if (OB_ISNULL(trans_info_expr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("null pointer", K(ret));
} else {
table_dml_info->trans_info_expr_ = trans_info_expr;
}
}
}
}
if (OB_SUCC(ret) && !has_tg) {
@ -1783,6 +1812,8 @@ int ObDelUpdLogPlan::prepare_table_dml_info_basic(const ObDmlTableInfo& table_in
index_dml_info->ref_table_id_ = index_tid[i];
index_dml_info->rowkey_cnt_ = index_schema->get_rowkey_column_num();
index_dml_info->spk_cnt_ = index_schema->get_shadow_rowkey_column_num();
// Trans_info_expr_ on the main table is recorded in all index_dml_info
index_dml_info->trans_info_expr_ = table_dml_info->trans_info_expr_;
ObSchemaObjVersion table_version;
table_version.object_id_ = index_tid[i];
table_version.object_type_ = DEPENDENCY_TABLE;

View File

@ -87,7 +87,7 @@ public:
ObIArray<ObRawExpr*> &rowkey_exprs);
int replace_assignment_expr_from_dml_info(const IndexDMLInfo &index_dml_info,
ObRawExpr* &expr);
ObRawExpr *&expr);
int candi_allocate_one_pdml_delete(bool is_index_maintenance,
bool is_last_dml_op,

View File

@ -56,6 +56,10 @@ int IndexDMLInfo::deep_copy(ObIRawExprCopier &expr_copier, const IndexDMLInfo &o
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));
} else if (OB_NOT_NULL(other.trans_info_expr_)) {
if (OB_FAIL(expr_copier.copy(other.trans_info_expr_, trans_info_expr_))) {
LOG_WARN("failed to trans info exprs", K(ret), KPC(other.trans_info_expr_));
}
}
for (int64_t i = 0; OB_SUCC(ret) && i < other.assignments_.count(); ++i) {
if (OB_FAIL(assignments_.at(i).deep_copy(expr_copier,
@ -79,6 +83,7 @@ int IndexDMLInfo::assign_basic(const IndexDMLInfo &other)
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_;
trans_info_expr_ = other.trans_info_expr_;
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_))) {
@ -286,7 +291,8 @@ ObLogDelUpd::ObLogDelUpd(ObDelUpdLogPlan &plan)
pdml_is_returning_(false),
err_log_define_(),
need_alloc_part_id_expr_(false),
has_instead_of_trigger_(false)
has_instead_of_trigger_(false),
produced_trans_exprs_()
{
}
@ -550,6 +556,80 @@ int ObLogDelUpd::find_pdml_part_id_producer(ObLogicalOperator &op,
return ret;
}
int ObLogDelUpd::find_trans_info_producer() {
int ret = OB_SUCCESS;
for (int64_t i = 0 ; OB_SUCC(ret) && i < index_dml_infos_.count(); i++) {
ObLogicalOperator *producer = NULL;
IndexDMLInfo *index_dml_info = index_dml_infos_.at(i);
if (OB_ISNULL(index_dml_info)) {
ret = OB_ERR_UNEXPECTED;
} else if ((!is_pdml() && !index_dml_info->is_primary_index_)) {
// Don't worry about non-pdml and non-main tables
// Every operator in pdml needs to try to press down once
} else if (OB_ISNULL(index_dml_info->trans_info_expr_)) {
// do nothing
} else if (OB_FAIL(find_trans_info_producer(*this, index_dml_info->table_id_, producer))) {
LOG_WARN("fail to find trans info producer", K(ret), KPC(index_dml_info), K(get_name()));
} else if (NULL == producer) {
// No error can be reported here,
// the producer of the corresponding trans_info expression was not found, ignore these
LOG_TRACE("can not found trans debug info expr producer", K(ret), K(index_dml_info->table_id_));
} else if (OB_FAIL(add_var_to_array_no_dup(produced_trans_exprs_,
index_dml_info->trans_info_expr_))) {
LOG_WARN("fail to push trans_info_expr_", K(ret));
} else {
if (producer->get_type() == log_op_def::LOG_TABLE_SCAN) {
if (static_cast<ObLogTableScan *>(producer)->get_trans_info_expr() == index_dml_info->trans_info_expr_) {
LOG_DEBUG("this expr has find the producer", K(ret));
} else {
static_cast<ObLogTableScan *>(producer)->
set_trans_info_expr(static_cast<ObOpPseudoColumnRawExpr *>(index_dml_info->trans_info_expr_));
}
} else {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected type of pdml partition id producer", K(ret), K(producer));
}
}
}
return ret;
}
int ObLogDelUpd::find_trans_info_producer(ObLogicalOperator &op,
const uint64_t tid,
ObLogicalOperator *&producer)
{
int ret = OB_SUCCESS;
producer = NULL;
if (op.get_type() == log_op_def::LOG_TABLE_SCAN) {
ObLogTableScan &tsc = static_cast<ObLogTableScan &>(op);
if (tid == tsc.get_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<ObLogJoin&>(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(SMART_CALL(find_trans_info_producer(*op.get_child(i), tid, producer)))) {
LOG_WARN("find pdml part id producer failed", K(ret));
}
} else if (OB_FAIL(SMART_CALL(find_trans_info_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<ObRawExpr*> &all_exprs, bool need_column_expr)
{
int ret = OB_SUCCESS;
@ -558,6 +638,8 @@ int ObLogDelUpd::inner_get_op_exprs(ObIArray<ObRawExpr*> &all_exprs, bool need_c
} 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(find_trans_info_producer())) {
LOG_WARN("failed to find trasn info producer", 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_))) {
@ -566,6 +648,8 @@ int ObLogDelUpd::inner_get_op_exprs(ObIArray<ObRawExpr*> &all_exprs, bool need_c
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(append_array_no_dup(all_exprs, produced_trans_exprs_))) {
LOG_WARN("failed to push back exprs", K(ret), K(produced_trans_exprs_));
} 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))) {
@ -1327,8 +1411,10 @@ int ObLogDelUpd::inner_replace_op_exprs(
int ret = OB_SUCCESS;
if (OB_FAIL(replace_dml_info_exprs(to_replace_exprs, get_index_dml_infos()))) {
LOG_WARN("failed to replace dml info exprs", K(ret));
} else if(OB_FAIL(replace_exprs_action(to_replace_exprs, view_check_exprs_))) {
} else if (OB_FAIL(replace_exprs_action(to_replace_exprs, view_check_exprs_))) {
LOG_WARN("failed to replace view check exprs", K(ret));
} else if (OB_FAIL(replace_exprs_action(to_replace_exprs, produced_trans_exprs_))) {
LOG_WARN("failed to replace produced trans exprs", K(ret));
} else if (NULL != pdml_partition_id_expr_ &&
OB_FAIL(replace_expr_action(to_replace_exprs, pdml_partition_id_expr_))) {
LOG_WARN("failed to replace pdml partition id expr", K(ret));

View File

@ -42,6 +42,7 @@ public:
new_part_id_expr_(NULL),
old_rowid_expr_(NULL),
new_rowid_expr_(NULL),
trans_info_expr_(NULL),
related_index_ids_()
{
}
@ -69,6 +70,7 @@ public:
new_part_id_expr_ = NULL,
old_rowid_expr_ = NULL,
new_rowid_expr_ = NULL,
trans_info_expr_ = NULL,
related_index_ids_.reset();
}
int64_t to_explain_string(char *buf, int64_t buf_len, ExplainType type) const;
@ -157,6 +159,9 @@ public:
ObRawExpr *new_part_id_expr_;
ObRawExpr *old_rowid_expr_;
ObRawExpr *new_rowid_expr_;
// When the defensive check level is set to 2,
// the transaction information of the current row is recorded for 4377 diagnosis
ObRawExpr *trans_info_expr_;
// for generated column, the diff between column_exprs_ and column_old_values_exprs_
// is virtual generated column is replaced.
common::ObSEArray<ObRawExpr*, 64, common::ModulePageAllocator, true> column_old_values_exprs_;
@ -178,7 +183,9 @@ public:
K_(ck_cst_exprs),
K_(is_update_unique_key),
K_(is_update_part_key),
K_(assignments));
K_(assignments),
K_(distinct_algo),
K_(related_index_ids));
};
class ObDelUpdLogPlan;
@ -225,6 +232,17 @@ public:
{
return view_check_exprs_;
}
inline const common::ObIArray<ObRawExpr*> &get_produced_trans_exprs() const
{
return produced_trans_exprs_;
}
inline common::ObIArray<ObRawExpr*> &get_produced_trans_exprs()
{
return produced_trans_exprs_;
}
virtual bool is_single_value() const { return false; }
virtual uint64_t get_hash(uint64_t seed) const { return seed; }
virtual uint64_t hash(uint64_t seed) const override;
@ -295,6 +313,10 @@ public:
int get_rowid_version(int64_t &rowid_version);
virtual int get_op_exprs(ObIArray<ObRawExpr*> &all_exprs) override = 0;
int inner_get_op_exprs(ObIArray<ObRawExpr*> &all_exprs, bool need_column_expr);
int find_trans_info_producer();
int find_trans_info_producer(ObLogicalOperator &op,
const uint64_t tid,
ObLogicalOperator *&producer);
int get_table_columns_exprs(const ObIArray<IndexDMLInfo *> &index_dml_infos,
ObIArray<ObRawExpr*> &all_exprs,
bool need_column_expr);
@ -397,6 +419,11 @@ protected:
// 但是对于非分区表,pdml中的dml是需要分配partition id expr
bool need_alloc_part_id_expr_; // pdml计划中,用于判断当前dml 算子是否需要分配partition id expr
bool has_instead_of_trigger_;
// Only when trans_info_expr can be pushed down to the corresponding table_scan operator,
// the expression will be added to produced_trans_exprs_
// When trans_info_expr does not find a producer operator,
// the upper layer dml operator cannot consume the expression
common::ObSEArray<ObRawExpr *, 4, common::ModulePageAllocator, true> produced_trans_exprs_;
};
}
}

View File

@ -165,6 +165,10 @@ int ObLogTableScan::get_op_exprs(ObIArray<ObRawExpr*> &all_exprs)
LOG_WARN("failed to push back expr", K(ret));
} else if (NULL != calc_part_id_expr_ && OB_FAIL(all_exprs.push_back(calc_part_id_expr_))) {
LOG_WARN("failed to push back expr", K(ret));
} else if (OB_FAIL(allocate_lookup_trans_info_expr())) {
LOG_WARN("failed to add lookup trans expr", K(ret));
} else if (NULL != trans_info_expr_ && OB_FAIL(all_exprs.push_back(trans_info_expr_))) {
LOG_WARN("failed to push back expr", K(ret));
} else if (OB_FAIL(append(all_exprs, access_exprs_))) {
LOG_WARN("failed to append exprs", K(ret));
} else if (OB_FAIL(append(all_exprs, pushdown_aggr_exprs_))) {
@ -246,6 +250,10 @@ int ObLogTableScan::check_output_dependance(common::ObIArray<ObRawExpr *> &child
} else if (use_batch() && nullptr != group_id_expr_
&& OB_FAIL(add_var_to_array_no_dup(exprs, group_id_expr_))) {
LOG_WARN("failed to push back group id expr", K(ret));
} else if (index_back_ &&
nullptr != trans_info_expr_ &&
OB_FAIL(add_var_to_array_no_dup(exprs, trans_info_expr_))) {
LOG_WARN("fail to add lookup trans info expr", K(ret));
} else if (OB_FAIL(dep_checker.check(exprs))) {
LOG_WARN("failed to check op_exprs", K(ret));
} else {
@ -701,6 +709,38 @@ int ObLogTableScan::get_mbr_column_exprs(const uint64_t table_id,
return ret;
}
int ObLogTableScan::allocate_lookup_trans_info_expr()
{
int ret = OB_SUCCESS;
// Is strict defensive check mode
// Is index_back (contain local lookup and global lookup)
// There is no trans_info_expr on the current table_scan operator
// Satisfy the three conditions, add trans_info_expr for lookup
// The result of Index_scan will contain the transaction information corresponding to each row
// The result of the lookup in the data table will also include the trans_info
// of the current row in the data table, But the trans_info will not be output to the upper operator
ObOptimizerContext *opt_ctx = nullptr;
ObOpPseudoColumnRawExpr *tmp_trans_info_expr = nullptr;
if (OB_ISNULL(get_plan())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null", K(ret));
} else if (OB_ISNULL(opt_ctx = &(get_plan()->get_optimizer_context()))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected null", K(ret));
} else if (index_back_ &&
opt_ctx->is_strict_defensive_check() &&
nullptr == trans_info_expr_) {
if (OB_FAIL(OB_FAIL(ObOptimizerUtil::generate_pseudo_trans_info_expr(*opt_ctx,
index_name_,
tmp_trans_info_expr)))) {
LOG_WARN("fail to generate pseudo trans info expr", K(ret), K(index_name_));
} else {
trans_info_expr_ = tmp_trans_info_expr;
}
}
return ret;
}
int ObLogTableScan::generate_necessary_rowkey_and_partkey_exprs()
{
int ret = OB_SUCCESS;

View File

@ -76,6 +76,7 @@ public:
tablet_id_expr_(NULL),
tablet_id_type_(0),
calc_part_id_expr_(NULL),
trans_info_expr_(NULL),
global_index_back_table_partition_info_(NULL),
has_index_scan_filter_(false),
has_index_lookup_filter_(false),
@ -419,7 +420,9 @@ public:
void set_access_path(AccessPath* path) { access_path_ = path; }
inline const AccessPath* get_access_path() const { return access_path_; }
void set_tablet_id_expr(ObOpPseudoColumnRawExpr *expr) { tablet_id_expr_ = expr; }
void set_trans_info_expr(ObOpPseudoColumnRawExpr *expr) { trans_info_expr_ = expr; }
ObOpPseudoColumnRawExpr *get_tablet_id_expr() const { return tablet_id_expr_; }
ObRawExpr *get_trans_info_expr() const { return trans_info_expr_; }
void set_tablet_id_type(int64_t type) { tablet_id_type_ = type; }
int64_t get_tablet_id_type() const { return tablet_id_type_; }
const common::ObIArray<ObRawExpr*> &get_rowkey_exprs() const { return rowkey_exprs_; }
@ -467,6 +470,7 @@ private: // member functions
int generate_necessary_rowkey_and_partkey_exprs();
int add_mapping_columns_for_vt(ObIArray<ObRawExpr*> &access_exprs);
int get_mbr_column_exprs(const uint64_t table_id, ObIArray<ObRawExpr *> &mbr_exprs);
int allocate_lookup_trans_info_expr();
protected: // memeber variables
// basic info
uint64_t table_id_; //table id or alias table id
@ -564,6 +568,7 @@ protected: // memeber variables
// 0 for tablet id, 1 for logical part id, 2 for logical subpart id
int64_t tablet_id_type_;
ObRawExpr *calc_part_id_expr_;
ObRawExpr *trans_info_expr_;
// begin for global index lookup
ObTablePartitionInfo *global_index_back_table_partition_info_;

View File

@ -254,11 +254,18 @@ ObOptimizerContext(ObSQLSessionInfo *session_info,
inline void set_parallel_rule(PXParallelRule rule) { px_parallel_rule_ = rule; }
const PXParallelRule& get_parallel_rule() const { return px_parallel_rule_; }
inline bool is_batched_multi_stmt() {
bool bret = false;
if (NULL != exec_ctx_ && NULL != exec_ctx_->get_sql_ctx()) {
return exec_ctx_->get_sql_ctx()->multi_stmt_item_.is_batched_multi_stmt();
} else {
return false;
bret = exec_ctx_->get_sql_ctx()->multi_stmt_item_.is_batched_multi_stmt();
}
return bret;
}
inline bool is_strict_defensive_check() {
bool bret = false;
if (NULL != exec_ctx_ && NULL != exec_ctx_->get_sql_ctx()) {
bret = exec_ctx_->get_sql_ctx()->is_strict_defensive_check_;
}
return bret;
}
void disable_batch_rpc() { enable_batch_opt_ = 0; }
bool enable_batch_rpc()

View File

@ -8636,3 +8636,36 @@ int ObOptimizerUtil::truncate_string_for_opt_stats(const ObObj *old_obj,
LOG_TRACE("Succeed to truncate string obj for opt stats", KPC(old_obj), KPC(new_obj), K(is_truncated));
return ret;
}
int ObOptimizerUtil::generate_pseudo_trans_info_expr(ObOptimizerContext &opt_ctx,
const common::ObString &table_name,
ObOpPseudoColumnRawExpr *&expr)
{
int ret = OB_SUCCESS;
ObExprResType res_type;
char *pseudo_name = nullptr;
res_type.set_type(ObVarcharType);
res_type.set_collation_type(CS_TYPE_BINARY);
res_type.set_accuracy(ObAccuracy::MAX_ACCURACY[ObVarcharType]);
const char *name = ".TRANS_DEBUG_INFO";
int64_t buf_len = table_name.length()+ STRLEN(name) + 1;
int64_t pos = 0;
if (OB_ISNULL(pseudo_name =
static_cast<char*>(opt_ctx.get_allocator().alloc(buf_len)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("allocate name buffer failed", K(ret), K(buf_len));
} else if (OB_FAIL(databuff_printf(pseudo_name, buf_len, pos, "%.*s", table_name.length(), table_name.ptr()))) {
LOG_WARN("databuff print column name failed", K(ret));
} else if (OB_FAIL(databuff_printf(pseudo_name, buf_len, pos, "%.*s", static_cast<int32_t>(STRLEN(name)), name))) {
LOG_WARN("databuff print column name failed", K(ret));
} else if (OB_FAIL(ObRawExprUtils::build_op_pseudo_column_expr(opt_ctx.get_expr_factory(),
T_PSEUDO_ROW_TRANS_INFO_COLUMN,
pseudo_name,
res_type,
expr))) {
LOG_WARN("build operator pseudo column failed", K(ret));
} else if (OB_FAIL(expr->formalize(opt_ctx.get_session_info()))) {
LOG_WARN("expr formalize failed", K(ret));
}
return ret;
}

View File

@ -1439,6 +1439,10 @@ public:
ObIAllocator &alloc,
ObObj *&new_obj);
static int generate_pseudo_trans_info_expr(ObOptimizerContext &opt_ctx,
const common::ObString &table_name,
ObOpPseudoColumnRawExpr *&expr);
private:
//disallow construct
ObOptimizerUtil();