[to #49019445] add exception log

This commit is contained in:
obdev
2023-04-13 10:43:23 +00:00
committed by ob-robot
parent 26be0d3275
commit ee3da76084
10 changed files with 95 additions and 76 deletions

View File

@ -308,7 +308,7 @@ int ObApplyStatus::init(const share::ObLSID &id,
max_applied_cb_scn_.reset();
get_info_debug_time_ = OB_INVALID_TIMESTAMP;
try_wrlock_debug_time_ = OB_INVALID_TIMESTAMP;
fs_cb_ = ObApplyFsCb(this);
IGNORE_RETURN new (&fs_cb_) ObApplyFsCb(this);
is_in_stop_state_ = false;
if (OB_FAIL(palf_handle_.register_file_size_cb(&fs_cb_))) {
CLOG_LOG(ERROR, "failed to register cb", K(ret), K(id));

View File

@ -604,7 +604,7 @@ int ObReplayStatus::init(const share::ObLSID &id,
check_enable_debug_time_ = OB_INVALID_TIMESTAMP;
palf_env_ = palf_env;
rp_sv_ = rp_sv;
fs_cb_ = ObReplayFsCb(this);
IGNORE_RETURN new (&fs_cb_) ObReplayFsCb(this);
is_inited_ = true;
if (OB_FAIL(palf_handle_.register_file_size_cb(&fs_cb_))) {
CLOG_LOG(ERROR, "failed to register cb", K(ret));
@ -1241,7 +1241,6 @@ int ObReplayStatus::check_replay_barrier(ObLogReplayTask *replay_task,
const int64_t replay_queue_idx)
{
int ret = OB_SUCCESS;
int64_t replay_hint = replay_task->replay_hint_;
if (!is_inited_) {
ret = OB_NOT_INIT;
CLOG_LOG(ERROR, "replay status not inited", K(ret));
@ -1250,6 +1249,7 @@ int ObReplayStatus::check_replay_barrier(ObLogReplayTask *replay_task,
ret = OB_INVALID_ARGUMENT;
CLOG_LOG(ERROR, "check_replay_barrier invalid argument", KP(replay_task), K(replay_queue_idx));
} else if (replay_task->is_pre_barrier_) {
int64_t replay_hint = replay_task->replay_hint_;
int64_t nv = -1;
if (NULL == (replay_log_buf = static_cast<ObLogReplayBuffer *>(replay_task->log_buf_))) {
ret = OB_ERR_UNEXPECTED;
@ -1394,7 +1394,7 @@ int ObReplayStatus::diagnose(ReplayDiagnoseInfo &diagnose_info)
}
if (OB_SUCC(ret) || OB_STATE_NOT_MATCH == ret) {
ret = OB_SUCCESS;
if (OB_FAIL(diagnose_info.diagnose_str_.append_fmt("is_enabled:%s"
if (OB_FAIL(diagnose_info.diagnose_str_.append_fmt("is_enabled:%s; "
"ret:%d; "
"min_unreplayed_lsn:%ld; "
"min_unreplayed_scn:%lu; "

View File

@ -437,6 +437,7 @@ public:
lsn_.reset();
scn_.set_min();
log_type_ = ObLogBaseType::INVALID_LOG_BASE_TYPE;
replay_hint_ = 0;
is_submit_err_ = false;
err_ts_ = 0;
err_ret_ = common::OB_SUCCESS;

View File

@ -401,7 +401,7 @@ int ObTenantRecoveryReportor::update_replayable_point_from_meta_()
max_replayable_point = replayable_point;
}
}
if (OB_SUCC(ret)) {
if (OB_ITER_END == ret) {
logservice::ObLogService *log_service = MTL(logservice::ObLogService*);
if (OB_FAIL(log_service->update_replayable_point(replayable_point))) {
LOG_WARN("logservice update_replayable_point failed", KR(ret), K(replayable_point));

View File

@ -646,8 +646,9 @@ int ObMultipleMerge::get_next_aggregate_row(ObDatumRow *&row)
void ObMultipleMerge::report_tablet_stat()
{
if (0 == access_ctx_->table_store_stat_.physical_read_cnt_ &&
0 == access_ctx_->table_store_stat_.micro_access_cnt_) {
if (OB_ISNULL(access_ctx_) || OB_ISNULL(access_param_)) {
} else if (0 == access_ctx_->table_store_stat_.physical_read_cnt_
&& (0 == access_ctx_->table_store_stat_.micro_access_cnt_ || !access_param_->iter_param_.enable_pd_blockscan())) {
// empty query, ignore it
} else {
int tmp_ret = OB_SUCCESS;
@ -657,7 +658,7 @@ void ObMultipleMerge::report_tablet_stat()
tablet_stat.query_cnt_ = 1;
tablet_stat.scan_logical_row_cnt_ = access_ctx_->table_store_stat_.logical_read_cnt_;
tablet_stat.scan_physical_row_cnt_ = access_ctx_->table_store_stat_.physical_read_cnt_;
tablet_stat.scan_micro_block_cnt_ = access_ctx_->table_store_stat_.micro_access_cnt_;
tablet_stat.scan_micro_block_cnt_ = access_param_->iter_param_.enable_pd_blockscan() ? access_ctx_->table_store_stat_.micro_access_cnt_ : 0;
tablet_stat.pushdown_micro_block_cnt_ = access_ctx_->table_store_stat_.pushdown_micro_access_cnt_;
if (OB_TMP_FAIL(MTL(storage::ObTenantTabletStatMgr *)->report_stat(tablet_stat))) {
STORAGE_LOG_RET(WARN, tmp_ret, "failed to report tablet stat", K(tmp_ret), K(tablet_stat));

View File

@ -1246,10 +1246,11 @@ int ObPartitionMinorRowMergeIter::collect_tnode_dml_stat(
if (OB_ISNULL(table_)) {
// do nothing
} else if (OB_UNLIKELY(!table_->is_memtable())) {
} else if (OB_UNLIKELY(!table_->is_data_memtable() ||
typeid(row_iter_) != typeid(memtable::ObMemtableMultiVersionScanIterator))) {
ret = OB_NOT_SUPPORTED;
LOG_WARN("only support to get mt stat from memtable", K(ret), KPC(table_));
} else if (OB_ISNULL(iter = reinterpret_cast<memtable::ObMemtableMultiVersionScanIterator *>(row_iter_))) {
} else if (OB_ISNULL(iter = static_cast<memtable::ObMemtableMultiVersionScanIterator *>(row_iter_))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get unexpected null row iter", K(ret), KPC(row_iter_));
} else if (OB_FAIL(iter->get_tnode_stat(tnode_stat))) {

View File

@ -1161,6 +1161,8 @@ int ObPartitionMinorMerger::merge_partition(ObTabletMergeCtx &ctx, const int64_t
}
} else if (OB_FAIL(close())){
STORAGE_LOG(WARN, "failed to close partition merger", K(ret));
} else if (ctx.param_.tablet_id_.is_special_merge_tablet()) {
// do nothing
} else {
int tmp_ret = OB_SUCCESS;
if (OB_TMP_FAIL(collect_merge_stat(merge_param.merge_type_, merge_helper, ctx))) {

View File

@ -680,11 +680,16 @@ int ObMemtableCtx::sync_log_succ(const SCN scn, const ObCallbackScope &callbacks
void ObMemtableCtx::sync_log_fail(const ObCallbackScope &callbacks)
{
int ret = OB_SUCCESS;
if (!callbacks.is_empty()) {
set_partial_rollbacked();
}
if (OB_SUCCESS == ATOMIC_LOAD(&end_code_)) {
log_gen_.sync_log_fail(callbacks);
if (OB_FAIL(reuse_log_generator_())) {
TRANS_LOG(ERROR, "fail to reset log generator", K(ret));
} else {
log_gen_.sync_log_fail(callbacks);
}
} else {
if (!callbacks.is_empty()) {
TRANS_LOG(INFO, "No memtable callbacks because of trans_end", K(end_code_), KPC(ctx_));

View File

@ -42,6 +42,7 @@ public:
const common::ObIArray<ObITable *> &tables,
const int64_t start_pos = 0);
bool empty() const { return count_ <= 0; }
bool is_valid() const { return is_inited_ && count_ > 0; }
int64_t count() const { return count_; }
virtual void destroy();

View File

@ -1341,79 +1341,87 @@ int64_t ObTabletTableStore::to_string(char *buf, const int64_t buf_len) const
J_KV(KP(this), KP_(tablet_ptr), K_(major_tables), K_(minor_tables), K_(memtables), K_(is_ready_for_read));
J_COMMA();
J_ARRAY_START();
for (int64_t i = 0; i < major_tables_.count_; ++i) {
ObITable *table = major_tables_[i];
if (NULL != table && table->is_sstable()) {
J_OBJ_START();
J_KV(K(i), "ptr", table, "type", ObITable::get_table_type_name(table->get_key().table_type_),
"tablet_id", table->get_key().tablet_id_,
"scn_range", table->get_key().scn_range_,
"ref", table->get_ref(),
"snapshot_version", table->get_snapshot_version(),
"max_merge_version", static_cast<ObSSTable *>(table)->get_max_merged_trans_version(),
"table_mode_flag", static_cast<ObSSTable*>(table)->get_meta().get_basic_meta().table_mode_.mode_flag_);
J_OBJ_END();
J_COMMA();
if (major_tables_.is_valid()) {
for (int64_t i = 0; i < major_tables_.count_; ++i) {
ObITable *table = major_tables_[i];
if (NULL != table && table->is_sstable()) {
J_OBJ_START();
J_KV(K(i), "ptr", table, "type", ObITable::get_table_type_name(table->get_key().table_type_),
"tablet_id", table->get_key().tablet_id_,
"scn_range", table->get_key().scn_range_,
"ref", table->get_ref(),
"snapshot_version", table->get_snapshot_version(),
"max_merge_version", static_cast<ObSSTable *>(table)->get_max_merged_trans_version(),
"table_mode_flag", static_cast<ObSSTable*>(table)->get_meta().get_basic_meta().table_mode_.mode_flag_);
J_OBJ_END();
J_COMMA();
}
}
}
for (int64_t i = 0; i < minor_tables_.count_; ++i) {
ObITable *table = minor_tables_[i];
if (NULL != table && table->is_sstable()) {
J_OBJ_START();
J_KV(K(i), "ptr", table, "type", ObITable::get_table_type_name(table->get_key().table_type_),
"tablet_id", table->get_key().tablet_id_,
"scn_range", table->get_key().scn_range_,
"ref", table->get_ref(),
"contain_uncommitted_row", static_cast<ObSSTable *>(table)->get_meta().contain_uncommitted_row() ? "yes" : "no",
"max_merge_version", static_cast<ObSSTable *>(table)->get_max_merged_trans_version(),
"upper_trans_version", static_cast<ObSSTable *>(table)->get_upper_trans_version(),
"table_mode_flag", static_cast<ObSSTable*>(table)->get_meta().get_basic_meta().table_mode_.mode_flag_);
J_OBJ_END();
J_COMMA();
if (minor_tables_.is_valid()) {
for (int64_t i = 0; i < minor_tables_.count_; ++i) {
ObITable *table = minor_tables_[i];
if (NULL != table && table->is_sstable()) {
J_OBJ_START();
J_KV(K(i), "ptr", table, "type", ObITable::get_table_type_name(table->get_key().table_type_),
"tablet_id", table->get_key().tablet_id_,
"scn_range", table->get_key().scn_range_,
"ref", table->get_ref(),
"contain_uncommitted_row", static_cast<ObSSTable *>(table)->get_meta().contain_uncommitted_row() ? "yes" : "no",
"max_merge_version", static_cast<ObSSTable *>(table)->get_max_merged_trans_version(),
"upper_trans_version", static_cast<ObSSTable *>(table)->get_upper_trans_version(),
"table_mode_flag", static_cast<ObSSTable*>(table)->get_meta().get_basic_meta().table_mode_.mode_flag_);
J_OBJ_END();
J_COMMA();
}
}
}
for (int64_t i = 0; i < ddl_sstables_.count_; ++i) {
ObITable *table = ddl_sstables_[i];
if (NULL != table && table->is_sstable()) {
J_OBJ_START();
J_KV(K(i), "ptr", table, "type", ObITable::get_table_type_name(table->get_key().table_type_),
"tablet_id", table->get_key().tablet_id_,
"scn_range", table->get_key().scn_range_,
"ref", table->get_ref(),
"max_merge_version", static_cast<ObSSTable *>(table)->get_max_merged_trans_version(),
"table_mode_flag", static_cast<ObSSTable*>(table)->get_meta().get_basic_meta().table_mode_.mode_flag_);
J_OBJ_END();
J_COMMA();
if (ddl_sstables_.is_valid()) {
for (int64_t i = 0; i < ddl_sstables_.count_; ++i) {
ObITable *table = ddl_sstables_[i];
if (NULL != table && table->is_sstable()) {
J_OBJ_START();
J_KV(K(i), "ptr", table, "type", ObITable::get_table_type_name(table->get_key().table_type_),
"tablet_id", table->get_key().tablet_id_,
"scn_range", table->get_key().scn_range_,
"ref", table->get_ref(),
"max_merge_version", static_cast<ObSSTable *>(table)->get_max_merged_trans_version(),
"table_mode_flag", static_cast<ObSSTable*>(table)->get_meta().get_basic_meta().table_mode_.mode_flag_);
J_OBJ_END();
J_COMMA();
}
}
}
for (int64_t i = 0; i < ddl_mem_sstables_.count_; ++i) {
ObITable *table = ddl_mem_sstables_[i];
if (NULL != table && table->is_sstable()) {
J_OBJ_START();
J_KV(K(i), "ptr", table, "type", ObITable::get_table_type_name(table->get_key().table_type_),
"tablet_id", table->get_key().tablet_id_,
"scn_range", table->get_key().scn_range_,
"ref", table->get_ref(),
"max_merge_version", static_cast<ObSSTable *>(table)->get_max_merged_trans_version(),
"table_mode_flag", static_cast<ObSSTable*>(table)->get_meta().get_basic_meta().table_mode_.mode_flag_);
J_OBJ_END();
J_COMMA();
if (ddl_mem_sstables_.is_valid()) {
for (int64_t i = 0; i < ddl_mem_sstables_.count_; ++i) {
ObITable *table = ddl_mem_sstables_[i];
if (NULL != table && table->is_sstable()) {
J_OBJ_START();
J_KV(K(i), "ptr", table, "type", ObITable::get_table_type_name(table->get_key().table_type_),
"tablet_id", table->get_key().tablet_id_,
"scn_range", table->get_key().scn_range_,
"ref", table->get_ref(),
"max_merge_version", static_cast<ObSSTable *>(table)->get_max_merged_trans_version(),
"table_mode_flag", static_cast<ObSSTable*>(table)->get_meta().get_basic_meta().table_mode_.mode_flag_);
J_OBJ_END();
J_COMMA();
}
}
}
for (int64_t i = 0; i < extend_tables_.count_; ++i) {
ObITable *table = extend_tables_[i];
if (NULL != table && table->is_sstable()) {
J_OBJ_START();
J_KV(K(i), "ptr", table, "type", ObITable::get_table_type_name(table->get_key().table_type_),
"tablet_id", table->get_key().tablet_id_,
"scn_range", table->get_key().scn_range_,
"ref", table->get_ref(),
"max_merge_version", static_cast<ObSSTable *>(table)->get_max_merged_trans_version(),
"table_mode_flag", static_cast<ObSSTable*>(table)->get_meta().get_basic_meta().table_mode_.mode_flag_);
J_OBJ_END();
J_COMMA();
if (extend_tables_.is_valid()) {
for (int64_t i = 0; i < extend_tables_.count_; ++i) {
ObITable *table = extend_tables_[i];
if (NULL != table && table->is_sstable()) {
J_OBJ_START();
J_KV(K(i), "ptr", table, "type", ObITable::get_table_type_name(table->get_key().table_type_),
"tablet_id", table->get_key().tablet_id_,
"scn_range", table->get_key().scn_range_,
"ref", table->get_ref(),
"max_merge_version", static_cast<ObSSTable *>(table)->get_max_merged_trans_version(),
"table_mode_flag", static_cast<ObSSTable*>(table)->get_meta().get_basic_meta().table_mode_.mode_flag_);
J_OBJ_END();
J_COMMA();
}
}
}
J_ARRAY_END();