[DeadLock.bad_case] fix remote sql retry may deadlock
This commit is contained in:
@ -61,6 +61,7 @@ public:
|
||||
// build a directed dependency relationship to other
|
||||
virtual int block(const ObDependencyResource &) = 0;
|
||||
virtual int block(const BlockCallBack &) = 0;
|
||||
virtual int get_block_list(common::ObIArray<ObDependencyResource> &cur_list) const = 0;
|
||||
// releace block list
|
||||
virtual int replace_block_list(const common::ObIArray<ObDependencyResource> &) = 0;
|
||||
// remove a directed dependency relationship to other
|
||||
|
||||
@ -126,6 +126,8 @@ public:
|
||||
template<typename T>
|
||||
int replace_block_list(const T &src_key,
|
||||
const common::ObIArray<ObDependencyResource> &new_list);
|
||||
template<typename T>
|
||||
int get_block_list(const T &src_key, common::ObIArray<ObDependencyResource> &cur_list);
|
||||
// remove directed dependency relationship between two detector
|
||||
template<typename T1, typename T2>
|
||||
int activate(const T1 &src_key, const T2 &dest_key);
|
||||
@ -450,6 +452,30 @@ int ObDeadLockDetectorMgr::replace_block_list(const T &src_key,
|
||||
return ret;
|
||||
#undef PRINT_WRAPPER
|
||||
}
|
||||
template<typename T>
|
||||
int ObDeadLockDetectorMgr::get_block_list(const T &src_key,
|
||||
common::ObIArray<ObDependencyResource> &cur_list)
|
||||
{
|
||||
CHECK_INIT();
|
||||
CHECK_ENABLED();
|
||||
#define PRINT_WRAPPER KR(ret), K(src_key), K(cur_list)
|
||||
int ret = common::OB_SUCCESS;
|
||||
DetectorRefGuard ref_guard;
|
||||
UserBinaryKey src_user_key;
|
||||
|
||||
if (OB_FAIL(src_user_key.set_user_key(src_key))) {
|
||||
DETECT_LOG(WARN, "src_key serialzation failed", PRINT_WRAPPER);
|
||||
} else if (OB_FAIL(get_detector_(src_user_key, ref_guard))) {
|
||||
DETECT_LOG(WARN, "get_detector failed", PRINT_WRAPPER);
|
||||
} else if (OB_FAIL(ref_guard.get_detector()->get_block_list(cur_list))) {
|
||||
DETECT_LOG(WARN, "get block list failed", PRINT_WRAPPER);
|
||||
} else {
|
||||
// DETECT_LOG(INFO, "replace block list success", PRINT_WRAPPER);
|
||||
}
|
||||
|
||||
return ret;
|
||||
#undef PRINT_WRAPPER
|
||||
}
|
||||
// call for removing directed dependency relationship between two detector(both in local)
|
||||
// thread-safe guaranteed
|
||||
//
|
||||
|
||||
@ -325,6 +325,19 @@ int ObLCLNode::activate_all()
|
||||
#undef PRINT_WRAPPER
|
||||
}
|
||||
|
||||
int ObLCLNode::get_block_list(common::ObIArray<ObDependencyResource> &cur_list) const
|
||||
{
|
||||
#define PRINT_WRAPPER KR(ret), K(*this)
|
||||
int ret = OB_SUCCESS;
|
||||
DETECT_TIME_GUARD(100_ms);
|
||||
LockGuard lock_guard(lock_);
|
||||
if (OB_FAIL(cur_list.assign(block_list_))) {
|
||||
DETECT_LOG_(WARN, "fail to get block list", PRINT_WRAPPER);
|
||||
}
|
||||
return ret;
|
||||
#undef PRINT_WRAPPER
|
||||
}
|
||||
|
||||
int ObLCLNode::replace_block_list(const ObIArray<ObDependencyResource> &new_list)
|
||||
{
|
||||
#define PRINT_WRAPPER KR(ret), K(*this), K(new_list)
|
||||
@ -528,15 +541,6 @@ int ObLCLNode::process_lcl_message(const ObLCLMessage &lcl_msg)
|
||||
}
|
||||
ret = OB_SUCCESS;
|
||||
}
|
||||
|
||||
if (INVALID_VALUE != lclv_snapshot &&
|
||||
public_label_snapshot.is_valid() &&
|
||||
!blocklist_snapshot.empty()) {
|
||||
CLICK();
|
||||
(void)broadcast_(blocklist_snapshot, lclv_snapshot, public_label_snapshot);
|
||||
DETECT_LOG_(TRACE, "receive lcl msg and keep broadcastiing", KR(ret), K(lcl_msg), K(*this));
|
||||
}
|
||||
|
||||
if (detected_flag) {
|
||||
CLICK();
|
||||
ObDeadLockCollectInfoMessage collect_info_message;
|
||||
@ -815,8 +819,7 @@ int ObLCLNode::push_state_to_downstreams_with_lock_()
|
||||
DETECT_LOG_(WARN, "not waiting", K(*this), K_(last_report_waiting_for_period));
|
||||
} else {
|
||||
int64_t detector_id = private_label_.get_id();
|
||||
DETECT_LOG_(INFO, "waiting for",
|
||||
KP(this), K(detector_id), K_(self_key), K(blocklist_snapshot));
|
||||
DETECT_LOG_(INFO, "waiting for", K_(self_key), K(blocklist_snapshot), KPC(this));
|
||||
}
|
||||
|
||||
if (!parent_list_.empty()) {
|
||||
@ -886,7 +889,7 @@ void ObLCLNode::PushStateTask::runTimerTask()
|
||||
lcl_node_.timeout_ts_ + 10_min :
|
||||
lcl_node_.created_time_ + 1_hour;
|
||||
if (current_ts > warn_threshold_ts) {
|
||||
DETECT_LOG(ERROR, "long lived lcl node, maybe leaked", K(*this));
|
||||
DETECT_LOG(WARN, "long lived lcl node, maybe leaked", K(*this));
|
||||
}
|
||||
if (false == ATOMIC_LOAD(&lcl_node_.is_timer_task_canceled_)) {
|
||||
if (expected_executed_ts > current_ts) {
|
||||
|
||||
@ -109,6 +109,7 @@ public:
|
||||
// build a directed dependency relationship to other
|
||||
int block(const ObDependencyResource &) override;
|
||||
int block(const BlockCallBack &) override;
|
||||
int get_block_list(common::ObIArray<ObDependencyResource> &cur_list) const override;
|
||||
// releace block list
|
||||
int replace_block_list(const common::ObIArray<ObDependencyResource> &) override;
|
||||
// remove a directed dependency relationship to other
|
||||
@ -119,9 +120,9 @@ public:
|
||||
int process_collect_info_message(const ObDeadLockCollectInfoMessage &) override;
|
||||
// handle message for scheme LCL
|
||||
int process_lcl_message(const ObLCLMessage &) override;
|
||||
TO_STRING_KV(KP(this), K_(self_key), K_(parent_key), K_(timeout_ts), K_(lclv), K_(private_label),
|
||||
TO_STRING_KV(KP(this), K_(self_key), K_(parent_key), KTIME_(timeout_ts), K_(lclv), K_(private_label),
|
||||
K_(public_label), K_(detect_callback),
|
||||
K_(auto_activate_when_detected), K_(created_time), K_(allow_detect_time),
|
||||
K_(auto_activate_when_detected), KTIME_(created_time), KTIME_(allow_detect_time),
|
||||
K_(is_timer_task_canceled), K_(block_list), K_(parent_list),
|
||||
K_(lcl_period), K_(last_send_collect_info_period), K(block_callback_list_.count()))
|
||||
private:
|
||||
|
||||
@ -124,7 +124,8 @@ ObExecContext::ObExecContext(ObIAllocator &allocator)
|
||||
is_ps_prepare_stage_(false),
|
||||
register_op_id_(OB_INVALID_ID),
|
||||
tmp_alloc_used_(false),
|
||||
table_direct_insert_ctx_()
|
||||
table_direct_insert_ctx_(),
|
||||
errcode_(OB_SUCCESS)
|
||||
{
|
||||
}
|
||||
|
||||
@ -181,6 +182,7 @@ ObExecContext::~ObExecContext()
|
||||
temp_expr_ctx_map_.destroy();
|
||||
}
|
||||
update_columns_ = nullptr;
|
||||
errcode_ = OB_SUCCESS;
|
||||
}
|
||||
|
||||
void ObExecContext::clean_resolve_ctx()
|
||||
|
||||
@ -459,6 +459,8 @@ public:
|
||||
eval_tmp_allocator_.set_attr(attr);
|
||||
}
|
||||
ObTableDirectInsertCtx &get_table_direct_insert_ctx() { return table_direct_insert_ctx_; }
|
||||
void set_errcode(const int errcode) { ATOMIC_STORE(&errcode_, errcode); }
|
||||
int get_errcode() const { return ATOMIC_LOAD(&errcode_); }
|
||||
private:
|
||||
int build_temp_expr_ctx(const ObTempExpr &temp_expr, ObTempExprCtx *&temp_expr_ctx);
|
||||
int set_phy_op_ctx_ptr(uint64_t index, void *phy_op);
|
||||
@ -631,6 +633,8 @@ protected:
|
||||
// -------------------
|
||||
// for direct insert
|
||||
ObTableDirectInsertCtx table_direct_insert_ctx_;
|
||||
// for deadlock detect, set in do_close_plan
|
||||
int errcode_;
|
||||
private:
|
||||
DISALLOW_COPY_AND_ASSIGN(ObExecContext);
|
||||
};
|
||||
|
||||
@ -743,6 +743,7 @@ OB_INLINE int ObResultSet::do_close_plan(int errcode, ObExecContext &ctx)
|
||||
err_ignored = plan_ctx->is_error_ignored();
|
||||
}
|
||||
bool rollback = need_rollback(ret, errcode, err_ignored);
|
||||
get_exec_context().set_errcode(errcode);
|
||||
sret = end_stmt(rollback || OB_SUCCESS != pret);
|
||||
// SQL_LOG(INFO, "end_stmt err code", K_(errcode), K(ret), K(pret), K(sret));
|
||||
// if branch fail is returned from end_stmt, then return it first
|
||||
|
||||
@ -783,6 +783,7 @@ int ObSqlTransControl::end_stmt(ObExecContext &exec_ctx, const bool rollback)
|
||||
OZ (get_tx_service(session, txs), *session);
|
||||
// plain select stmt don't require txn descriptor
|
||||
if (OB_SUCC(ret) && !is_plain_select) {
|
||||
ObTransDeadlockDetectorAdapter::maintain_deadlock_info_when_end_stmt(exec_ctx, rollback);
|
||||
CK (OB_NOT_NULL(tx_desc));
|
||||
auto &tx_result = session->get_trans_result();
|
||||
if (OB_FAIL(ret)) {
|
||||
@ -817,10 +818,6 @@ int ObSqlTransControl::end_stmt(ObExecContext &exec_ctx, const bool rollback)
|
||||
ret = COVER_SUCC(tmp_ret);
|
||||
}
|
||||
|
||||
if (!is_plain_select) {
|
||||
ObTransDeadlockDetectorAdapter::maintain_deadlock_info_when_end_stmt(exec_ctx, rollback);
|
||||
}
|
||||
|
||||
bool print_log = false;
|
||||
#ifndef NDEBUG
|
||||
print_log = true;
|
||||
@ -835,7 +832,8 @@ int ObSqlTransControl::end_stmt(ObExecContext &exec_ctx, const bool rollback)
|
||||
"tx_desc", PC(session->get_tx_desc()),
|
||||
"trans_result", session->get_trans_result(),
|
||||
K(rollback),
|
||||
KPC(session));
|
||||
KPC(session),
|
||||
K(exec_ctx.get_errcode()));
|
||||
}
|
||||
if (OB_NOT_NULL(session)) {
|
||||
session->get_trans_result().reset();
|
||||
|
||||
@ -312,9 +312,9 @@ int ObMemtableCtx::write_lock_yield()
|
||||
void ObMemtableCtx::on_wlock_retry(const ObMemtableKey& key, const transaction::ObTransID &conflict_tx_id)
|
||||
{
|
||||
mtstat_.on_wlock_retry();
|
||||
if (log_conflict_interval_.reach()) {
|
||||
TRANS_LOG_RET(WARN, OB_SUCCESS, "mvcc_write conflict", K(key), "tx_id", get_tx_id(), K(conflict_tx_id), KPC(this));
|
||||
}
|
||||
#define USING_LOG_PREFIX TRANS
|
||||
FLOG_INFO("mvcc_write conflict", K(key), "tx_id", get_tx_id(), K(conflict_tx_id), KPC(this));
|
||||
#undef USING_LOG_PREFIX
|
||||
}
|
||||
|
||||
void ObMemtableCtx::on_tsc_retry(const ObMemtableKey& key,
|
||||
|
||||
@ -353,7 +353,8 @@ int ObTransDeadlockDetectorAdapter::register_remote_execution_to_deadlock_detect
|
||||
} else if (OB_FAIL(MTL(ObDeadLockDetectorMgr*)->register_key(self_tx_id,
|
||||
on_detect_op,
|
||||
on_collect_op,
|
||||
~session_guard->get_tx_desc()->get_active_ts()))) {
|
||||
~session_guard->get_tx_desc()->get_active_ts(),
|
||||
500_ms))) {
|
||||
DETECT_LOG(WARN, "fail to register deadlock", PRINT_WRAPPER);
|
||||
} else {
|
||||
MTL(ObDeadLockDetectorMgr*)->set_timeout(self_tx_id, query_timeout);
|
||||
@ -375,21 +376,42 @@ int ObTransDeadlockDetectorAdapter::register_remote_execution_to_deadlock_detect
|
||||
}
|
||||
|
||||
int ObTransDeadlockDetectorAdapter::remote_execution_replace_conflict_trans_ids_(const ObTransID self_tx_id,
|
||||
const ObIArray<ObTransIDAndAddr> &conflict_tx_ids)
|
||||
const ObIArray<ObTransIDAndAddr> &conflict_tx_ids,
|
||||
SessionGuard &session_guard)
|
||||
{
|
||||
#define PRINT_WRAPPER KR(ret), K(self_tx_id), K(conflict_tx_ids), K(blocked_resources)
|
||||
#define PRINT_WRAPPER KR(ret), K(self_tx_id), K(conflict_tx_ids), K(current_blocked_resources)
|
||||
int ret = OB_SUCCESS;
|
||||
ObSEArray<ObDependencyResource, DEFAULT_BLOCKED_TRANS_ID_COUNT> blocked_resources;
|
||||
ObSEArray<ObDependencyResource, DEFAULT_BLOCKED_TRANS_ID_COUNT> current_blocked_resources;
|
||||
auto check_at_least_one_holder_same = [](ObSEArray<ObDependencyResource, DEFAULT_BLOCKED_TRANS_ID_COUNT> &l,
|
||||
ObSEArray<ObDependencyResource, DEFAULT_BLOCKED_TRANS_ID_COUNT> &r) -> bool {
|
||||
bool has_same_holder = false;
|
||||
for (int64_t idx1 = 0; idx1 < l.count() && !has_same_holder; ++idx1) {
|
||||
for (int64_t idx2 = 0; idx2 < r.count() && !has_same_holder; ++idx2) {
|
||||
if (l[idx1] == r[idx2]) {
|
||||
has_same_holder = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return has_same_holder;
|
||||
};
|
||||
if (OB_UNLIKELY(!conflict_tx_ids.empty())) {
|
||||
if (OB_ISNULL(MTL(ObDeadLockDetectorMgr*))) {
|
||||
ret = OB_ERR_UNEXPECTED;
|
||||
DETECT_LOG(ERROR, "mtl deadlock detector mgr is null", PRINT_WRAPPER);
|
||||
} else if (OB_FAIL(gen_dependency_resource_array_(conflict_tx_ids, blocked_resources))) {
|
||||
DETECT_LOG(ERROR, "generate dependency array failed", PRINT_WRAPPER);
|
||||
} else if (OB_FAIL(MTL(ObDeadLockDetectorMgr*)->replace_block_list(self_tx_id, blocked_resources))) {
|
||||
DETECT_LOG(WARN, "block on resource failed", PRINT_WRAPPER);
|
||||
} else if (OB_FAIL(MTL(ObDeadLockDetectorMgr*)->get_block_list(self_tx_id, current_blocked_resources))) {
|
||||
DETECT_LOG(WARN, "generate dependency array failed", PRINT_WRAPPER);
|
||||
} else if (check_at_least_one_holder_same(current_blocked_resources, blocked_resources)) {
|
||||
if (OB_FAIL(MTL(ObDeadLockDetectorMgr*)->replace_block_list(self_tx_id, blocked_resources))) {
|
||||
DETECT_LOG(WARN, "replace block list failed", PRINT_WRAPPER);
|
||||
}
|
||||
} else {
|
||||
DETECT_LOG(TRACE, "remote execution block on new resource success", PRINT_WRAPPER);
|
||||
try_unregister_deadlock_detector_(session_guard.get_session(),
|
||||
self_tx_id,
|
||||
UnregisterPath::REPLACE_MEET_TOTAL_DIFFERENT_LIST);
|
||||
DETECT_LOG(WARN, "unregister detector cause meet total different block list", PRINT_WRAPPER);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -422,7 +444,7 @@ int ObTransDeadlockDetectorAdapter::register_remote_execution_or_replace_conflic
|
||||
DETECT_LOG(INFO, "register new detector in remote execution", PRINT_WRAPPER);
|
||||
}
|
||||
} else {
|
||||
if (OB_FAIL(remote_execution_replace_conflict_trans_ids_(self_tx_id, conflict_tx_ids))) {
|
||||
if (OB_FAIL(remote_execution_replace_conflict_trans_ids_(self_tx_id, conflict_tx_ids, session_guard))) {
|
||||
DETECT_LOG(INFO, "replace block list in remote execution", PRINT_WRAPPER);
|
||||
}
|
||||
}
|
||||
@ -602,7 +624,8 @@ int ObTransDeadlockDetectorAdapter::create_detector_node_and_set_parent_if_neede
|
||||
int ObTransDeadlockDetectorAdapter::maintain_deadlock_info_when_end_stmt(sql::ObExecContext &exec_ctx,
|
||||
const bool is_rollback)
|
||||
{
|
||||
#define PRINT_WRAPPER KR(ret), K(exec_ctx), K(is_rollback)
|
||||
#define PRINT_WRAPPER KR(ret), KR(exec_ctx.get_errcode()), KPC(session),\
|
||||
KPC(desc), K(is_rollback), K(conflict_txs)
|
||||
int ret = OB_SUCCESS;
|
||||
CHECK_DEADLOCK_ENABLED();
|
||||
memtable::ObLockWaitMgr::Node *node = nullptr;
|
||||
@ -611,36 +634,39 @@ int ObTransDeadlockDetectorAdapter::maintain_deadlock_info_when_end_stmt(sql::Ob
|
||||
ObArray<ObTransIDAndAddr> conflict_txs;
|
||||
if (OB_ISNULL(session = GET_MY_SESSION(exec_ctx))) {
|
||||
ret = OB_BAD_NULL_ERROR;
|
||||
DETECT_LOG(ERROR, "session is NULL", KR(ret));
|
||||
DETECT_LOG(ERROR, "session is NULL", PRINT_WRAPPER);
|
||||
} else if (session->is_inner()) {
|
||||
// inner session no need register to deadlock
|
||||
} else if (memtable::TLOCAL_NEED_WAIT_IN_LOCK_WAIT_MGR) {
|
||||
// will call post_process() in lock_wait_mgr, will register deadlock info there, no need process here
|
||||
} else if (OB_ISNULL(desc = session->get_tx_desc())) {
|
||||
ret = OB_BAD_NULL_ERROR;
|
||||
DETECT_LOG(ERROR, "desc in session is NULL", KR(ret));
|
||||
DETECT_LOG(ERROR, "desc in session is NULL", PRINT_WRAPPER);
|
||||
} else if (!desc->is_valid()) {
|
||||
// no trans opened, for example:read-only trans
|
||||
} else if (is_rollback) {// statment is failed, maybe will try again, check if need register to deadlock detector
|
||||
if (session->get_query_timeout_ts() < ObClockGenerator::getCurrentTime()) {
|
||||
DETECT_LOG(INFO, "query timeout, no need register to deadlock", KR(ret), K(desc->tid()));
|
||||
try_unregister_deadlock_detector_(*session, desc->tid(), UnregisterPath::END_STMT_TIMEOUT);
|
||||
DETECT_LOG(INFO, "query timeout, no need register to deadlock", PRINT_WRAPPER);
|
||||
} else if (OB_FAIL(desc->fetch_conflict_txs(conflict_txs))) {
|
||||
DETECT_LOG(WARN, "fail to get conflict txs from desc", KR(ret), K(desc->tid()));
|
||||
DETECT_LOG(WARN, "fail to get conflict txs from desc", PRINT_WRAPPER);
|
||||
} else if (conflict_txs.empty()) {
|
||||
// no row conflicted, no need register to deadlock
|
||||
try_unregister_deadlock_detector_(*session, desc->tid(), UnregisterPath::END_STMT_NO_CONFLICT);
|
||||
DETECT_LOG(INFO, "try unregister deadlock detecotr cause conflict array is empty", PRINT_WRAPPER);
|
||||
} else if (exec_ctx.get_errcode() != OB_TRY_LOCK_ROW_CONFLICT) {
|
||||
try_unregister_deadlock_detector_(*session, desc->tid(), UnregisterPath::END_STMT_OTHER_ERR);
|
||||
DETECT_LOG(INFO, "try unregister deadlock detecotr cause meet non-lock error", PRINT_WRAPPER);
|
||||
} else if (OB_FAIL(register_remote_execution_or_replace_conflict_trans_ids(desc->tid(),
|
||||
session->get_sessid(),
|
||||
conflict_txs))) {
|
||||
DETECT_LOG(WARN, "register or replace list failed", KR(ret), K(desc->tid()));
|
||||
DETECT_LOG(WARN, "register or replace list failed", PRINT_WRAPPER);
|
||||
} else {
|
||||
desc->reset_conflict_txs();
|
||||
DETECT_LOG(INFO, "maintain deadlock info when end_stmt", PRINT_WRAPPER);
|
||||
}
|
||||
} else {// statment is done, will not try again, all related deadlock info should be resetted
|
||||
if (OB_UNLIKELY(session->is_registered_to_deadlock())) {
|
||||
unregister_from_deadlock_detector(desc->tid(), UnregisterPath::END_STMT);
|
||||
session->set_registered_to_deadlock(false);
|
||||
DETECT_LOG(TRACE, "unregister from deadlock detector", KR(ret), K(desc->tid()));
|
||||
}
|
||||
try_unregister_deadlock_detector_(*session, desc->tid(), UnregisterPath::END_STMT_DONE);
|
||||
DETECT_LOG(TRACE, "unregister from deadlock detector", KR(ret), K(desc->tid()));
|
||||
}
|
||||
return ret;
|
||||
#undef PRINT_WRAPPER
|
||||
@ -800,5 +826,16 @@ void ObTransDeadlockDetectorAdapter::unregister_from_deadlock_detector(const ObT
|
||||
}
|
||||
}
|
||||
|
||||
void ObTransDeadlockDetectorAdapter::try_unregister_deadlock_detector_(sql::ObSQLSessionInfo &session,
|
||||
const ObTransID &trans_id,
|
||||
UnregisterPath path)
|
||||
{
|
||||
if (OB_UNLIKELY(session.is_registered_to_deadlock())) {
|
||||
unregister_from_deadlock_detector(trans_id, path);
|
||||
session.set_registered_to_deadlock(false);
|
||||
DETECT_LOG(INFO, "unregister from deadlock detector", K(trans_id), K(to_string(path)));
|
||||
}
|
||||
}
|
||||
|
||||
} // namespace transaction
|
||||
} // namespace oceanbase
|
||||
|
||||
@ -72,7 +72,11 @@ class ObTransDeadlockDetectorAdapter
|
||||
LOCK_WAIT_MGR_REPOST,
|
||||
LOCK_WAIT_MGR_WAIT_FAILED,
|
||||
LOCK_WAIT_MGR_TRANSFORM_WAITING_ROW_TO_TX,
|
||||
END_STMT,
|
||||
END_STMT_DONE,
|
||||
END_STMT_OTHER_ERR,
|
||||
END_STMT_NO_CONFLICT,
|
||||
END_STMT_TIMEOUT,
|
||||
REPLACE_MEET_TOTAL_DIFFERENT_LIST
|
||||
};
|
||||
static const char* to_string(const UnregisterPath path)
|
||||
{
|
||||
@ -85,8 +89,16 @@ class ObTransDeadlockDetectorAdapter
|
||||
return "LOCK_WAIT_MGR_WAIT_FAILED";
|
||||
case UnregisterPath::LOCK_WAIT_MGR_TRANSFORM_WAITING_ROW_TO_TX:
|
||||
return "LOCK_WAIT_MGR_TRANSFORM_WAITING_ROW_TO_TX";
|
||||
case UnregisterPath::END_STMT:
|
||||
return "END_STMT";
|
||||
case UnregisterPath::END_STMT_DONE:
|
||||
return "END_STMT_DONE";
|
||||
case UnregisterPath::END_STMT_OTHER_ERR:
|
||||
return "END_STMT_OTHER_ERROR";
|
||||
case UnregisterPath::END_STMT_NO_CONFLICT:
|
||||
return "END_STMT_NO_CONFLICT";
|
||||
case UnregisterPath::END_STMT_TIMEOUT:
|
||||
return "END_STMT_TIMEOUT";
|
||||
case UnregisterPath::REPLACE_MEET_TOTAL_DIFFERENT_LIST:
|
||||
return "REPLACE_MEET_TOTAL_DIFFERENT_LIST";
|
||||
default:
|
||||
return "UNKNOWN";
|
||||
}
|
||||
@ -139,13 +151,15 @@ private:
|
||||
const ObIArray<ObTransIDAndAddr> &conflict_tx_ids,
|
||||
SessionGuard &session_guard);
|
||||
static int remote_execution_replace_conflict_trans_ids_(const ObTransID self_tx_id,
|
||||
const ObIArray<ObTransIDAndAddr> &conflict_tx_ids);
|
||||
const ObIArray<ObTransIDAndAddr> &conflict_tx_ids,
|
||||
SessionGuard &session_guard);
|
||||
static int create_detector_node_and_set_parent_if_needed_(CollectCallBack &on_collect_op,
|
||||
const ObTransID &self_trans_id,
|
||||
const uint32_t sess_id);
|
||||
static int get_session_related_info_(const uint32_t sess_id, int64_t &query_timeout);
|
||||
static int gen_dependency_resource_array_(const ObIArray<ObTransIDAndAddr> &blocked_trans_ids_and_addrs,
|
||||
ObIArray<share::detector::ObDependencyResource> &dependency_resources);
|
||||
static void try_unregister_deadlock_detector_(sql::ObSQLSessionInfo &session, const ObTransID &trans_id, UnregisterPath path);
|
||||
};
|
||||
|
||||
} // namespace transaction
|
||||
|
||||
Reference in New Issue
Block a user