[DeadLock] remove thread local execution flag to enabel register deadlock in PL
This commit is contained in:
20
deps/oblib/src/common/ob_clock_generator.h
vendored
20
deps/oblib/src/common/ob_clock_generator.h
vendored
@ -44,7 +44,6 @@ public:
|
|||||||
static void destroy();
|
static void destroy();
|
||||||
static int64_t getClock();
|
static int64_t getClock();
|
||||||
static int64_t getRealClock();
|
static int64_t getRealClock();
|
||||||
static int64_t getCurrentTime();
|
|
||||||
static void msleep(const int64_t ms);
|
static void msleep(const int64_t ms);
|
||||||
static void usleep(const int64_t us);
|
static void usleep(const int64_t us);
|
||||||
static void try_advance_cur_ts(const int64_t cur_ts);
|
static void try_advance_cur_ts(const int64_t cur_ts);
|
||||||
@ -81,25 +80,6 @@ inline int64_t ObClockGenerator::getRealClock()
|
|||||||
return clock_generator_.get_us();
|
return clock_generator_.get_us();
|
||||||
}
|
}
|
||||||
|
|
||||||
inline int64_t ObClockGenerator::getCurrentTime()
|
|
||||||
{
|
|
||||||
int64_t ret_val = ATOMIC_LOAD(&clock_generator_.last_used_time_);
|
|
||||||
while (true) {
|
|
||||||
const int64_t last = ATOMIC_LOAD(&clock_generator_.last_used_time_);
|
|
||||||
const int64_t now = ObTimeUtility::current_time();
|
|
||||||
if (now < last) {
|
|
||||||
ret_val = last;
|
|
||||||
break;
|
|
||||||
} else if (ATOMIC_BCAS(&clock_generator_.last_used_time_, last, now)) {
|
|
||||||
ret_val = now;
|
|
||||||
break;
|
|
||||||
} else {
|
|
||||||
PAUSE();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return ret_val;
|
|
||||||
}
|
|
||||||
|
|
||||||
inline void ObClockGenerator::msleep(const int64_t ms)
|
inline void ObClockGenerator::msleep(const int64_t ms)
|
||||||
{
|
{
|
||||||
if (ms > 0) {
|
if (ms > 0) {
|
||||||
|
@ -144,7 +144,7 @@ struct ObLightSharedPtr// RAII used
|
|||||||
if (!is_valid()) {
|
if (!is_valid()) {
|
||||||
ret = OB_INVALID_DATA;
|
ret = OB_INVALID_DATA;
|
||||||
} else {
|
} else {
|
||||||
int64_t start_sync_time = ObClockGenerator::getCurrentTime();
|
int64_t start_sync_time = ObClockGenerator::getClock();
|
||||||
int64_t loop_times = 0;
|
int64_t loop_times = 0;
|
||||||
int64_t ref_cnt;
|
int64_t ref_cnt;
|
||||||
while (1 != (ref_cnt = ctrl_ptr_->get_ref())) {
|
while (1 != (ref_cnt = ctrl_ptr_->get_ref())) {
|
||||||
|
@ -132,8 +132,8 @@ int ElectionAcceptor::start()
|
|||||||
|
|
||||||
LockGuard lock_guard(p_election_->lock_);
|
LockGuard lock_guard(p_election_->lock_);
|
||||||
// 周期性打印选举的状态
|
// 周期性打印选举的状态
|
||||||
if (ObClockGenerator::getCurrentTime() > last_dump_acceptor_info_ts_ + 3_s) {
|
if (ObClockGenerator::getClock() > last_dump_acceptor_info_ts_ + 3_s) {
|
||||||
last_dump_acceptor_info_ts_ = ObClockGenerator::getCurrentTime();
|
last_dump_acceptor_info_ts_ = ObClockGenerator::getClock();
|
||||||
ELECT_LOG(INFO, "dump acceptor info", K(*this));
|
ELECT_LOG(INFO, "dump acceptor info", K(*this));
|
||||||
}
|
}
|
||||||
// 当acceptor的Lease有效状态发生变化时需要打印日志以及汇报事件
|
// 当acceptor的Lease有效状态发生变化时需要打印日志以及汇报事件
|
||||||
@ -163,7 +163,7 @@ int ElectionAcceptor::start()
|
|||||||
if (last_record_lease_valid_state && !lease_valid_state) {// 这个定时任务可能是被延迟致lease到期时触发的,为了在lease到期的第一时间投票
|
if (last_record_lease_valid_state && !lease_valid_state) {// 这个定时任务可能是被延迟致lease到期时触发的,为了在lease到期的第一时间投票
|
||||||
can_vote = true;
|
can_vote = true;
|
||||||
LOG_ELECT_LEADER(INFO, "vote when lease expired");
|
LOG_ELECT_LEADER(INFO, "vote when lease expired");
|
||||||
} else if (ObClockGenerator::getCurrentTime() - last_time_window_open_ts_ >= CALCULATE_TIME_WINDOW_SPAN_TS()) {
|
} else if (ObClockGenerator::getClock() - last_time_window_open_ts_ >= CALCULATE_TIME_WINDOW_SPAN_TS()) {
|
||||||
can_vote = true;
|
can_vote = true;
|
||||||
} else {
|
} else {
|
||||||
LOG_ELECT_LEADER(INFO, "can't vote now", K(last_record_lease_valid_state),
|
LOG_ELECT_LEADER(INFO, "can't vote now", K(last_record_lease_valid_state),
|
||||||
@ -269,7 +269,7 @@ void ElectionAcceptor::on_prepare_request(const ElectionPrepareRequestMsg &prepa
|
|||||||
LOG_PHASE(ERROR, phase, "open time window failed");
|
LOG_PHASE(ERROR, phase, "open time window failed");
|
||||||
} else {
|
} else {
|
||||||
is_time_window_opened_ = true;// 定时任务注册成功,打开时间窗口
|
is_time_window_opened_ = true;// 定时任务注册成功,打开时间窗口
|
||||||
last_time_window_open_ts_ = ObClockGenerator::getCurrentTime();
|
last_time_window_open_ts_ = ObClockGenerator::getClock();
|
||||||
LOG_PHASE(INFO, phase, "open time window success", K(timewindow_span));
|
LOG_PHASE(INFO, phase, "open time window success", K(timewindow_span));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -223,13 +223,13 @@ int ElectionProposer::register_renew_lease_task_()
|
|||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
LockGuard lock_guard(p_election_->lock_);
|
LockGuard lock_guard(p_election_->lock_);
|
||||||
// 周期性打印选举的状态
|
// 周期性打印选举的状态
|
||||||
if (ObClockGenerator::getCurrentTime() > last_dump_proposer_info_ts_ + 3_s) {
|
if (ObClockGenerator::getClock() > last_dump_proposer_info_ts_ + 3_s) {
|
||||||
last_dump_proposer_info_ts_ = ObClockGenerator::getCurrentTime();
|
last_dump_proposer_info_ts_ = ObClockGenerator::getClock();
|
||||||
ELECT_LOG(INFO, "dump proposer info", K(*this));
|
ELECT_LOG(INFO, "dump proposer info", K(*this));
|
||||||
}
|
}
|
||||||
// 周期性打印选举的消息收发统计信息
|
// 周期性打印选举的消息收发统计信息
|
||||||
if (ObClockGenerator::getCurrentTime() > last_dump_election_msg_count_state_ts_ + 10_s) {
|
if (ObClockGenerator::getClock() > last_dump_election_msg_count_state_ts_ + 10_s) {
|
||||||
last_dump_election_msg_count_state_ts_ = ObClockGenerator::getCurrentTime();
|
last_dump_election_msg_count_state_ts_ = ObClockGenerator::getClock();
|
||||||
char ls_id_buffer[32] = {0};
|
char ls_id_buffer[32] = {0};
|
||||||
auto pretend_to_be_ls_id = [ls_id_buffer](const int64_t id) mutable {
|
auto pretend_to_be_ls_id = [ls_id_buffer](const int64_t id) mutable {
|
||||||
int64_t pos = 0;
|
int64_t pos = 0;
|
||||||
@ -329,7 +329,7 @@ void ElectionProposer::prepare(const ObRole role)
|
|||||||
ELECT_TIME_GUARD(500_ms);
|
ELECT_TIME_GUARD(500_ms);
|
||||||
#define PRINT_WRAPPER KR(ret), K(role), K(*this)
|
#define PRINT_WRAPPER KR(ret), K(role), K(*this)
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
int64_t cur_ts = ObClockGenerator::getCurrentTime();
|
int64_t cur_ts = ObClockGenerator::getClock();
|
||||||
LogPhase phase = role == ObRole::LEADER ? LogPhase::RENEW_LEASE : LogPhase::ELECT_LEADER;
|
LogPhase phase = role == ObRole::LEADER ? LogPhase::RENEW_LEASE : LogPhase::ELECT_LEADER;
|
||||||
if (memberlist_with_states_.get_member_list().get_addr_list().empty()) {
|
if (memberlist_with_states_.get_member_list().get_addr_list().empty()) {
|
||||||
LOG_PHASE(INFO, phase, "memberlist is empty, give up do prepare this time");
|
LOG_PHASE(INFO, phase, "memberlist is empty, give up do prepare this time");
|
||||||
@ -427,7 +427,7 @@ void ElectionProposer::on_prepare_request(const ElectionPrepareRequestMsg &prepa
|
|||||||
.get_addr_list()))) {
|
.get_addr_list()))) {
|
||||||
LOG_ELECT_LEADER(ERROR, "broadcast prepare request failed");
|
LOG_ELECT_LEADER(ERROR, "broadcast prepare request failed");
|
||||||
} else {
|
} else {
|
||||||
last_do_prepare_ts_ = ObClockGenerator::getCurrentTime();
|
last_do_prepare_ts_ = ObClockGenerator::getClock();
|
||||||
if (role_ == ObRole::LEADER) {
|
if (role_ == ObRole::LEADER) {
|
||||||
LOG_ELECT_LEADER(INFO, "join elect leader phase as leader");
|
LOG_ELECT_LEADER(INFO, "join elect leader phase as leader");
|
||||||
} else if (role_ == ObRole::FOLLOWER) {
|
} else if (role_ == ObRole::FOLLOWER) {
|
||||||
|
@ -271,13 +271,13 @@ int64_t MemberListWithStates::to_string(char *buf, const int64_t buf_len) const
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (is_synced && !v.empty()) {
|
if (is_synced && !v.empty()) {
|
||||||
int64_t map_wall_clock_ts = v[0] - get_monotonic_ts() + ObClockGenerator::getCurrentTime();
|
int64_t map_wall_clock_ts = v[0] - get_monotonic_ts() + ObClockGenerator::getClock();
|
||||||
common::databuff_printf(buf, buf_len, pos, "accept_ok_promised_ts:%s, ",
|
common::databuff_printf(buf, buf_len, pos, "accept_ok_promised_ts:%s, ",
|
||||||
v[0] != 0 ? common::ObTime2Str::ob_timestamp_str_range<HOUR, MSECOND>(map_wall_clock_ts) : "invalid");
|
v[0] != 0 ? common::ObTime2Str::ob_timestamp_str_range<HOUR, MSECOND>(map_wall_clock_ts) : "invalid");
|
||||||
} else {
|
} else {
|
||||||
common::databuff_printf(buf, buf_len, pos, "accept_ok_promised_ts:[");
|
common::databuff_printf(buf, buf_len, pos, "accept_ok_promised_ts:[");
|
||||||
for (int64_t idx = 0; idx < v.count(); ++idx) {
|
for (int64_t idx = 0; idx < v.count(); ++idx) {
|
||||||
int64_t map_wall_clock_ts = v[idx] - get_monotonic_ts() + ObClockGenerator::getCurrentTime();
|
int64_t map_wall_clock_ts = v[idx] - get_monotonic_ts() + ObClockGenerator::getClock();
|
||||||
if (idx == v.count() - 1) {
|
if (idx == v.count() - 1) {
|
||||||
common::databuff_printf(buf, buf_len, pos, "%s]", v[idx] != 0 ?
|
common::databuff_printf(buf, buf_len, pos, "%s]", v[idx] != 0 ?
|
||||||
common::ObTime2Str::ob_timestamp_str_range<HOUR, MSECOND>(map_wall_clock_ts) : "invalid");
|
common::ObTime2Str::ob_timestamp_str_range<HOUR, MSECOND>(map_wall_clock_ts) : "invalid");
|
||||||
|
@ -355,8 +355,8 @@ class ElectionMsgCounter
|
|||||||
struct Counter
|
struct Counter
|
||||||
{
|
{
|
||||||
Counter() : send_count_(0), receive_count_(0), last_send_ts_(0), last_received_ts_(0) {}
|
Counter() : send_count_(0), receive_count_(0), last_send_ts_(0), last_received_ts_(0) {}
|
||||||
void add_send_count() { ++send_count_; last_send_ts_ = ObClockGenerator::getCurrentTime(); }
|
void add_send_count() { ++send_count_; last_send_ts_ = ObClockGenerator::getClock(); }
|
||||||
void add_received_count() { ++receive_count_; last_received_ts_ = ObClockGenerator::getCurrentTime(); }
|
void add_received_count() { ++receive_count_; last_received_ts_ = ObClockGenerator::getClock(); }
|
||||||
void reset() { new (this)Counter(); }
|
void reset() { new (this)Counter(); }
|
||||||
int64_t send_count_;
|
int64_t send_count_;
|
||||||
int64_t receive_count_;
|
int64_t receive_count_;
|
||||||
|
@ -38,10 +38,6 @@ using namespace oceanbase::rpc::frame;
|
|||||||
|
|
||||||
namespace oceanbase
|
namespace oceanbase
|
||||||
{
|
{
|
||||||
namespace memtable
|
|
||||||
{
|
|
||||||
extern TLOCAL(bool, TLOCAL_NEED_WAIT_IN_LOCK_WAIT_MGR);
|
|
||||||
}
|
|
||||||
|
|
||||||
namespace omt
|
namespace omt
|
||||||
{
|
{
|
||||||
@ -243,7 +239,6 @@ inline void ObThWorker::process_request(rpc::ObRequest &req)
|
|||||||
reset_sql_throttle_current_priority();
|
reset_sql_throttle_current_priority();
|
||||||
set_req_flag(&req);
|
set_req_flag(&req);
|
||||||
|
|
||||||
memtable::TLOCAL_NEED_WAIT_IN_LOCK_WAIT_MGR = false;
|
|
||||||
MTL(memtable::ObLockWaitMgr*)->setup(req.get_lock_wait_node(), req.get_receive_timestamp());
|
MTL(memtable::ObLockWaitMgr*)->setup(req.get_lock_wait_node(), req.get_receive_timestamp());
|
||||||
if (OB_FAIL(procor_.process(req))) {
|
if (OB_FAIL(procor_.process(req))) {
|
||||||
LOG_WARN("process request fail", K(ret));
|
LOG_WARN("process request fail", K(ret));
|
||||||
|
@ -582,7 +582,7 @@ int ObAllTenantInfoCache::refresh_tenant_info(const uint64_t tenant_id,
|
|||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
ObAllTenantInfo new_tenant_info;
|
ObAllTenantInfo new_tenant_info;
|
||||||
int64_t ora_rowscn = 0;
|
int64_t ora_rowscn = 0;
|
||||||
const int64_t new_refresh_time_us = ObClockGenerator::getCurrentTime();
|
const int64_t new_refresh_time_us = ObClockGenerator::getClock();
|
||||||
content_changed = false;
|
content_changed = false;
|
||||||
if (OB_ISNULL(sql_proxy) || !is_user_tenant(tenant_id)) {
|
if (OB_ISNULL(sql_proxy) || !is_user_tenant(tenant_id)) {
|
||||||
ret = OB_INVALID_ARGUMENT;
|
ret = OB_INVALID_ARGUMENT;
|
||||||
|
@ -123,7 +123,7 @@ int ObLCLNode::register_timer_task()
|
|||||||
CLICK();
|
CLICK();
|
||||||
revert_self_ref_count_();
|
revert_self_ref_count_();
|
||||||
} else {
|
} else {
|
||||||
DETECT_LOG_(INFO, "register first timer task successfully", K(*this));
|
DETECT_LOG_(TRACE, "register first timer task successfully", K(*this));
|
||||||
}
|
}
|
||||||
|
|
||||||
return ret;
|
return ret;
|
||||||
@ -262,7 +262,7 @@ int ObLCLNode::block_(const ObDependencyResource &resource)
|
|||||||
if (OB_FAIL(add_resource_to_list_(resource, block_list_))) {
|
if (OB_FAIL(add_resource_to_list_(resource, block_list_))) {
|
||||||
DETECT_LOG_(WARN, "block_list_ push resource failed", PRINT_WRAPPER);
|
DETECT_LOG_(WARN, "block_list_ push resource failed", PRINT_WRAPPER);
|
||||||
} else {
|
} else {
|
||||||
DETECT_LOG_(INFO, "block resource success", PRINT_WRAPPER);
|
DETECT_LOG_(TRACE, "block resource success", PRINT_WRAPPER);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -288,7 +288,7 @@ int ObLCLNode::block(const BlockCallBack &func)
|
|||||||
K(func), K(*this));
|
K(func), K(*this));
|
||||||
block_callback_list_.pop_back();
|
block_callback_list_.pop_back();
|
||||||
} else {
|
} else {
|
||||||
DETECT_LOG_(INFO, "block callback success", K(*this));
|
DETECT_LOG_(TRACE, "block callback success", K(*this));
|
||||||
}
|
}
|
||||||
|
|
||||||
return ret;
|
return ret;
|
||||||
@ -911,7 +911,7 @@ void ObLCLNode::PushStateTask::runTimerTask()
|
|||||||
}
|
}
|
||||||
if (false == ATOMIC_LOAD(&lcl_node_.is_timer_task_canceled_)) {
|
if (false == ATOMIC_LOAD(&lcl_node_.is_timer_task_canceled_)) {
|
||||||
if (expected_executed_ts > current_ts) {
|
if (expected_executed_ts > current_ts) {
|
||||||
DETECT_LOG(ERROR, "schedule error", K(current_ts), K(expected_executed_ts));
|
DETECT_LOG(WARN, "schedule error", K(current_ts), K(expected_executed_ts));
|
||||||
} else if (current_ts - expected_executed_ts > 100 * 1000) {// 100ms
|
} else if (current_ts - expected_executed_ts > 100 * 1000) {// 100ms
|
||||||
if (REACH_TIME_INTERVAL(100 * 1000)) {// 100ms
|
if (REACH_TIME_INTERVAL(100 * 1000)) {// 100ms
|
||||||
DETECT_LOG(WARN, "task scheduled out of range", K(current_ts), K(expected_executed_ts));
|
DETECT_LOG(WARN, "task scheduled out of range", K(current_ts), K(expected_executed_ts));
|
||||||
|
@ -577,7 +577,7 @@ public:
|
|||||||
ATOMIC_STORE(&is_running_, false);
|
ATOMIC_STORE(&is_running_, false);
|
||||||
int64_t last_print_time = 0;
|
int64_t last_print_time = 0;
|
||||||
while (ATOMIC_LOAD(&total_running_task_count_) != 0) {
|
while (ATOMIC_LOAD(&total_running_task_count_) != 0) {
|
||||||
int64_t current_time = ObClockGenerator::getCurrentTime();
|
int64_t current_time = ObClockGenerator::getClock();
|
||||||
if (current_time - last_print_time > 500_ms) {// print log every 500ms
|
if (current_time - last_print_time > 500_ms) {// print log every 500ms
|
||||||
last_print_time = current_time;
|
last_print_time = current_time;
|
||||||
OCCAM_LOG(INFO, "OccamTimr waiting running task finished",
|
OCCAM_LOG(INFO, "OccamTimr waiting running task finished",
|
||||||
|
@ -58,9 +58,6 @@ using namespace memtable::tablelock;
|
|||||||
|
|
||||||
namespace memtable
|
namespace memtable
|
||||||
{
|
{
|
||||||
// this flag is used to indicate wether if need register to deadlock when end_stmt
|
|
||||||
// CAUTIONS: DO NOT use get_thread_node().neet_wait(), it MAY CORE!
|
|
||||||
TLOCAL(bool, TLOCAL_NEED_WAIT_IN_LOCK_WAIT_MGR) = false;
|
|
||||||
|
|
||||||
static const uint64_t TRANS_FLAG = 1L << 63L; // 10
|
static const uint64_t TRANS_FLAG = 1L << 63L; // 10
|
||||||
static const uint64_t TABLE_LOCK_FLAG = 1L << 62L; // 01
|
static const uint64_t TABLE_LOCK_FLAG = 1L << 62L; // 01
|
||||||
@ -166,7 +163,7 @@ void RowHolderMapper::reset_hash_holder(const ObTabletID &tablet_id,
|
|||||||
};
|
};
|
||||||
if (OB_FAIL(map_.erase_if(ObIntWarp(hash), remove_if_op))) {
|
if (OB_FAIL(map_.erase_if(ObIntWarp(hash), remove_if_op))) {
|
||||||
if (ret != OB_ENTRY_NOT_EXIST) {
|
if (ret != OB_ENTRY_NOT_EXIST) {
|
||||||
TRANS_LOG(WARN, "clear hash holder error", KR(ret), K(hash), K(tx_id));
|
TRANS_LOG(TRACE, "clear hash holder error", KR(ret), K(hash), K(tx_id));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -199,7 +196,7 @@ void ObLockWaitMgr::run1()
|
|||||||
(void)repost(cur);
|
(void)repost(cur);
|
||||||
}
|
}
|
||||||
// dump debug info, and check deadlock enabdle, clear mapper if deadlock is disabled
|
// dump debug info, and check deadlock enabdle, clear mapper if deadlock is disabled
|
||||||
now = ObClockGenerator::getCurrentTime();
|
now = ObClockGenerator::getClock();
|
||||||
if (now - last_dump_ts > 5_s) {
|
if (now - last_dump_ts > 5_s) {
|
||||||
last_dump_ts = now;
|
last_dump_ts = now;
|
||||||
row_holder_mapper_.dump_mapper_info();
|
row_holder_mapper_.dump_mapper_info();
|
||||||
@ -289,22 +286,22 @@ int ObLockWaitMgr::register_to_deadlock_detector_(const ObTransID &self_tx_id,
|
|||||||
self_sess_id)));
|
self_sess_id)));
|
||||||
if (LockHashHelper::is_rowkey_hash(node->hash())) {// waiting for row
|
if (LockHashHelper::is_rowkey_hash(node->hash())) {// waiting for row
|
||||||
DeadLockBlockCallBack deadlock_block_call_back(row_holder_mapper_, node->hash());
|
DeadLockBlockCallBack deadlock_block_call_back(row_holder_mapper_, node->hash());
|
||||||
if (OB_FAIL(ObTransDeadlockDetectorAdapter::register_local_execution_to_deadlock_detector_waiting_for_row(on_collect_callback,
|
if (OB_FAIL(ObTransDeadlockDetectorAdapter::lock_wait_mgr_reconstruct_detector_waiting_for_row(on_collect_callback,
|
||||||
deadlock_block_call_back,
|
deadlock_block_call_back,
|
||||||
self_tx_id,
|
self_tx_id,
|
||||||
self_sess_id))) {
|
self_sess_id))) {
|
||||||
TRANS_LOG(WARN, "fail to regester to deadlock detector", K(ret), K(self_sess_id));
|
TRANS_LOG(WARN, "fail to regester to deadlock detector", K(ret), K(self_sess_id));
|
||||||
} else {
|
} else {
|
||||||
TRANS_LOG(INFO, "wait for row", K(node->hash()), K(self_tx_id), K(blocked_tx_id), K(self_sess_id));
|
TRANS_LOG(TRACE, "wait for row", K(node->hash()), K(self_tx_id), K(blocked_tx_id), K(self_sess_id));
|
||||||
}
|
}
|
||||||
} else {// waiting for other trans
|
} else {// waiting for other trans
|
||||||
if (OB_FAIL(ObTransDeadlockDetectorAdapter::register_local_execution_to_deadlock_detector_waiting_for_trans(on_collect_callback,
|
if (OB_FAIL(ObTransDeadlockDetectorAdapter::lock_wait_mgr_reconstruct_detector_waiting_for_trans(on_collect_callback,
|
||||||
blocked_tx_id,
|
blocked_tx_id,
|
||||||
self_tx_id,
|
self_tx_id,
|
||||||
self_sess_id))) {
|
self_sess_id))) {
|
||||||
TRANS_LOG(WARN, "fail to regester to deadlock detector", K(ret), K(self_sess_id));
|
TRANS_LOG(WARN, "fail to regester to deadlock detector", K(ret), K(self_sess_id));
|
||||||
} else {
|
} else {
|
||||||
TRANS_LOG(INFO, "wait for trans", K(node->hash()), K(self_tx_id), K(blocked_tx_id), K(self_sess_id));
|
TRANS_LOG(TRACE, "wait for trans", K(node->hash()), K(self_tx_id), K(blocked_tx_id), K(self_sess_id));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -577,7 +574,6 @@ int ObLockWaitMgr::post_lock(const int tmp_ret,
|
|||||||
ObFunction<int(bool&, bool&)> &rechecker)
|
ObFunction<int(bool&, bool&)> &rechecker)
|
||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
TLOCAL_NEED_WAIT_IN_LOCK_WAIT_MGR = false;
|
|
||||||
Node *node = NULL;
|
Node *node = NULL;
|
||||||
if (OB_NOT_NULL(node = get_thread_node())) {
|
if (OB_NOT_NULL(node = get_thread_node())) {
|
||||||
Key key(&row_key);
|
Key key(&row_key);
|
||||||
@ -609,7 +605,6 @@ int ObLockWaitMgr::post_lock(const int tmp_ret,
|
|||||||
tx_id,
|
tx_id,
|
||||||
holder_tx_id);
|
holder_tx_id);
|
||||||
node->set_need_wait();
|
node->set_need_wait();
|
||||||
TLOCAL_NEED_WAIT_IN_LOCK_WAIT_MGR = true;// to tell end_stmt() not register deadlock
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -630,7 +625,6 @@ int ObLockWaitMgr::post_lock(const int tmp_ret,
|
|||||||
ObFunction<int(bool&)> &check_need_wait)
|
ObFunction<int(bool&)> &check_need_wait)
|
||||||
{
|
{
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
TLOCAL_NEED_WAIT_IN_LOCK_WAIT_MGR = false;
|
|
||||||
Node *node = NULL;
|
Node *node = NULL;
|
||||||
|
|
||||||
if (OB_UNLIKELY(!is_inited_)) {
|
if (OB_UNLIKELY(!is_inited_)) {
|
||||||
@ -667,7 +661,6 @@ int ObLockWaitMgr::post_lock(const int tmp_ret,
|
|||||||
holder_tx_id);
|
holder_tx_id);
|
||||||
node->set_need_wait();
|
node->set_need_wait();
|
||||||
node->set_lock_mode(lock_mode);
|
node->set_lock_mode(lock_mode);
|
||||||
TLOCAL_NEED_WAIT_IN_LOCK_WAIT_MGR = true;// to tell end_stmt() not register deadlock
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return ret;
|
return ret;
|
||||||
|
@ -68,7 +68,7 @@ int MdsTableBase::init(const ObTabletID tablet_id,
|
|||||||
mgr_handle_.set_mds_table_mgr(p_mgr);
|
mgr_handle_.set_mds_table_mgr(p_mgr);
|
||||||
debug_info_.do_init_tablet_pointer_ = pointer;
|
debug_info_.do_init_tablet_pointer_ = pointer;
|
||||||
debug_info_.init_trace_id_ = *ObCurTraceId::get_trace_id();
|
debug_info_.init_trace_id_ = *ObCurTraceId::get_trace_id();
|
||||||
debug_info_.init_ts_ = ObClockGenerator::getCurrentTime();
|
debug_info_.init_ts_ = ObClockGenerator::getClock();
|
||||||
if (MDS_FAIL(register_to_mds_table_mgr())) {
|
if (MDS_FAIL(register_to_mds_table_mgr())) {
|
||||||
MDS_LOG(WARN, "fail to register mds table", KR(ret), K(*this), K(ls_id), K(tablet_id));
|
MDS_LOG(WARN, "fail to register mds table", KR(ret), K(*this), K(ls_id), K(tablet_id));
|
||||||
}
|
}
|
||||||
@ -102,7 +102,7 @@ void MdsTableBase::mark_removed_from_t3m(ObTabletPointer *pointer)
|
|||||||
} else {
|
} else {
|
||||||
debug_info_.do_remove_tablet_pointer_ = pointer;
|
debug_info_.do_remove_tablet_pointer_ = pointer;
|
||||||
debug_info_.remove_trace_id_ = *ObCurTraceId::get_trace_id();
|
debug_info_.remove_trace_id_ = *ObCurTraceId::get_trace_id();
|
||||||
ATOMIC_STORE(&debug_info_.remove_ts_, ObClockGenerator::getCurrentTime());
|
ATOMIC_STORE(&debug_info_.remove_ts_, ObClockGenerator::getClock());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -111,7 +111,7 @@ void MdsTableBase::mark_switched_to_empty_shell()
|
|||||||
if (ATOMIC_LOAD(&debug_info_.switch_to_empty_shell_ts_) != 0) {
|
if (ATOMIC_LOAD(&debug_info_.switch_to_empty_shell_ts_) != 0) {
|
||||||
MDS_LOG_RET(WARN, OB_ERR_UNEXPECTED, "this MdsTable has been marked switch to empty shell", K(*this));
|
MDS_LOG_RET(WARN, OB_ERR_UNEXPECTED, "this MdsTable has been marked switch to empty shell", K(*this));
|
||||||
} else {
|
} else {
|
||||||
ATOMIC_STORE(&debug_info_.switch_to_empty_shell_ts_, ObClockGenerator::getCurrentTime());
|
ATOMIC_STORE(&debug_info_.switch_to_empty_shell_ts_, ObClockGenerator::getClock());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1461,7 +1461,7 @@ int MdsTableImpl<MdsTableType>::forcely_reset_mds_table(const char *reason)
|
|||||||
if (OB_FAIL(for_each_scan_row(FowEachRowAction::RESET, op))) {
|
if (OB_FAIL(for_each_scan_row(FowEachRowAction::RESET, op))) {
|
||||||
MDS_LOG_GC(ERROR, "fail to do reset");
|
MDS_LOG_GC(ERROR, "fail to do reset");
|
||||||
} else {
|
} else {
|
||||||
debug_info_.last_reset_ts_ = ObClockGenerator::getCurrentTime();
|
debug_info_.last_reset_ts_ = ObClockGenerator::getClock();
|
||||||
flushing_scn_.reset();
|
flushing_scn_.reset();
|
||||||
last_inner_recycled_scn_ = share::SCN::min_scn();
|
last_inner_recycled_scn_ = share::SCN::min_scn();
|
||||||
rec_scn_ = share::SCN::max_scn();
|
rec_scn_ = share::SCN::max_scn();
|
||||||
|
@ -193,7 +193,7 @@ int ObStandbyTimestampService::switch_to_leader()
|
|||||||
if (OB_FAIL(MTL(logservice::ObLogService *)->get_palf_role(share::GTS_LS, role, tmp_epoch))) {
|
if (OB_FAIL(MTL(logservice::ObLogService *)->get_palf_role(share::GTS_LS, role, tmp_epoch))) {
|
||||||
TRANS_LOG(WARN, "get ObStandbyTimestampService role fail", KR(ret));
|
TRANS_LOG(WARN, "get ObStandbyTimestampService role fail", KR(ret));
|
||||||
} else {
|
} else {
|
||||||
ATOMIC_STORE(&switch_to_leader_ts_, ObClockGenerator::getCurrentTime());
|
ATOMIC_STORE(&switch_to_leader_ts_, ObClockGenerator::getClock());
|
||||||
epoch_ = tmp_epoch;
|
epoch_ = tmp_epoch;
|
||||||
int64_t type = MTL(ObTimestampAccess *)->get_service_type();
|
int64_t type = MTL(ObTimestampAccess *)->get_service_type();
|
||||||
if (ObTimestampAccess::ServiceType::FOLLOWER == type) {
|
if (ObTimestampAccess::ServiceType::FOLLOWER == type) {
|
||||||
|
@ -30,10 +30,6 @@
|
|||||||
|
|
||||||
namespace oceanbase
|
namespace oceanbase
|
||||||
{
|
{
|
||||||
namespace memtable
|
|
||||||
{
|
|
||||||
extern TLOCAL(bool, TLOCAL_NEED_WAIT_IN_LOCK_WAIT_MGR);
|
|
||||||
}
|
|
||||||
using namespace sql;
|
using namespace sql;
|
||||||
using namespace common;
|
using namespace common;
|
||||||
using namespace share::detector;
|
using namespace share::detector;
|
||||||
@ -325,7 +321,7 @@ int ObTransDeadlockDetectorAdapter::gen_dependency_resource_array_(const ObIArra
|
|||||||
return ret;
|
return ret;
|
||||||
}
|
}
|
||||||
|
|
||||||
int ObTransDeadlockDetectorAdapter::register_remote_execution_to_deadlock_detector_(const ObTransID self_tx_id,
|
int ObTransDeadlockDetectorAdapter::register_to_deadlock_detector_(const ObTransID self_tx_id,
|
||||||
const uint32_t self_session_id,
|
const uint32_t self_session_id,
|
||||||
const ObIArray<ObTransIDAndAddr> &conflict_tx_ids,
|
const ObIArray<ObTransIDAndAddr> &conflict_tx_ids,
|
||||||
SessionGuard &session_guard)
|
SessionGuard &session_guard)
|
||||||
@ -375,7 +371,7 @@ int ObTransDeadlockDetectorAdapter::register_remote_execution_to_deadlock_detect
|
|||||||
#undef PRINT_WRAPPER
|
#undef PRINT_WRAPPER
|
||||||
}
|
}
|
||||||
|
|
||||||
int ObTransDeadlockDetectorAdapter::remote_execution_replace_conflict_trans_ids_(const ObTransID self_tx_id,
|
int ObTransDeadlockDetectorAdapter::replace_conflict_trans_ids_(const ObTransID self_tx_id,
|
||||||
const ObIArray<ObTransIDAndAddr> &conflict_tx_ids,
|
const ObIArray<ObTransIDAndAddr> &conflict_tx_ids,
|
||||||
SessionGuard &session_guard)
|
SessionGuard &session_guard)
|
||||||
{
|
{
|
||||||
@ -418,7 +414,7 @@ int ObTransDeadlockDetectorAdapter::remote_execution_replace_conflict_trans_ids_
|
|||||||
#undef PRINT_WRAPPER
|
#undef PRINT_WRAPPER
|
||||||
}
|
}
|
||||||
|
|
||||||
int ObTransDeadlockDetectorAdapter::register_remote_execution_or_replace_conflict_trans_ids(const ObTransID self_tx_id,
|
int ObTransDeadlockDetectorAdapter::register_or_replace_conflict_trans_ids(const ObTransID self_tx_id,
|
||||||
const uint32_t self_session_id,
|
const uint32_t self_session_id,
|
||||||
const ObArray<ObTransIDAndAddr> &conflict_tx_ids)
|
const ObArray<ObTransIDAndAddr> &conflict_tx_ids)
|
||||||
{
|
{
|
||||||
@ -444,13 +440,13 @@ int ObTransDeadlockDetectorAdapter::register_remote_execution_or_replace_conflic
|
|||||||
} else if (OB_FAIL(MTL(ObDeadLockDetectorMgr*)->check_detector_exist(self_tx_id, is_detector_exist))) {
|
} else if (OB_FAIL(MTL(ObDeadLockDetectorMgr*)->check_detector_exist(self_tx_id, is_detector_exist))) {
|
||||||
DETECT_LOG(WARN, "fail to get detector exist status", PRINT_WRAPPER);
|
DETECT_LOG(WARN, "fail to get detector exist status", PRINT_WRAPPER);
|
||||||
} else if (!is_detector_exist) {
|
} else if (!is_detector_exist) {
|
||||||
if (OB_FAIL(register_remote_execution_to_deadlock_detector_(self_tx_id, self_session_id, conflict_tx_ids, session_guard))) {
|
if (OB_FAIL(register_to_deadlock_detector_(self_tx_id, self_session_id, conflict_tx_ids, session_guard))) {
|
||||||
DETECT_LOG(WARN, "register new detector in remote execution failed", PRINT_WRAPPER);
|
DETECT_LOG(WARN, "register new detector in remote execution failed", PRINT_WRAPPER);
|
||||||
} else {
|
} else {
|
||||||
DETECT_LOG(INFO, "register new detector in remote execution", PRINT_WRAPPER);
|
DETECT_LOG(INFO, "register new detector in remote execution", PRINT_WRAPPER);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
if (OB_FAIL(remote_execution_replace_conflict_trans_ids_(self_tx_id, conflict_tx_ids, session_guard))) {
|
if (OB_FAIL(replace_conflict_trans_ids_(self_tx_id, conflict_tx_ids, session_guard))) {
|
||||||
DETECT_LOG(INFO, "replace block list in remote execution", PRINT_WRAPPER);
|
DETECT_LOG(INFO, "replace block list in remote execution", PRINT_WRAPPER);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -628,8 +624,6 @@ int ObTransDeadlockDetectorAdapter::maintain_deadlock_info_when_end_stmt(sql::Ob
|
|||||||
DETECT_LOG(ERROR, "session is NULL", PRINT_WRAPPER);
|
DETECT_LOG(ERROR, "session is NULL", PRINT_WRAPPER);
|
||||||
} else if (++step && session->is_inner()) {
|
} else if (++step && session->is_inner()) {
|
||||||
DETECT_LOG(TRACE, "inner session no need register to deadlock", PRINT_WRAPPER);
|
DETECT_LOG(TRACE, "inner session no need register to deadlock", PRINT_WRAPPER);
|
||||||
} else if (++step && memtable::TLOCAL_NEED_WAIT_IN_LOCK_WAIT_MGR) {
|
|
||||||
DETECT_LOG(TRACE, "will register deadlock in LockWaitMgr::post_process after end_stmt()", PRINT_WRAPPER);
|
|
||||||
} else if (++step && OB_ISNULL(desc = session->get_tx_desc())) {
|
} else if (++step && OB_ISNULL(desc = session->get_tx_desc())) {
|
||||||
ret = OB_BAD_NULL_ERROR;
|
ret = OB_BAD_NULL_ERROR;
|
||||||
DETECT_LOG(ERROR, "desc in session is NULL", PRINT_WRAPPER);
|
DETECT_LOG(ERROR, "desc in session is NULL", PRINT_WRAPPER);
|
||||||
@ -638,7 +632,7 @@ int ObTransDeadlockDetectorAdapter::maintain_deadlock_info_when_end_stmt(sql::Ob
|
|||||||
} else if (++step && !desc->is_valid()) {
|
} else if (++step && !desc->is_valid()) {
|
||||||
DETECT_LOG(INFO, "invalid tx desc no need register to deadlock", PRINT_WRAPPER);
|
DETECT_LOG(INFO, "invalid tx desc no need register to deadlock", PRINT_WRAPPER);
|
||||||
} else if (++step && is_rollback) {// statment is failed, maybe will try again, check if need register to deadlock detector
|
} else if (++step && is_rollback) {// statment is failed, maybe will try again, check if need register to deadlock detector
|
||||||
if (++step && session->get_query_timeout_ts() < ObClockGenerator::getCurrentTime()) {
|
if (++step && session->get_query_timeout_ts() < ObClockGenerator::getClock()) {
|
||||||
unregister_from_deadlock_detector(desc->tid(), UnregisterPath::END_STMT_TIMEOUT);
|
unregister_from_deadlock_detector(desc->tid(), UnregisterPath::END_STMT_TIMEOUT);
|
||||||
DETECT_LOG(INFO, "query timeout, no need register to deadlock", PRINT_WRAPPER);
|
DETECT_LOG(INFO, "query timeout, no need register to deadlock", PRINT_WRAPPER);
|
||||||
} else if (++step && conflict_txs.empty()) {
|
} else if (++step && conflict_txs.empty()) {
|
||||||
@ -647,9 +641,7 @@ int ObTransDeadlockDetectorAdapter::maintain_deadlock_info_when_end_stmt(sql::Ob
|
|||||||
} else if (++step && exec_ctx.get_errcode() != OB_TRY_LOCK_ROW_CONFLICT) {
|
} else if (++step && exec_ctx.get_errcode() != OB_TRY_LOCK_ROW_CONFLICT) {
|
||||||
unregister_from_deadlock_detector(desc->tid(), UnregisterPath::END_STMT_OTHER_ERR);
|
unregister_from_deadlock_detector(desc->tid(), UnregisterPath::END_STMT_OTHER_ERR);
|
||||||
DETECT_LOG(INFO, "try unregister deadlock detecotr cause meet non-lock error", PRINT_WRAPPER);
|
DETECT_LOG(INFO, "try unregister deadlock detecotr cause meet non-lock error", PRINT_WRAPPER);
|
||||||
} else if (++step && OB_FAIL(register_remote_execution_or_replace_conflict_trans_ids(desc->tid(),
|
} else if (++step && OB_FAIL(register_or_replace_conflict_trans_ids(desc->tid(), session->get_sessid(), conflict_txs))) {
|
||||||
session->get_sessid(),
|
|
||||||
conflict_txs))) {
|
|
||||||
DETECT_LOG(WARN, "register or replace list failed", PRINT_WRAPPER);
|
DETECT_LOG(WARN, "register or replace list failed", PRINT_WRAPPER);
|
||||||
} else {
|
} else {
|
||||||
// do nothing, register success or keep retrying
|
// do nothing, register success or keep retrying
|
||||||
@ -664,7 +656,7 @@ int ObTransDeadlockDetectorAdapter::maintain_deadlock_info_when_end_stmt(sql::Ob
|
|||||||
int exec_ctx_err_code = exec_ctx.get_errcode();
|
int exec_ctx_err_code = exec_ctx.get_errcode();
|
||||||
if (OB_SUCC(ret)) {
|
if (OB_SUCC(ret)) {
|
||||||
if (OB_SUCCESS != exec_ctx_err_code) {
|
if (OB_SUCCESS != exec_ctx_err_code) {
|
||||||
if ((OB_ITER_END != exec_ctx_err_code) && (2 != step)) {
|
if ((OB_ITER_END != exec_ctx_err_code)) {
|
||||||
if (session->get_retry_info().get_retry_cnt() <= 1 ||// first time lock conflict or other error
|
if (session->get_retry_info().get_retry_cnt() <= 1 ||// first time lock conflict or other error
|
||||||
session->get_retry_info().get_retry_cnt() % 10 == 0) {// other wise, control log print frequency
|
session->get_retry_info().get_retry_cnt() % 10 == 0) {// other wise, control log print frequency
|
||||||
DETECT_LOG(INFO, "maintain deadlock info", PRINT_WRAPPER);
|
DETECT_LOG(INFO, "maintain deadlock info", PRINT_WRAPPER);
|
||||||
@ -683,19 +675,29 @@ int ObTransDeadlockDetectorAdapter::maintain_deadlock_info_when_end_stmt(sql::Ob
|
|||||||
// @param [in] self_trans_id who am i.
|
// @param [in] self_trans_id who am i.
|
||||||
// @param [in] sess_id which session to kill if this node is killed.
|
// @param [in] sess_id which session to kill if this node is killed.
|
||||||
// @return the error code.
|
// @return the error code.
|
||||||
int ObTransDeadlockDetectorAdapter::register_local_execution_to_deadlock_detector_waiting_for_row(CollectCallBack &on_collect_op,
|
int ObTransDeadlockDetectorAdapter::lock_wait_mgr_reconstruct_detector_waiting_for_row(CollectCallBack &on_collect_op,
|
||||||
const BlockCallBack &func,
|
const BlockCallBack &func,
|
||||||
const ObTransID &self_trans_id,
|
const ObTransID &self_trans_id,
|
||||||
const uint32_t sess_id)
|
const uint32_t sess_id)
|
||||||
{
|
{
|
||||||
#define PRINT_WRAPPER KR(ret), K(self_trans_id), K(sess_id)
|
#define PRINT_WRAPPER KR(ret), K(self_trans_id), K(sess_id), K(exist)
|
||||||
CHECK_DEADLOCK_ENABLED();
|
CHECK_DEADLOCK_ENABLED();
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
|
bool exist = false;
|
||||||
if (sess_id == 0) {
|
if (sess_id == 0) {
|
||||||
DETECT_LOG(ERROR, "invalid session id", PRINT_WRAPPER);
|
DETECT_LOG(ERROR, "invalid session id", PRINT_WRAPPER);
|
||||||
} else if (nullptr == (MTL(ObDeadLockDetectorMgr*))) {
|
} else if (nullptr == (MTL(ObDeadLockDetectorMgr*))) {
|
||||||
ret = OB_ERR_UNEXPECTED;
|
ret = OB_ERR_UNEXPECTED;
|
||||||
DETECT_LOG(WARN, "fail to get ObDeadLockDetectorMgr", PRINT_WRAPPER);
|
DETECT_LOG(WARN, "fail to get ObDeadLockDetectorMgr", PRINT_WRAPPER);
|
||||||
|
} else if (OB_FAIL(MTL(ObDeadLockDetectorMgr*)->check_detector_exist(self_trans_id, exist))) {
|
||||||
|
DETECT_LOG(WARN, "fail to check detector exist", PRINT_WRAPPER);
|
||||||
|
} else if (exist) {
|
||||||
|
if (MTL(ObDeadLockDetectorMgr*)->unregister_key(self_trans_id)) {
|
||||||
|
DETECT_LOG(WARN, "fail to unregister key", PRINT_WRAPPER);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (OB_FAIL(ret)) {
|
||||||
|
DETECT_LOG(WARN, "local execution register to deadlock detector waiting for row failed", PRINT_WRAPPER);
|
||||||
} else if (OB_FAIL(create_detector_node_and_set_parent_if_needed_(on_collect_op, self_trans_id, sess_id))) {
|
} else if (OB_FAIL(create_detector_node_and_set_parent_if_needed_(on_collect_op, self_trans_id, sess_id))) {
|
||||||
DETECT_LOG(WARN, "fail to create detector node", PRINT_WRAPPER);
|
DETECT_LOG(WARN, "fail to create detector node", PRINT_WRAPPER);
|
||||||
} else if (OB_FAIL(MTL(ObDeadLockDetectorMgr*)->block(self_trans_id, func))) {
|
} else if (OB_FAIL(MTL(ObDeadLockDetectorMgr*)->block(self_trans_id, func))) {
|
||||||
@ -714,18 +716,28 @@ int ObTransDeadlockDetectorAdapter::register_local_execution_to_deadlock_detecto
|
|||||||
// @param [in] self_trans_id who am i.
|
// @param [in] self_trans_id who am i.
|
||||||
// @param [in] sess_id which session to kill if this node is killed.
|
// @param [in] sess_id which session to kill if this node is killed.
|
||||||
// @return the error code.
|
// @return the error code.
|
||||||
int ObTransDeadlockDetectorAdapter::register_local_execution_to_deadlock_detector_waiting_for_trans(CollectCallBack &on_collect_op,
|
int ObTransDeadlockDetectorAdapter::lock_wait_mgr_reconstruct_detector_waiting_for_trans(CollectCallBack &on_collect_op,
|
||||||
const ObTransID &conflict_trans_id,
|
const ObTransID &conflict_trans_id,
|
||||||
const ObTransID &self_trans_id,
|
const ObTransID &self_trans_id,
|
||||||
const uint32_t sess_id)
|
const uint32_t sess_id)
|
||||||
{
|
{
|
||||||
#define PRINT_WRAPPER KR(ret), K(scheduler_addr), K(self_trans_id), K(sess_id)
|
#define PRINT_WRAPPER KR(ret), K(scheduler_addr), K(self_trans_id), K(sess_id), K(exist)
|
||||||
CHECK_DEADLOCK_ENABLED();
|
CHECK_DEADLOCK_ENABLED();
|
||||||
int ret = OB_SUCCESS;
|
int ret = OB_SUCCESS;
|
||||||
ObAddr scheduler_addr;
|
ObAddr scheduler_addr;
|
||||||
|
bool exist = false;
|
||||||
if (nullptr == (MTL(ObDeadLockDetectorMgr*))) {
|
if (nullptr == (MTL(ObDeadLockDetectorMgr*))) {
|
||||||
ret = OB_ERR_UNEXPECTED;
|
ret = OB_ERR_UNEXPECTED;
|
||||||
DETECT_LOG(WARN, "fail to get ObDeadLockDetectorMgr", PRINT_WRAPPER);
|
DETECT_LOG(WARN, "fail to get ObDeadLockDetectorMgr", PRINT_WRAPPER);
|
||||||
|
} else if (OB_FAIL(MTL(ObDeadLockDetectorMgr*)->check_detector_exist(self_trans_id, exist))) {
|
||||||
|
DETECT_LOG(WARN, "fail to check detector exist", PRINT_WRAPPER);
|
||||||
|
} else if (exist) {
|
||||||
|
if (MTL(ObDeadLockDetectorMgr*)->unregister_key(self_trans_id)) {
|
||||||
|
DETECT_LOG(WARN, "fail to unregister key", PRINT_WRAPPER);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (OB_FAIL(ret)) {
|
||||||
|
DETECT_LOG(WARN, "local execution register to deadlock detector waiting for row failed", PRINT_WRAPPER);
|
||||||
} else if (OB_FAIL(get_conflict_trans_scheduler(conflict_trans_id, scheduler_addr))) {
|
} else if (OB_FAIL(get_conflict_trans_scheduler(conflict_trans_id, scheduler_addr))) {
|
||||||
DETECT_LOG(WARN, "fail to get conflict trans scheduler addr", PRINT_WRAPPER);
|
DETECT_LOG(WARN, "fail to get conflict trans scheduler addr", PRINT_WRAPPER);
|
||||||
} else if (OB_FAIL(create_detector_node_and_set_parent_if_needed_(on_collect_op, self_trans_id, sess_id))) {
|
} else if (OB_FAIL(create_detector_node_and_set_parent_if_needed_(on_collect_op, self_trans_id, sess_id))) {
|
||||||
@ -733,7 +745,7 @@ int ObTransDeadlockDetectorAdapter::register_local_execution_to_deadlock_detecto
|
|||||||
} else if (OB_FAIL(MTL(ObDeadLockDetectorMgr*)->block(self_trans_id, scheduler_addr, conflict_trans_id))) {
|
} else if (OB_FAIL(MTL(ObDeadLockDetectorMgr*)->block(self_trans_id, scheduler_addr, conflict_trans_id))) {
|
||||||
DETECT_LOG(WARN, "fail to block on conflict trans", PRINT_WRAPPER);
|
DETECT_LOG(WARN, "fail to block on conflict trans", PRINT_WRAPPER);
|
||||||
} else {
|
} else {
|
||||||
DETECT_LOG(INFO, "local execution register to deadlock detector waiting for trans success", PRINT_WRAPPER);
|
DETECT_LOG(TRACE, "local execution register to deadlock detector waiting for trans success", PRINT_WRAPPER);
|
||||||
}
|
}
|
||||||
return ret;
|
return ret;
|
||||||
#undef PRINT_WRAPPER
|
#undef PRINT_WRAPPER
|
||||||
|
@ -111,11 +111,11 @@ class ObTransDeadlockDetectorAdapter
|
|||||||
}
|
}
|
||||||
/**********MAIN INTERFACE**********/
|
/**********MAIN INTERFACE**********/
|
||||||
// for local execution, call from lock wait mgr
|
// for local execution, call from lock wait mgr
|
||||||
static int register_local_execution_to_deadlock_detector_waiting_for_row(CollectCallBack &on_collect_op,
|
static int lock_wait_mgr_reconstruct_detector_waiting_for_row(CollectCallBack &on_collect_op,
|
||||||
const BlockCallBack &call_back,
|
const BlockCallBack &call_back,
|
||||||
const ObTransID &self_trans_id,
|
const ObTransID &self_trans_id,
|
||||||
const uint32_t sess_id);
|
const uint32_t sess_id);
|
||||||
static int register_local_execution_to_deadlock_detector_waiting_for_trans(CollectCallBack &on_collect_op,
|
static int lock_wait_mgr_reconstruct_detector_waiting_for_trans(CollectCallBack &on_collect_op,
|
||||||
const ObTransID &conflict_trans_id,
|
const ObTransID &conflict_trans_id,
|
||||||
const ObTransID &self_trans_id,
|
const ObTransID &self_trans_id,
|
||||||
const uint32_t sess_id);
|
const uint32_t sess_id);
|
||||||
@ -143,7 +143,7 @@ class ObTransDeadlockDetectorAdapter
|
|||||||
static int get_trans_scheduler_info_on_participant(const ObTransID trans_id,
|
static int get_trans_scheduler_info_on_participant(const ObTransID trans_id,
|
||||||
const share::ObLSID ls_id,
|
const share::ObLSID ls_id,
|
||||||
ObAddr &scheduler_addr);
|
ObAddr &scheduler_addr);
|
||||||
static int register_remote_execution_or_replace_conflict_trans_ids(const ObTransID self_tx_id,
|
static int register_or_replace_conflict_trans_ids(const ObTransID self_tx_id,
|
||||||
const uint32_t self_session_id,
|
const uint32_t self_session_id,
|
||||||
const ObArray<ObTransIDAndAddr> &conflict_tx_ids);
|
const ObArray<ObTransIDAndAddr> &conflict_tx_ids);
|
||||||
static int kill_stmt(const uint32_t sess_id);
|
static int kill_stmt(const uint32_t sess_id);
|
||||||
@ -152,11 +152,11 @@ class ObTransDeadlockDetectorAdapter
|
|||||||
char *dest_ptr,// C-style str, contain '\0'
|
char *dest_ptr,// C-style str, contain '\0'
|
||||||
const int64_t dest_len);
|
const int64_t dest_len);
|
||||||
private:
|
private:
|
||||||
static int register_remote_execution_to_deadlock_detector_(const ObTransID self_tx_id,
|
static int register_to_deadlock_detector_(const ObTransID self_tx_id,
|
||||||
const uint32_t self_session_id,
|
const uint32_t self_session_id,
|
||||||
const ObIArray<ObTransIDAndAddr> &conflict_tx_ids,
|
const ObIArray<ObTransIDAndAddr> &conflict_tx_ids,
|
||||||
SessionGuard &session_guard);
|
SessionGuard &session_guard);
|
||||||
static int remote_execution_replace_conflict_trans_ids_(const ObTransID self_tx_id,
|
static int replace_conflict_trans_ids_(const ObTransID self_tx_id,
|
||||||
const ObIArray<ObTransIDAndAddr> &conflict_tx_ids,
|
const ObIArray<ObTransIDAndAddr> &conflict_tx_ids,
|
||||||
SessionGuard &session_guard);
|
SessionGuard &session_guard);
|
||||||
static int create_detector_node_and_set_parent_if_needed_(CollectCallBack &on_collect_op,
|
static int create_detector_node_and_set_parent_if_needed_(CollectCallBack &on_collect_op,
|
||||||
|
Reference in New Issue
Block a user