[fix] fix some error logs

This commit is contained in:
BinChenn
2023-06-27 08:17:51 +00:00
committed by ob-robot
parent 4085b6c0c6
commit 6a6deadb47
7 changed files with 61 additions and 17 deletions

View File

@ -156,7 +156,7 @@ TEST_F(TestObSimpleLogClusterArbMockEleService, switch_leader_during_degrading)
LogConfigInfo new_config_info;
EXPECT_EQ(OB_SUCCESS, leader.palf_handle_impl_->check_args_and_generate_config_(upgrade_b_args, upgrade_b_pid, upgrade_b_ele_epoch,
is_already_finished, new_config_info));
EXPECT_EQ(OB_SUCCESS, leader.palf_handle_impl_->wait_log_barrier_(upgrade_b_args, new_config_info, not_timeout));
EXPECT_UNTIL_EQ(OB_SUCCESS, leader.palf_handle_impl_->wait_log_barrier_(upgrade_b_args, new_config_info, not_timeout));
const LSN &leader_max_flushed_end_lsn = leader.palf_handle_impl_->sw_.max_flushed_end_lsn_;
EXPECT_GT(leader.palf_handle_impl_->sw_.max_flushed_end_lsn_, leader.palf_handle_impl_->config_mgr_.reconfig_barrier_.prev_lsn_);
@ -272,7 +272,7 @@ TEST_F(TestObSimpleLogClusterArbMockEleService, switch_leader_to_other_during_de
LogConfigInfo new_config_info;
EXPECT_EQ(OB_SUCCESS, leader.palf_handle_impl_->check_args_and_generate_config_(upgrade_a_args, upgrade_a_pid, upgrade_a_ele_epoch,
is_already_finished, new_config_info));
EXPECT_EQ(OB_SUCCESS, leader.palf_handle_impl_->wait_log_barrier_(upgrade_a_args, new_config_info, not_timeout));
EXPECT_UNTIL_EQ(OB_SUCCESS, leader.palf_handle_impl_->wait_log_barrier_(upgrade_a_args, new_config_info, not_timeout));
EXPECT_GT(leader.palf_handle_impl_->sw_.max_flushed_end_lsn_, leader.palf_handle_impl_->config_mgr_.reconfig_barrier_.prev_lsn_);
EXPECT_GT(b_handle->palf_handle_impl_->sw_.max_flushed_end_lsn_, leader.palf_handle_impl_->config_mgr_.reconfig_barrier_.prev_lsn_);
@ -632,7 +632,7 @@ TEST_F(TestObSimpleLogClusterArbMockEleService, test_add_remove_lose_logs)
LogConfigInfo new_config_info;
EXPECT_EQ(OB_SUCCESS, leader.palf_handle_impl_->check_args_and_generate_config_(remove_e_args, remove_e_pid, remove_e_ele_epoch,
is_already_finished, new_config_info));
EXPECT_EQ(OB_SUCCESS, leader.palf_handle_impl_->wait_log_barrier_(remove_e_args, new_config_info, not_timeout));
EXPECT_UNTIL_EQ(OB_SUCCESS, leader.palf_handle_impl_->wait_log_barrier_(remove_e_args, new_config_info, not_timeout));
const LSN &remove_e_barrier = leader.palf_handle_impl_->config_mgr_.checking_barrier_.prev_end_lsn_;
leader.palf_handle_impl_->state_mgr_.reset_changing_config_with_arb();

View File

@ -437,6 +437,36 @@ TEST_F(TestObSimpleLogClusterSingleReplica, single_replica_flashback)
EXPECT_EQ(new_log_tail, leader.palf_handle_impl_->get_end_lsn());
EXPECT_EQ(OB_ITER_END, read_log(leader));
// flashback reconfirming leader
EXPECT_EQ(OB_SUCCESS, submit_log(leader, 100, leader_idx));
wait_until_has_committed(leader, leader.palf_handle_impl_->sw_.get_max_lsn());
SCN flashback_scn = leader.palf_handle_impl_->get_max_scn();
EXPECT_EQ(OB_SUCCESS, submit_log(leader, 100, leader_idx));
wait_until_has_committed(leader, leader.palf_handle_impl_->sw_.get_max_lsn());
EXPECT_EQ(OB_ITER_END, read_log(leader));
switch_append_to_flashback(leader, mode_version);
dynamic_cast<palf::PalfEnvImpl*>(get_cluster()[0]->get_palf_env())->log_loop_thread_.stop();
dynamic_cast<palf::PalfEnvImpl*>(get_cluster()[0]->get_palf_env())->log_loop_thread_.wait();
leader.palf_handle_impl_->state_mgr_.role_ = LEADER;
leader.palf_handle_impl_->state_mgr_.state_ = RECONFIRM;
EXPECT_EQ(OB_EAGAIN, leader.palf_handle_impl_->flashback(mode_version, max_scn, timeout_ts_us));
EXPECT_GT(leader.palf_handle_impl_->sw_.get_max_scn(), flashback_scn);
leader.palf_handle_impl_->state_mgr_.role_ = FOLLOWER;
leader.palf_handle_impl_->state_mgr_.state_ = ACTIVE;
EXPECT_EQ(OB_SUCCESS, leader.palf_handle_impl_->flashback(mode_version, max_scn, timeout_ts_us));
EXPECT_LT(leader.palf_handle_impl_->sw_.get_max_scn(), flashback_scn);
EXPECT_EQ(new_log_tail, leader.palf_handle_impl_->get_end_lsn());
EXPECT_EQ(OB_ITER_END, read_log(leader));
leader.palf_handle_impl_->state_mgr_.role_ = LEADER;
leader.palf_handle_impl_->state_mgr_.state_ = ACTIVE;
dynamic_cast<palf::PalfEnvImpl*>(get_cluster()[0]->get_palf_env())->log_loop_thread_.start();
switch_flashback_to_append(leader, mode_version);
// 数据全部清空
wait_until_has_committed(leader, leader.palf_handle_impl_->sw_.get_max_lsn());
switch_append_to_flashback(leader, mode_version);

View File

@ -538,7 +538,6 @@ int LogEngine::submit_purge_throttling_task(const PurgeThrottlingType purge_type
int LogEngine::append_log(const LSN &lsn, const LogWriteBuf &write_buf, const SCN &scn)
{
int ret = OB_SUCCESS;
ObTimeGuard time_guard("append_log", 100);
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
PALF_LOG(ERROR, "LogEngine not inited!!!", K(ret), K_(palf_id), K_(is_inited));
@ -551,7 +550,6 @@ int LogEngine::append_log(const LSN &lsn, const LogWriteBuf &write_buf, const SC
PALF_LOG(
TRACE, "LogEngine append_log success", K(ret), K_(palf_id), K_(is_inited), K(lsn), K(write_buf), K(scn));
}
time_guard.click("append_log");
return ret;
}

View File

@ -219,7 +219,7 @@ int LogIOWorker::handle_io_task_(LogIOTask *io_task)
do_task_used_ts_ += cost_ts;
do_task_count_ ++;
if (palf_reach_time_interval(5 * 1000 * 1000, print_log_interval_)) {
PALF_EVENT("io statistics", 0, K_(do_task_used_ts), K_(do_task_count),
PALF_LOG(INFO, "[PALF STAT IO STAT]", K_(do_task_used_ts), K_(do_task_count),
"average_cost_ts", do_task_used_ts_ / do_task_count_,
"io_queue_size", queue_.size());
do_task_count_ = 0;

View File

@ -722,8 +722,7 @@ int LogModeMgr::receive_mode_meta_(const common::ObAddr &server,
{
int ret = OB_SUCCESS;
if (proposal_id == INVALID_PROPOSAL_ID ||
false == mode_meta.is_valid() ||
proposal_id < mode_meta.proposal_id_) {
false == mode_meta.is_valid()) {
ret = OB_INVALID_ARGUMENT;
PALF_LOG(WARN, "invalid arguments", K(ret), K_(palf_id), K_(self), K(server), K(proposal_id), K(mode_meta));
} else {

View File

@ -78,7 +78,7 @@ PalfHandleImpl::PalfHandleImpl()
mode_change_lock_(),
flashback_lock_(),
last_dump_info_time_us_(OB_INVALID_TIMESTAMP),
is_flashback_done_(false),
flashback_state_(LogFlashbackState::FLASHBACK_INIT),
last_check_sync_time_us_(OB_INVALID_TIMESTAMP),
last_renew_loc_time_us_(OB_INVALID_TIMESTAMP),
last_print_in_sync_time_us_(OB_INVALID_TIMESTAMP),
@ -3859,7 +3859,6 @@ int PalfHandleImpl::inner_after_truncate_prefix_blocks(const TruncatePrefixBlock
int PalfHandleImpl::inner_after_flashback(const FlashbackCbCtx &flashback_ctx)
{
int ret = OB_SUCCESS;
is_flashback_done_ = true;
// do nothing
return ret;
}
@ -4268,7 +4267,7 @@ int PalfHandleImpl::flashback(const int64_t mode_version,
FlashbackCbCtx flashback_cb_ctx(flashback_scn);
const LSN max_lsn = get_max_lsn();
const LSN end_lsn = get_end_lsn();
is_flashback_done_ = false;
flashback_state_ = LogFlashbackState::FLASHBACK_INIT;
PALF_EVENT("[BEGIN FLASHBACK]", palf_id_, KPC(this), K(mode_version), K(flashback_scn), K(timeout_us),
K(end_lsn), K(max_lsn));
do {
@ -4279,8 +4278,15 @@ int PalfHandleImpl::flashback(const int64_t mode_version,
} while (0);
TimeoutChecker not_timeout(timeout_us);
while (OB_SUCC(ret) && OB_SUCC(not_timeout())) {
if (is_flashback_done_ == true) {
RLockGuard guard(lock_);
if (LogFlashbackState::FLASHBACK_FAILED == flashback_state_) {
ret = OB_EAGAIN;
PALF_LOG(WARN, "flashback failed", K(ret), KPC(this), K(mode_version), K(flashback_scn));
} else if (LogFlashbackState::FLASHBACK_RECONFIRM == flashback_state_) {
ret = OB_EAGAIN;
PALF_LOG(WARN, "can not flashback a reconfirming leader", K(ret), KPC(this),
K(mode_version), K(flashback_scn));
} else if (LogFlashbackState::FLASHBACK_SUCCESS == flashback_state_) {
const SCN &curr_end_scn = get_end_scn();
const SCN &curr_max_scn = get_max_scn();
if (flashback_scn >= curr_max_scn) {
@ -4292,9 +4298,6 @@ int PalfHandleImpl::flashback(const int64_t mode_version,
}
PALF_EVENT("[END FLASHBACK]", palf_id_, K(ret), KPC(this), K(mode_version),
K(flashback_scn), K(timeout_us), K(curr_end_scn), K(curr_max_scn), K(time_guard));
FLOG_INFO("[END FLASHBACK PALF_DUMP]", K(ret), K_(palf_id), K_(self), "[SlidingWindow]", sw_,
"[StateMgr]", state_mgr_, "[ConfigMgr]", config_mgr_, "[ModeMgr]", mode_mgr_,
"[LogEngine]", log_engine_, "[Reconfirm]", reconfirm_);
plugins_.record_flashback_event(palf_id_, mode_version, flashback_scn, curr_end_scn, curr_max_scn);
break;
} else {
@ -4302,6 +4305,9 @@ int PalfHandleImpl::flashback(const int64_t mode_version,
PALF_LOG(INFO, "flashback not finished", K(ret), KPC(this), K(flashback_scn), K(log_engine_));
}
}
FLOG_INFO("[END FLASHBACK PALF_DUMP]", K(ret), K_(palf_id), K_(self), "[SlidingWindow]", sw_,
"[StateMgr]", state_mgr_, "[ConfigMgr]", config_mgr_, "[ModeMgr]", mode_mgr_,
"[LogEngine]", log_engine_, "[Reconfirm]", reconfirm_);
}
if (OB_SUCCESS == lock_ret) {
flashback_lock_.unlock();
@ -4369,12 +4375,14 @@ int PalfHandleImpl::inner_flashback(const share::SCN &flashback_scn)
PALF_LOG(ERROR, "PalfHandleImpl not inited", KPC(this));
} else if (state_mgr_.is_leader_reconfirm()) {
PALF_LOG(INFO, "can not do flashback in leader reconfirm state", KPC(this), K(flashback_scn));
flashback_state_ = LogFlashbackState::FLASHBACK_RECONFIRM;
} else if (OB_FAIL(get_block_id_by_scn_for_flashback_(flashback_scn, start_block))
&& OB_ENTRY_NOT_EXIST != ret) {
PALF_LOG(ERROR, "get_block_id_by_scn_for_flashback_ failed", K(ret), KPC(this), K(flashback_scn));
} else if (OB_ENTRY_NOT_EXIST == ret) {
ret = OB_SUCCESS;
PALF_LOG(WARN, "there is no log on disk, flashback successfully", K(ret), KPC(this), K(flashback_scn));
flashback_state_ = LogFlashbackState::FLASHBACK_SUCCESS;
} else if (FALSE_IT(start_lsn_of_block.val_ = start_block * PALF_BLOCK_SIZE)) {
} else if (OB_FAIL(log_engine_.begin_flashback(start_lsn_of_block))) {
PALF_LOG(ERROR, "LogEngine begin_flashback failed", K(ret), K(start_lsn_of_block));
@ -4384,8 +4392,10 @@ int PalfHandleImpl::inner_flashback(const share::SCN &flashback_scn)
} else if (OB_FAIL(log_engine_.end_flashback(start_lsn_of_block))) {
PALF_LOG(ERROR, "LogEngine end_flashback failed", K(ret), K(start_lsn_of_block), K(flashback_scn));
} else {
flashback_state_ = LogFlashbackState::FLASHBACK_SUCCESS;
PALF_LOG(INFO, "inner_flashback success", K(ret), KPC(this), K(flashback_scn));
}
flashback_state_ = (OB_FAIL(ret))? LogFlashbackState::FLASHBACK_FAILED: flashback_state_;
return ret;
}

View File

@ -1247,6 +1247,13 @@ private:
typedef common::ObSpinLock SpinLock;
typedef common::ObSpinLockGuard SpinLockGuard;
typedef common::RWLock::WLockGuardWithTimeout WLockGuardWithTimeout;
enum LogFlashbackState
{
FLASHBACK_INIT = 0,
FLASHBACK_SUCCESS = 1,
FLASHBACK_FAILED = 2,
FLASHBACK_RECONFIRM = 3,
};
private:
mutable RWLock lock_;
char log_dir_[common::MAX_PATH_SIZE];
@ -1300,7 +1307,7 @@ private:
// a spin lock for single replica mutex
SpinLock flashback_lock_;
int64_t last_dump_info_time_us_;
bool is_flashback_done_;
LogFlashbackState flashback_state_;
int64_t last_check_sync_time_us_;
int64_t last_renew_loc_time_us_;
int64_t last_print_in_sync_time_us_;