Fix error in the preparation phase of snapshot creation and other bugfix

This commit is contained in:
obdev
2024-02-08 10:34:55 +00:00
committed by ob-robot
parent efa7bd8083
commit 2f77d05cff
7 changed files with 29 additions and 9 deletions

View File

@ -549,6 +549,10 @@ int ObCloneScheduler::clone_wait_create_snapshot_for_fork_tenant(const share::Ob
need_wait = true; need_wait = true;
} else if (ObTenantSnapStatus::CLONING == item.get_status()) { } else if (ObTenantSnapStatus::CLONING == item.get_status()) {
// no need to update snapshot status // no need to update snapshot status
} else if (ObTenantSnapStatus::FAILED == item.get_status()) {
ret = OB_ERR_TENANT_SNAPSHOT;
LOG_WARN("create snapshot for fork tenant failed", KR(ret), K(source_tenant_id),
K(tenant_snapshot_id), K(item));
} else if (ObTenantSnapStatus::NORMAL != item.get_status()) { } else if (ObTenantSnapStatus::NORMAL != item.get_status()) {
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
LOG_WARN("invalid status for fork tenant snapshot", KR(ret), K(source_tenant_id), LOG_WARN("invalid status for fork tenant snapshot", KR(ret), K(source_tenant_id),
@ -1208,6 +1212,7 @@ int ObCloneScheduler::clone_recycle_failed_job(const share::ObCloneJob &job)
K(ret), K(ret),
"cur_clone_status", status_str); "cur_clone_status", status_str);
} }
LOG_INFO("[RESTORE] clone recycle failed job", KR(ret), K(job));
return ret; return ret;
} }

View File

@ -481,6 +481,7 @@ int ObTenantCloneUtil::release_source_tenant_resource_of_clone_job(common::ObISQ
LOG_WARN("fail to recycle tenant snapshot ls replicas", KR(ret), K(clone_job)); LOG_WARN("fail to recycle tenant snapshot ls replicas", KR(ret), K(clone_job));
} else { } else {
need_notify_tenant_snapshot_scheduler = true; need_notify_tenant_snapshot_scheduler = true;
LOG_INFO("release source tenant resource", KR(ret), K(clone_job));
} }
} }
} }

View File

@ -348,6 +348,7 @@ int ObTenantSnapshotScheduler::build_tenant_snapshot_delete_job_(const ObTenantS
//************************************************************************* //*************************************************************************
//Last, insert data into inner table:__all_tenant_snapshot_ls_replica //Last, insert data into inner table:__all_tenant_snapshot_ls_replica
//************************************************************************* //*************************************************************************
ERRSIM_POINT_DEF(ERRSIM_PREPARE_CREATE_SNAPSHOT_ERROR);
int ObTenantSnapshotScheduler::prepare_for_create_tenant_snapshot_( int ObTenantSnapshotScheduler::prepare_for_create_tenant_snapshot_(
const ObCreateSnapshotJob &create_job) const ObCreateSnapshotJob &create_job)
{ {
@ -388,6 +389,8 @@ int ObTenantSnapshotScheduler::prepare_for_create_tenant_snapshot_(
LOG_WARN("failed to insert snapshot ls items", KR(ret), K(snap_ls_items)); LOG_WARN("failed to insert snapshot ls items", KR(ret), K(snap_ls_items));
} else if (OB_FAIL(second_table_op.insert_tenant_snap_ls_replica_simple_items(ls_replica_items))) { } else if (OB_FAIL(second_table_op.insert_tenant_snap_ls_replica_simple_items(ls_replica_items))) {
LOG_WARN("failed to insert snapshot ls replica simple items", KR(ret), K(ls_replica_items)); LOG_WARN("failed to insert snapshot ls replica simple items", KR(ret), K(ls_replica_items));
} else if (OB_UNLIKELY(ERRSIM_PREPARE_CREATE_SNAPSHOT_ERROR)) {
ret = ERRSIM_PREPARE_CREATE_SNAPSHOT_ERROR;
} }
if (trans.is_started()) { if (trans.is_started()) {
int tmp_ret = OB_SUCCESS; int tmp_ret = OB_SUCCESS;
@ -531,8 +534,8 @@ int ObTenantSnapshotScheduler::get_ls_valid_replicas_(
const ObIArray<ObLSReplica> &replicas = ls_info.get_replicas(); const ObIArray<ObLSReplica> &replicas = ls_info.get_replicas();
ARRAY_FOREACH_N(replicas, j, cnt) { ARRAY_FOREACH_N(replicas, j, cnt) {
const ObLSReplica &replica = replicas.at(j); const ObLSReplica &replica = replicas.at(j);
if (replica.get_replica_type() == REPLICA_TYPE_FULL if (REPLICA_TYPE_FULL == replica.get_replica_type()
&& replica.get_replica_status() == REPLICA_STATUS_NORMAL) { && REPLICA_STATUS_NORMAL == replica.get_replica_status()) {
if (OB_FAIL(valid_replicas.push_back(&replica))) { if (OB_FAIL(valid_replicas.push_back(&replica))) {
LOG_WARN("fail to push back", KR(ret), K(replica), KP(&replica)); LOG_WARN("fail to push back", KR(ret), K(replica), KP(&replica));
} }
@ -552,6 +555,7 @@ int ObTenantSnapshotScheduler::get_ls_valid_replicas_(
return ret; return ret;
} }
ERRSIM_POINT_DEF(ERRSIM_PROCESS_CREATE_SNAPSHOT_ERROR);
int ObTenantSnapshotScheduler::process_create_tenant_snapshot_( int ObTenantSnapshotScheduler::process_create_tenant_snapshot_(
const ObCreateSnapshotJob &create_job) const ObCreateSnapshotJob &create_job)
{ {
@ -604,6 +608,8 @@ int ObTenantSnapshotScheduler::process_create_tenant_snapshot_(
} else if (OB_FAIL(check_log_archive_finish_(user_tenant_id, snapshot_scn, need_wait_archive_finish))) { } else if (OB_FAIL(check_log_archive_finish_(user_tenant_id, snapshot_scn, need_wait_archive_finish))) {
LOG_WARN("failed to execute check_log_archive_finish", KR(ret), K(user_tenant_id), K(snapshot_scn)); LOG_WARN("failed to execute check_log_archive_finish", KR(ret), K(user_tenant_id), K(snapshot_scn));
} else if (need_wait_archive_finish) { } else if (need_wait_archive_finish) {
} else if (OB_UNLIKELY(ERRSIM_PROCESS_CREATE_SNAPSHOT_ERROR)) {
ret = ERRSIM_PROCESS_CREATE_SNAPSHOT_ERROR;
} else if (OB_FAIL(finish_create_tenant_snapshot_(tenant_snapshot_id, user_tenant_id, } else if (OB_FAIL(finish_create_tenant_snapshot_(tenant_snapshot_id, user_tenant_id,
clog_start_scn, snapshot_scn))) { clog_start_scn, snapshot_scn))) {
LOG_WARN("failed to execute finish_create_tenant_snapshot", KR(ret), K(tenant_snapshot_id), LOG_WARN("failed to execute finish_create_tenant_snapshot", KR(ret), K(tenant_snapshot_id),
@ -803,9 +809,9 @@ int ObTenantSnapshotScheduler::check_create_tenant_snapshot_result_(
if (OB_SUCC(ret)) { if (OB_SUCC(ret)) {
int32_t failed_member = failed_addrs.count(); int32_t failed_member = failed_addrs.count();
int32_t creating_member = processing_addrs.count(); int32_t creating_member = processing_addrs.count();
if (arbitration_service_status.is_enable_like()) { // if (arbitration_service_status.is_enable_like()) {
succ_member++; // succ_member++;
} // }
// attention: // attention:
// in normal case, creating_member + succ_member + failed_member == paxos_replica_num // in normal case, creating_member + succ_member + failed_member == paxos_replica_num
// in transfer case, it might happened that creating_member + succ_member + failed_member > paxos_replica_num // in transfer case, it might happened that creating_member + succ_member + failed_member > paxos_replica_num
@ -1145,6 +1151,9 @@ int ObTenantSnapshotScheduler::process_delete_tenant_snapshots_(
const ObTenantSnapshotID &tenant_snapshot_id = delete_job.get_tenant_snapshot_id(); const ObTenantSnapshotID &tenant_snapshot_id = delete_job.get_tenant_snapshot_id();
if (OB_FAIL(table_op.get_tenant_snap_related_addrs(tenant_snapshot_id, addr_array))) { if (OB_FAIL(table_op.get_tenant_snap_related_addrs(tenant_snapshot_id, addr_array))) {
LOG_WARN("failed to get snapshot related addrs", KR(ret), K(tenant_snapshot_id)); LOG_WARN("failed to get snapshot related addrs", KR(ret), K(tenant_snapshot_id));
} else if (addr_array.empty()) {
//may be that creating the snapshot failed during the preparation phase.
LOG_INFO("addr_array in __all_tenant_snapshot_ls_replica is empty", KR(ret), K(tenant_snapshot_id));
} else if (OB_FAIL(send_delete_tenant_snapshot_rpc_(tenant_snapshot_id, user_tenant_id, addr_array))) { } else if (OB_FAIL(send_delete_tenant_snapshot_rpc_(tenant_snapshot_id, user_tenant_id, addr_array))) {
LOG_WARN("failed to send delete snapshot rpc", KR(ret), K(tenant_snapshot_id), LOG_WARN("failed to send delete snapshot rpc", KR(ret), K(tenant_snapshot_id),
K(user_tenant_id), K(addr_array)); K(user_tenant_id), K(addr_array));

View File

@ -246,6 +246,7 @@ ObCloneJob::ObCloneJob() :
restore_scn_(), restore_scn_(),
status_(ObTenantCloneStatus::Status::CLONE_MAX_STATUS), status_(ObTenantCloneStatus::Status::CLONE_MAX_STATUS),
job_type_(ObTenantCloneJobType::CLONE_JOB_MAX_TYPE), job_type_(ObTenantCloneJobType::CLONE_JOB_MAX_TYPE),
ret_code_(OB_SUCCESS),
allocator_("CloneJob") allocator_("CloneJob")
{} {}
@ -344,7 +345,7 @@ int ObCloneJob::assign(const ObCloneJob &other)
restore_scn_ = other.restore_scn_; restore_scn_ = other.restore_scn_;
status_ = other.status_; status_ = other.status_;
job_type_ = other.job_type_; job_type_ = other.job_type_;
ret_code_ = ret_code_; ret_code_ = other.ret_code_;
} }
return ret; return ret;
} }

View File

@ -879,7 +879,8 @@ int ObTenantSnapshotTableOperator::delete_tenant_snap_item(const ObTenantSnapsho
LOG_WARN("splice delete sql failed", KR(ret)); LOG_WARN("splice delete sql failed", KR(ret));
} else if (OB_FAIL(proxy_->write(gen_meta_tenant_id(user_tenant_id_), sql.ptr(), affected_rows))) { } else if (OB_FAIL(proxy_->write(gen_meta_tenant_id(user_tenant_id_), sql.ptr(), affected_rows))) {
LOG_WARN("exec sql failed", KR(ret), K(gen_meta_tenant_id(user_tenant_id_)), K(sql)); LOG_WARN("exec sql failed", KR(ret), K(gen_meta_tenant_id(user_tenant_id_)), K(sql));
} else if (!is_single_row(affected_rows)) { } else if (!is_zero_row(affected_rows)
&& !is_single_row(affected_rows)) {
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected affected rows", KR(ret), K(affected_rows)); LOG_WARN("unexpected affected rows", KR(ret), K(affected_rows));
} }

View File

@ -182,12 +182,15 @@ int ObCloneTenantResolver::resolve_option_list_(const ParseNode *node,
} else { } else {
for (int64_t i = 0; OB_SUCC(ret) && i < node->num_child_; i++) { for (int64_t i = 0; OB_SUCC(ret) && i < node->num_child_; i++) {
const ParseNode *option_node = node->children_[i]; const ParseNode *option_node = node->children_[i];
if (OB_UNLIKELY(NULL == option_node)) { if (OB_ISNULL(option_node)) {
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
LOG_ERROR("invalid node", KR(ret)); LOG_ERROR("invalid node", KR(ret));
} else if (OB_UNLIKELY(1 != option_node->num_child_)) { } else if (OB_UNLIKELY(1 != option_node->num_child_)) {
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
LOG_ERROR("invalid node", KR(ret)); LOG_ERROR("invalid node", KR(ret));
} else if (OB_ISNULL(option_node->children_[0])) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("invalid node", KR(ret));
} else if (T_RESOURCE_POOL_LIST == option_node->type_) { } else if (T_RESOURCE_POOL_LIST == option_node->type_) {
resource_pool_name.assign_ptr((char *)(option_node->children_[0]->str_value_), resource_pool_name.assign_ptr((char *)(option_node->children_[0]->str_value_),
static_cast<int32_t>(option_node->children_[0]->str_len_)); static_cast<int32_t>(option_node->children_[0]->str_len_));

View File

@ -946,7 +946,7 @@ void ObTenantSnapshotService::notify_unit_is_deleting()
ObThreadCondGuard guard(cond_); ObThreadCondGuard guard(cond_);
cond_.signal(); cond_.signal();
} }
LOG_INFO("try_set_running_mode_to_gc finished", KR(ret), KPC(this)); LOG_INFO("notify_unit_is_deleting finished", KR(ret), KPC(this));
} }
bool ObTenantSnapshotService::DumpTenantSnapInfoFunctor::operator()( bool ObTenantSnapshotService::DumpTenantSnapInfoFunctor::operator()(