[DeadLock] adjust some log print and fix get_scheudler may deadlock risk
This commit is contained in:
parent
d2e539d594
commit
7d09de4477
@ -319,7 +319,7 @@ int ObDeadLockDetectorMgr::unregister_key_(const UserBinaryKey &key)
|
||||
if (OB_FAIL(detector_map_.del(key))) {
|
||||
DETECT_LOG(WARN, "detector_map_ erase node failed", PRINT_WRAPPER);
|
||||
} else {
|
||||
DETECT_LOG(INFO, "unregister key success", PRINT_WRAPPER);
|
||||
DETECT_LOG(TRACE, "unregister key success", PRINT_WRAPPER);
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
|
@ -280,7 +280,7 @@ int ObDeadLockDetectorMgr::register_key(const KeyType &key,
|
||||
detector_map_.revert(p_detector);
|
||||
} else {
|
||||
detector_map_.revert(p_detector);
|
||||
DETECT_LOG(INFO, "register key success", PRINT_WRAPPER);
|
||||
DETECT_LOG(TRACE, "register key success", PRINT_WRAPPER);
|
||||
}
|
||||
} else {
|
||||
DETECT_LOG(ERROR, "get key error, couldn't handle", PRINT_WRAPPER);
|
||||
|
@ -67,7 +67,7 @@ ObLCLNode::ObLCLNode(const UserBinaryKey &user_key,
|
||||
on_collect_operation.is_valid();
|
||||
created_time_ = ObClockGenerator::getRealClock();
|
||||
allow_detect_time_ = created_time_ + start_delay;
|
||||
DETECT_LOG_(INFO, "new detector instance created", PRINT_WRAPPER);
|
||||
DETECT_LOG_(TRACE, "new detector instance created", PRINT_WRAPPER);
|
||||
#undef PRINT_WRAPPER
|
||||
ATOMIC_INC(&(ObIDeadLockDetector::total_constructed_count));
|
||||
}
|
||||
|
@ -140,7 +140,7 @@ void ObMemtableCtx::reset()
|
||||
unsynced_cnt_ = 0;
|
||||
unsubmitted_cnt_ = 0;
|
||||
lock_mem_ctx_.reset();
|
||||
mtstat_.reset();
|
||||
retry_info_.reset();
|
||||
trans_mgr_.reset();
|
||||
log_gen_.reset();
|
||||
ref_ = 0;
|
||||
@ -298,12 +298,12 @@ int ObMemtableCtx::write_lock_yield()
|
||||
|
||||
void ObMemtableCtx::on_wlock_retry(const ObMemtableKey& key, const transaction::ObTransID &conflict_tx_id)
|
||||
{
|
||||
mtstat_.on_wlock_retry();
|
||||
#define USING_LOG_PREFIX TRANS
|
||||
if (mtstat_.need_print()) {
|
||||
if (retry_info_.need_print()) {
|
||||
FLOG_INFO("mvcc_write conflict", K(key), "tx_id", get_tx_id(), K(conflict_tx_id), KPC(this));
|
||||
}
|
||||
#undef USING_LOG_PREFIX
|
||||
retry_info_.on_conflict();
|
||||
}
|
||||
|
||||
void ObMemtableCtx::on_tsc_retry(const ObMemtableKey& key,
|
||||
@ -311,10 +311,10 @@ void ObMemtableCtx::on_tsc_retry(const ObMemtableKey& key,
|
||||
const SCN max_trans_version,
|
||||
const transaction::ObTransID &conflict_tx_id)
|
||||
{
|
||||
mtstat_.on_tsc_retry();
|
||||
if (mtstat_.need_print()) {
|
||||
if (retry_info_.need_print()) {
|
||||
TRANS_LOG_RET(WARN, OB_SUCCESS, "transaction_set_consistency conflict", K(key), K(snapshot_version), K(max_trans_version), K(conflict_tx_id), KPC(this));
|
||||
}
|
||||
retry_info_.on_conflict();
|
||||
}
|
||||
|
||||
void *ObMemtableCtx::old_row_alloc(const int64_t size)
|
||||
|
@ -15,8 +15,11 @@
|
||||
|
||||
#include "lib/allocator/ob_fifo_allocator.h"
|
||||
#include "lib/checksum/ob_crc64.h"
|
||||
#include "lib/list/ob_dlist.h"
|
||||
#include "lib/lock/ob_spin_lock.h"
|
||||
#include "lib/lock/ob_small_spin_lock.h"
|
||||
#include "lib/utility/ob_macro_utils.h"
|
||||
#include "ob_clock_generator.h"
|
||||
#include "share/ob_define.h"
|
||||
#include "storage/memtable/ob_memtable_interface.h"
|
||||
#include "storage/memtable/ob_memtable_mutator.h"
|
||||
@ -42,30 +45,33 @@ struct ObTableLockInfo;
|
||||
namespace memtable
|
||||
{
|
||||
|
||||
class MemtableCtxStat
|
||||
struct RetryInfo
|
||||
{
|
||||
public:
|
||||
MemtableCtxStat(): wlock_retry_(0), tsc_retry_(0) {}
|
||||
~MemtableCtxStat() {}
|
||||
void reset()
|
||||
{
|
||||
wlock_retry_ = 0;
|
||||
tsc_retry_ = 0;
|
||||
RetryInfo() : retry_cnt_(0), last_retry_ts_(0) {}
|
||||
int64_t to_string(char *buf, const int64_t buf_len) const {
|
||||
int64_t pos = 0;
|
||||
int64_t retry_cnt = ATOMIC_LOAD(&retry_cnt_);
|
||||
(void) databuff_printf(buf, buf_len, pos, "retry_cnt:%ld, last_retry_ts:%s",
|
||||
retry_cnt, ObTime2Str::ob_timestamp_str(last_retry_ts_));
|
||||
return pos;
|
||||
}
|
||||
void on_wlock_retry() { (void)ATOMIC_FAA(&wlock_retry_, 1); }
|
||||
void on_tsc_retry() { (void)ATOMIC_FAA(&tsc_retry_, 1); }
|
||||
int32_t get_wlock_retry_count() { return ATOMIC_LOAD(&wlock_retry_); }
|
||||
int32_t get_tsc_retry_count() { return ATOMIC_LOAD(&tsc_retry_); }
|
||||
bool need_print() const
|
||||
{
|
||||
return 1 == wlock_retry_
|
||||
|| 1 == tsc_retry_
|
||||
|| 0 == wlock_retry_ % 10
|
||||
|| 0 == tsc_retry_ % 10;
|
||||
void reset() { retry_cnt_ = 0; last_retry_ts_ = 0; }
|
||||
void on_conflict() {
|
||||
ATOMIC_AAF(&retry_cnt_, 1);
|
||||
last_retry_ts_ = ObClockGenerator::getClock();
|
||||
}
|
||||
private:
|
||||
int32_t wlock_retry_;
|
||||
int32_t tsc_retry_;
|
||||
bool need_print() const {
|
||||
bool ret = false;
|
||||
int64_t ts = ObClockGenerator::getClock();
|
||||
if (ATOMIC_LOAD(&retry_cnt_) % 10 == 0 ||// retry cnt more than specified times
|
||||
ts - last_retry_ts_ >= 1_s ||// retry interval more than specified interval seconds
|
||||
last_retry_ts_ == 0) {// retry ts is invalid
|
||||
ret = true;
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
int64_t retry_cnt_;
|
||||
int64_t last_retry_ts_;
|
||||
};
|
||||
|
||||
// 1. When fill redo log, if there is a big row, the meta info should record
|
||||
@ -531,7 +537,7 @@ private:
|
||||
ObQueryAllocator query_allocator_;
|
||||
ObMemtableCtxCbAllocator ctx_cb_allocator_;
|
||||
ObRedoLogGenerator log_gen_;
|
||||
MemtableCtxStat mtstat_;
|
||||
RetryInfo retry_info_;
|
||||
transaction::ObPartTransCtx *ctx_;
|
||||
int64_t truncate_cnt_;
|
||||
// the retry count of lock for read
|
||||
|
@ -503,33 +503,17 @@ int ObTransDeadlockDetectorAdapter::get_trans_scheduler_info_on_participant(cons
|
||||
const share::ObLSID ls_id,
|
||||
ObAddr &scheduler_addr)
|
||||
{
|
||||
#define PRINT_WRAPPER KR(ret), KR(tmp_ret), K(trans_id), K(ls_id), K(scheduler_addr)
|
||||
#define PRINT_WRAPPER KR(ret), K(trans_id), K(ls_id), K(scheduler_addr)
|
||||
int ret = OB_SUCCESS;
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
ObTransService *tx_service = MTL(ObTransService*);
|
||||
transaction::ObPartTransCtx *part_trans_ctx = nullptr;
|
||||
if (OB_ISNULL(tx_service)) {
|
||||
ObLSService *ls_service = MTL(ObLSService*);
|
||||
ObLSHandle ls_handle;
|
||||
if (OB_ISNULL(ls_service)) {
|
||||
ret = OB_BAD_NULL_ERROR;
|
||||
DETECT_LOG(ERROR, "tx_service is NULL", PRINT_WRAPPER);
|
||||
} else if (OB_FAIL(tx_service->get_tx_ctx_mgr().get_tx_ctx(ls_id, trans_id, false, part_trans_ctx))) {
|
||||
if (ret == OB_TRANS_CTX_NOT_EXIST) {
|
||||
DETECT_LOG(INFO, "conflict trans_id's ctx is not exist anymore",PRINT_WRAPPER);
|
||||
scheduler_addr.reset();
|
||||
} else {
|
||||
DETECT_LOG(WARN, "get part_trans_ctx failed", PRINT_WRAPPER);
|
||||
}
|
||||
} else if (OB_ISNULL(part_trans_ctx)) {
|
||||
ret = OB_BAD_NULL_ERROR;
|
||||
DETECT_LOG(WARN, "pat_trans_ctx is NULL", PRINT_WRAPPER);
|
||||
} else {
|
||||
scheduler_addr = part_trans_ctx->get_scheduler();
|
||||
if (!scheduler_addr.is_valid()) {
|
||||
ret = OB_TRANS_CTX_NOT_EXIST;
|
||||
DETECT_LOG(WARN, "scheduler on part trans ctx is invalid", PRINT_WRAPPER);
|
||||
}
|
||||
if (OB_TMP_FAIL(tx_service->get_tx_ctx_mgr().revert_tx_ctx(part_trans_ctx))) {
|
||||
DETECT_LOG(ERROR, "revert tx ctx failed", PRINT_WRAPPER, KPC(part_trans_ctx));
|
||||
}
|
||||
DETECT_LOG(ERROR, "ls_service is NULL", PRINT_WRAPPER);
|
||||
} else if (OB_FAIL(ls_service->get_ls(ls_id, ls_handle, ObLSGetMod::DEADLOCK_MOD))) {
|
||||
DETECT_LOG(WARN, "fail to get ls", PRINT_WRAPPER);
|
||||
} else if (OB_FAIL(ls_handle.get_ls()->get_tx_scheduler(trans_id, scheduler_addr))) {
|
||||
DETECT_LOG(WARN, "fail to get tx scheduler", PRINT_WRAPPER);
|
||||
}
|
||||
return ret;
|
||||
#undef PRINT_WRAPPER
|
||||
@ -678,9 +662,14 @@ int ObTransDeadlockDetectorAdapter::maintain_deadlock_info_when_end_stmt(sql::Ob
|
||||
desc->reset_conflict_txs();
|
||||
}
|
||||
int exec_ctx_err_code = exec_ctx.get_errcode();
|
||||
if (OB_SUCCESS != exec_ctx_err_code) {
|
||||
if ((OB_ITER_END != exec_ctx_err_code) && (2 != step)) {
|
||||
DETECT_LOG(INFO, "maintain deadlock info", PRINT_WRAPPER);
|
||||
if (OB_SUCC(ret)) {
|
||||
if (OB_SUCCESS != exec_ctx_err_code) {
|
||||
if ((OB_ITER_END != exec_ctx_err_code) && (2 != step)) {
|
||||
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
|
||||
DETECT_LOG(INFO, "maintain deadlock info", PRINT_WRAPPER);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -712,7 +701,7 @@ int ObTransDeadlockDetectorAdapter::register_local_execution_to_deadlock_detecto
|
||||
} else if (OB_FAIL(MTL(ObDeadLockDetectorMgr*)->block(self_trans_id, func))) {
|
||||
DETECT_LOG(WARN, "fail to block on call back function", PRINT_WRAPPER);
|
||||
} else {
|
||||
DETECT_LOG(INFO, "local execution register to deadlock detector waiting for row success", PRINT_WRAPPER);
|
||||
DETECT_LOG(TRACE, "local execution register to deadlock detector waiting for row success", PRINT_WRAPPER);
|
||||
}
|
||||
return ret;
|
||||
#undef PRINT_WRAPPER
|
||||
@ -839,7 +828,7 @@ void ObTransDeadlockDetectorAdapter::unregister_from_deadlock_detector(const ObT
|
||||
ret = OB_SUCCESS;// it's ok if detector not exist
|
||||
}
|
||||
} else {
|
||||
DETECT_LOG(INFO, "unregister from deadlock detector success", K(self_trans_id), K(to_string(path)));
|
||||
DETECT_LOG(TRACE, "unregister from deadlock detector success", K(self_trans_id), K(to_string(path)));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -34,6 +34,7 @@
|
||||
#include "storage/tx/ob_trans_define.h"
|
||||
#include "share/deadlock/ob_deadlock_detector_rpc.h"
|
||||
#include "share/deadlock/mock_deadlock_rpc.h"
|
||||
#include "storage/memtable/ob_memtable_context.h"
|
||||
|
||||
using namespace oceanbase::obrpc;
|
||||
using namespace std;
|
||||
@ -440,6 +441,25 @@ TEST_F(TestObDeadLockDetector, small_cycle_in_big_cycle_bad_case) {
|
||||
ASSERT_EQ(true, TestOperation::v_killed_node[1] == 2 || TestOperation::v_killed_node[1] == 5);
|
||||
}
|
||||
|
||||
// TEST_F(TestObDeadLockDetector, test_lock_conflict_print) {
|
||||
// memtable::RetryInfo retry_info;
|
||||
// while ((ObClockGenerator::getClock() % 1000000) < 100_ms);
|
||||
// DETECT_LOG(INFO, "DEBUG1", K(ObClockGenerator::getClock()));
|
||||
// ASSERT_EQ(retry_info.need_print(), true);
|
||||
// retry_info.on_conflict();
|
||||
// ASSERT_EQ(retry_info.need_print(), false);
|
||||
// this_thread::sleep_for(std::chrono::milliseconds(100));
|
||||
// DETECT_LOG(INFO, "DEBUG2", K(ObClockGenerator::getClock()));
|
||||
// ASSERT_EQ(retry_info.need_print(), false);
|
||||
// this_thread::sleep_for(std::chrono::milliseconds(1000));
|
||||
// DETECT_LOG(INFO, "DEBUG", K(retry_info));
|
||||
// ASSERT_EQ(retry_info.need_print(), true);
|
||||
// for (int i = 0; i < 9; ++i) {
|
||||
// retry_info.on_conflict();
|
||||
// }
|
||||
// ASSERT_EQ(retry_info.need_print(), true);
|
||||
// }
|
||||
|
||||
// TEST_F(TestObDeadLockDetector, print_timestamp) {
|
||||
// int64_t ts_ = 1623827288705600;
|
||||
// DETECT_LOG(INFO, "test ts", KTIME(ts_));
|
||||
|
Loading…
x
Reference in New Issue
Block a user