Fix error in the preparation phase of snapshot creation and other bugfix
This commit is contained in:
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@ -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));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|||||||
@ -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));
|
||||||
|
|||||||
@ -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;
|
||||||
}
|
}
|
||||||
|
|||||||
@ -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));
|
||||||
}
|
}
|
||||||
|
|||||||
@ -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_));
|
||||||
|
|||||||
@ -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()(
|
||||||
|
|||||||
Reference in New Issue
Block a user