fix restore report -4002 because mismatch schema version

This commit is contained in:
oceanoverflow 2022-11-11 09:39:42 +00:00 committed by wangzelin.wzl
parent 44b9dbca38
commit 384903cef3
15 changed files with 362 additions and 26 deletions

View File

@ -17,6 +17,7 @@
#include "rootserver/backup/ob_backup_service.h"
#include "share/ls/ob_ls_status_operator.h"
#include "storage/backup/ob_backup_operator.h"
#include "rootserver/ob_rs_event_history_table_operator.h"
namespace oceanbase
{
@ -119,6 +120,12 @@ int ObBackupDataLSTaskMgr::gen_and_add_task_()
case ObBackupDataTaskType::Type::BACKUP_DATA_SYS:
case ObBackupDataTaskType::Type::BACKUP_DATA_MINOR:
case ObBackupDataTaskType::Type::BACKUP_DATA_MAJOR: {
if (ObBackupDataTaskType::Type::BACKUP_DATA_MAJOR == ls_attr_->task_type_.type_) {
#ifdef ERRSIM
ROOTSERVICE_EVENT_ADD("backup", "before_backup_major_sstable");
DEBUG_SYNC(BEFORE_BACKUP_MAJOR_SSTABLE);
#endif
}
if (OB_FAIL(gen_and_add_backup_data_task_())) {
LOG_WARN("[DATA_BACKUP]failed to gen and add backup data task", K(ret), K(*ls_attr_));
}

View File

@ -357,6 +357,7 @@ class ObString;
ACT(BEFORE_CREATE_USER_LS,)\
ACT(BEFORE_BACKUP_META,)\
ACT(BEFORE_BACKUP_DATA,)\
ACT(BEFORE_BACKUP_MAJOR_SSTABLE,)\
ACT(BEFORE_BACKUP_BUILD_INDEX,)\
ACT(BEFORE_BACKUP_COMPLEMENT_LOG,)\
ACT(BEFORE_BACKUP_FINISH,)\

View File

@ -34,9 +34,6 @@ int ObLSBackupRestoreUtil::read_tablet_meta(const common::ObString &path, const
} else if (BACKUP_TABLET_META != meta_index.meta_key_.meta_type_) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("meta type do not match", K(meta_index));
} else if (!backup_data_type.is_sys_backup() && !backup_data_type.is_minor_backup()) {
ret = OB_ERR_SYS;
LOG_WARN("incorrect backup data type", K(ret), K(backup_data_type));
} else if (OB_ISNULL(buf = reinterpret_cast<char *>(allocator.alloc(meta_index.length_)))) {
ret = OB_ALLOCATE_MEMORY_FAILED;
LOG_WARN("failed to alloc read buf", K(ret), K(meta_index));

View File

@ -2481,6 +2481,7 @@ int ObTabletMigrationTask::build_copy_sstable_info_mgr_()
param.tablet_id_ = copy_tablet_ctx_->tablet_id_;
param.is_leader_restore_ = false;
param.local_rebuild_seq_ = ctx_->local_rebuild_seq_;
param.meta_index_store_ = nullptr;
param.second_meta_index_store_ = nullptr;
param.need_check_seq_ = true;
param.restore_base_info_ = nullptr;
@ -2550,6 +2551,7 @@ int ObTabletMigrationTask::generate_tablet_copy_finish_task_(
ObLS *ls = nullptr;
ObTabletMigrationDag *tablet_migration_dag = nullptr;
observer::ObIMetaReport *reporter = GCTX.ob_service_;
const ObTabletRestoreAction::ACTION restore_action = ObTabletRestoreAction::RESTORE_NONE;
const ObMigrationTabletParam *src_tablet_meta = nullptr;
if (!is_inited_) {
@ -2562,7 +2564,7 @@ int ObTabletMigrationTask::generate_tablet_copy_finish_task_(
LOG_WARN("failed to get ls", K(ret), KPC(ctx_));
} else if (OB_FAIL(ctx_->ha_table_info_mgr_.get_tablet_meta(copy_tablet_ctx_->tablet_id_, src_tablet_meta))) {
LOG_WARN("failed to get src tablet meta", K(ret), KPC(copy_tablet_ctx_));
} else if (OB_FAIL(tablet_copy_finish_task->init(copy_tablet_ctx_->tablet_id_, ls, reporter, src_tablet_meta))) {
} else if (OB_FAIL(tablet_copy_finish_task->init(copy_tablet_ctx_->tablet_id_, ls, reporter, restore_action, src_tablet_meta))) {
LOG_WARN("failed to init tablet copy finish task", K(ret), KPC(ctx_), KPC(copy_tablet_ctx_));
}
return ret;
@ -4034,6 +4036,7 @@ int ObLSMigrationUtils::init_ha_tablets_builder(
param.meta_index_store_ = nullptr;
param.need_check_seq_ = true;
param.restore_base_info_ = nullptr;
param.restore_action_ = ObTabletRestoreAction::RESTORE_NONE;
param.src_info_ = src_info;
param.storage_rpc_ = ls_service->get_storage_rpc();
param.svr_rpc_proxy_ = ls_service->get_storage_rpc_proxy();

View File

@ -1285,11 +1285,13 @@ int ObSysTabletsRestoreTask::init()
storage_rpc_ = ls_restore_dag_net->get_storage_rpc();
meta_index_store_ = ls_restore_dag_net->get_meta_index_store();
second_meta_index_store_ = ls_restore_dag_net->get_second_meta_index_store();
const ObTabletRestoreAction::ACTION &restore_action = ObTabletRestoreAction::ACTION::RESTORE_ALL;
if (OB_FAIL(ObStorageHADagUtils::get_ls(ctx_->arg_.ls_id_, ls_handle_))) {
LOG_WARN("failed to get ls", K(ret), KPC(ctx_));
} else if (OB_FAIL(ObTabletGroupRestoreUtils::init_ha_tablets_builder(
ctx_->arg_.tenant_id_, ctx_->sys_tablet_id_array_, ctx_->arg_.is_leader_, ctx_->src_,
ls_handle_.get_ls(), &ctx_->arg_.restore_base_info_, meta_index_store_, &ctx_->ha_table_info_mgr_,
ls_handle_.get_ls(), &ctx_->arg_.restore_base_info_, restore_action,
meta_index_store_, &ctx_->ha_table_info_mgr_,
ha_tablets_builder_))) {
LOG_WARN("failed to init ha tablets builder", K(ret), KPC(ctx_));
} else {
@ -1388,6 +1390,7 @@ int ObSysTabletsRestoreTask::generate_sys_tablet_restore_dag_()
param.action_ = action;
param.restore_base_info_ = &ctx_->arg_.restore_base_info_;
param.ha_table_info_mgr_ = &ctx_->ha_table_info_mgr_;
param.meta_index_store_ = meta_index_store_;
param.second_meta_index_store_ = second_meta_index_store_;
if (!param.is_valid()) {

View File

@ -72,6 +72,7 @@ void ObPhysicalCopyCtx::reset()
svr_rpc_proxy_ = nullptr;
is_leader_restore_ = false;
restore_base_info_ = nullptr;
meta_index_store_ = nullptr;
second_meta_index_store_ = nullptr;
ha_dag_ = nullptr;
sstable_index_builder_ = nullptr;
@ -90,6 +91,7 @@ ObPhysicalCopyTaskInitParam::ObPhysicalCopyTaskInitParam()
ls_(nullptr),
is_leader_restore_(false),
restore_base_info_(nullptr),
meta_index_store_(nullptr),
second_meta_index_store_(nullptr)
{
}
@ -106,7 +108,9 @@ bool ObPhysicalCopyTaskInitParam::is_valid() const
if (bool_ret) {
if (!is_leader_restore_) {
bool_ret = src_info_.is_valid();
} else if (OB_ISNULL(restore_base_info_) || OB_ISNULL(second_meta_index_store_)) {
} else if (OB_ISNULL(restore_base_info_)
|| OB_ISNULL(meta_index_store_)
|| OB_ISNULL(second_meta_index_store_)) {
bool_ret = false;
}
}
@ -125,6 +129,7 @@ void ObPhysicalCopyTaskInitParam::reset()
ls_ = nullptr;
is_leader_restore_ = false;
restore_base_info_ = nullptr;
meta_index_store_ = nullptr;
second_meta_index_store_ = nullptr;
}
@ -510,6 +515,7 @@ int ObPhysicalCopyTask::build_copy_macro_block_reader_init_param_(
init_param.bandwidth_throttle_ = copy_ctx_->bandwidth_throttle_;
init_param.svr_rpc_proxy_ = copy_ctx_->svr_rpc_proxy_;
init_param.restore_base_info_ = copy_ctx_->restore_base_info_;
init_param.meta_index_store_ = copy_ctx_->meta_index_store_;
init_param.second_meta_index_store_ = copy_ctx_->second_meta_index_store_;
init_param.restore_macro_block_id_mgr_ = copy_ctx_->restore_macro_block_id_mgr_;
init_param.copy_macro_range_info_ = copy_macro_range_info_;
@ -598,6 +604,7 @@ int ObPhysicalCopyFinishTask::init(
copy_ctx_.svr_rpc_proxy_ = svr_rpc_proxy;
copy_ctx_.is_leader_restore_ = init_param.is_leader_restore_;
copy_ctx_.restore_base_info_ = init_param.restore_base_info_;
copy_ctx_.meta_index_store_ = init_param.meta_index_store_;
copy_ctx_.second_meta_index_store_ = init_param.second_meta_index_store_;
copy_ctx_.ha_dag_ = ha_dag;
copy_ctx_.sstable_index_builder_ = &sstable_index_builder_;
@ -1089,6 +1096,7 @@ ObTabletCopyFinishTask::ObTabletCopyFinishTask()
reporter_(nullptr),
ha_dag_(nullptr),
tables_handle_(),
restore_action_(ObTabletRestoreAction::MAX),
src_tablet_meta_(nullptr)
{
@ -1102,21 +1110,24 @@ int ObTabletCopyFinishTask::init(
const common::ObTabletID &tablet_id,
ObLS *ls,
observer::ObIMetaReport *reporter,
const ObTabletRestoreAction::ACTION &restore_action,
const ObMigrationTabletParam *src_tablet_meta)
{
int ret = OB_SUCCESS;
if (is_inited_) {
ret = OB_INIT_TWICE;
LOG_WARN("tablet copy finish task init twice", K(ret));
} else if (!tablet_id.is_valid() || OB_ISNULL(ls) || OB_ISNULL(reporter) || OB_ISNULL(src_tablet_meta)) {
} else if (!tablet_id.is_valid() || OB_ISNULL(ls) || OB_ISNULL(reporter) || OB_ISNULL(src_tablet_meta)
|| !ObTabletRestoreAction::is_valid(restore_action)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("init tablet copy finish task get invalid argument", K(ret), K(tablet_id), KP(ls),
KP(reporter), KP(src_tablet_meta));
KP(reporter), KP(src_tablet_meta), K(restore_action));
} else {
tablet_id_ = tablet_id;
ls_ = ls;
reporter_ = reporter;
ha_dag_ = static_cast<ObStorageHADag *>(this->get_dag());
restore_action_ = restore_action;
src_tablet_meta_ = src_tablet_meta;
is_inited_ = true;
}
@ -1131,8 +1142,12 @@ int ObTabletCopyFinishTask::process()
LOG_WARN("tablet copy finish task do not init", K(ret));
} else if (ha_dag_->get_ha_dag_net_ctx()->is_failed()) {
FLOG_INFO("ha dag net is already failed, skip physical copy finish task", K(tablet_id_), KPC(ha_dag_));
} else if (OB_FAIL(create_new_table_store_())) {
} else if (!ObTabletRestoreAction::is_restore_major(restore_action_)
&& OB_FAIL(create_new_table_store_())) {
LOG_WARN("failed to create new table store", K(ret), K(tablet_id_));
} else if (ObTabletRestoreAction::is_restore_major(restore_action_)
&& OB_FAIL(create_new_table_store_restore_major_())) {
LOG_WARN("failed to create new table store restore major", K(ret), K(tablet_id_));
} else if (OB_FAIL(update_tablet_data_status_())) {
LOG_WARN("failed to update tablet data status", K(ret), K(tablet_id_));
}
@ -1245,6 +1260,82 @@ int ObTabletCopyFinishTask::create_new_table_store_()
return ret;
}
int ObTabletCopyFinishTask::create_new_table_store_restore_major_()
{
int ret = OB_SUCCESS;
ObTabletHandle tablet_handle;
ObTablet *tablet = nullptr;
const bool is_rollback = false;
bool need_merge = false;
ObTenantMetaMemMgr *meta_mem_mgr = nullptr;
if (!is_inited_) {
ret = OB_NOT_INIT;
LOG_WARN("tablet copy finish task do not init", K(ret));
} else if (OB_ISNULL(src_tablet_meta_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("src tablet meta should not be null", K(ret));
} else if (!ObTabletRestoreAction::is_restore_major(restore_action_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("not restore major", K(ret));
} else if (OB_FAIL(ls_->get_tablet(tablet_id_, tablet_handle))) {
LOG_WARN("failed to get tablet", K(ret), K(tablet_id_));
} else if (OB_ISNULL(tablet = tablet_handle.get_obj())) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("tablet should not be NULL", K(ret), K(tablet_id_));
} else if (tables_handle_.empty()) {
if (src_tablet_meta_->table_store_flag_.with_major_sstable()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("should has major sstable", K(ret), K(tablet_id_), K(tables_handle_), KPC(tablet));
} else {
LOG_INFO("tablet do not has sstable", K(ret), K(tablet_id_), K(tables_handle_), KPC(tablet));
}
} else if (1 != tables_handle_.get_count()) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("major tablet should only has one sstable", K(ret), K(tables_handle_));
} else if (OB_ISNULL(meta_mem_mgr = MTL(ObTenantMetaMemMgr *))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("failed to get meta mem mgr from MTL", K(ret));
} else {
ObITable *table_ptr = tables_handle_.get_table(0);
ObTableHandleV2 table_handle_v2;
if (OB_ISNULL(table_ptr)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("table ptr should not be null", K(ret));
} else if (OB_FAIL(table_handle_v2.set_table(table_ptr, meta_mem_mgr, table_ptr->get_key().table_type_))) {
LOG_WARN("failed to set table handle v2", K(ret), KPC(table_ptr));
} else {
int64_t update_snapshot_version = MAX(tablet->get_snapshot_version(), table_ptr->get_key().get_snapshot_version());
int64_t update_multi_version_start = MAX(tablet->get_multi_version_start(), table_ptr->get_key().get_snapshot_version());
ObUpdateTableStoreParam param(table_handle_v2,
update_snapshot_version,
update_multi_version_start,
&src_tablet_meta_->storage_schema_,
ls_->get_rebuild_seq(),
true/*need_report*/,
0/*clog_checkpoint_ts*/,
true/*need_check_sstable*/);
#ifdef ERRSIM
SERVER_EVENT_ADD("storage_ha", "update_major_tablet_table_store",
"tablet_id", tablet_id_.id(),
"old_multi_version_start", tablet->get_multi_version_start(),
"new_multi_version_start", src_tablet_meta_->multi_version_start_,
"old_snapshot_version", tablet->get_snapshot_version(),
"new_snapshot_version", table_ptr->get_key().get_snapshot_version());
#endif
if (FAILEDx(ls_->update_tablet_table_store(tablet_id_, param, tablet_handle))) {
LOG_WARN("failed to build ha tablet new table store", K(ret), K(tablet_id_), K(param));
}
}
if (OB_FAIL(ret)) {
} else if (tablet->get_tablet_meta().has_next_tablet_
&& OB_FAIL(ls_->trim_rebuild_tablet(tablet_id_, is_rollback))) {
LOG_WARN("failed to trim rebuild tablet", K(ret), K(tablet_id_));
}
}
return ret;
}
int ObTabletCopyFinishTask::update_tablet_data_status_()
{
int ret = OB_SUCCESS;

View File

@ -41,7 +41,8 @@ struct ObPhysicalCopyCtx
bool is_valid() const;
void reset();
TO_STRING_KV(K_(tenant_id), K_(ls_id), K_(tablet_id), K_(src_info), KP_(bandwidth_throttle),
KP_(svr_rpc_proxy), K_(is_leader_restore), KP_(restore_base_info), KP_(second_meta_index_store), KP_(ha_dag),
KP_(svr_rpc_proxy), K_(is_leader_restore), KP_(restore_base_info),
KP_(meta_index_store), KP_(second_meta_index_store), KP_(ha_dag),
KP_(sstable_index_builder), KP_(restore_macro_block_id_mgr));
uint64_t tenant_id_;
share::ObLSID ls_id_;
@ -51,6 +52,7 @@ struct ObPhysicalCopyCtx
obrpc::ObStorageRpcProxy *svr_rpc_proxy_;
bool is_leader_restore_;
const ObRestoreBaseInfo *restore_base_info_;
backup::ObBackupMetaIndexStoreWrapper *meta_index_store_;
backup::ObBackupMetaIndexStoreWrapper *second_meta_index_store_;
ObStorageHADag *ha_dag_;
ObSSTableIndexBuilder *sstable_index_builder_;
@ -79,6 +81,7 @@ struct ObPhysicalCopyTaskInitParam final
ObLS *ls_;
bool is_leader_restore_;
const ObRestoreBaseInfo *restore_base_info_;
backup::ObBackupMetaIndexStoreWrapper *meta_index_store_;
backup::ObBackupMetaIndexStoreWrapper *second_meta_index_store_;
DISALLOW_COPY_AND_ASSIGN(ObPhysicalCopyTaskInitParam);
};
@ -207,6 +210,7 @@ public:
const common::ObTabletID &tablet_id,
ObLS *ls,
observer::ObIMetaReport *reporter,
const ObTabletRestoreAction::ACTION &restore_action,
const ObMigrationTabletParam *src_tablet_meta);
virtual int process() override;
VIRTUAL_TO_STRING_KV(K("ObTabletCopyFinishTask"), KP(this));
@ -216,6 +220,7 @@ public:
ObTableHandleV2 &table_handle);
private:
int create_new_table_store_();
int create_new_table_store_restore_major_();
int update_tablet_data_status_();
int check_need_merge_tablet_meta_(
ObTablet *tablet,
@ -232,6 +237,7 @@ private:
observer::ObIMetaReport *reporter_;
ObStorageHADag *ha_dag_;
ObTablesHandleArray tables_handle_;
ObTabletRestoreAction::ACTION restore_action_;
const ObMigrationTabletParam *src_tablet_meta_;
DISALLOW_COPY_AND_ASSIGN(ObTabletCopyFinishTask);
};

View File

@ -91,6 +91,7 @@ ObCopyMacroBlockReaderInitParam::ObCopyMacroBlockReaderInitParam()
bandwidth_throttle_(nullptr),
svr_rpc_proxy_(nullptr),
restore_base_info_(nullptr),
meta_index_store_(nullptr),
second_meta_index_store_(nullptr),
restore_macro_block_id_mgr_(nullptr)
{
@ -107,7 +108,10 @@ bool ObCopyMacroBlockReaderInitParam::is_valid() const
if (bool_ret) {
if (!is_leader_restore_) {
bool_ret = src_info_.is_valid() && OB_NOT_NULL(bandwidth_throttle_) && OB_NOT_NULL(svr_rpc_proxy_);
} else if (OB_ISNULL(restore_base_info_) || OB_ISNULL(second_meta_index_store_) || OB_ISNULL(restore_macro_block_id_mgr_)) {
} else if (OB_ISNULL(restore_base_info_)
|| OB_ISNULL(meta_index_store_)
|| OB_ISNULL(second_meta_index_store_)
|| OB_ISNULL(restore_macro_block_id_mgr_)) {
bool_ret = false;
}
}
@ -125,6 +129,7 @@ void ObCopyMacroBlockReaderInitParam::reset()
bandwidth_throttle_ = nullptr;
svr_rpc_proxy_ = nullptr;
restore_base_info_ = nullptr;
meta_index_store_ = nullptr;
second_meta_index_store_ = nullptr;
restore_macro_block_id_mgr_ = nullptr;
}
@ -144,6 +149,7 @@ int ObCopyMacroBlockReaderInitParam::assign(const ObCopyMacroBlockReaderInitPara
bandwidth_throttle_ = param.bandwidth_throttle_;
svr_rpc_proxy_ = param.svr_rpc_proxy_;
restore_base_info_ = param.restore_base_info_;
meta_index_store_ = param.meta_index_store_;
second_meta_index_store_ = param.second_meta_index_store_;
restore_macro_block_id_mgr_ = param.restore_macro_block_id_mgr_;
copy_macro_range_info_ = param.copy_macro_range_info_;
@ -1139,6 +1145,7 @@ int ObCopySSTableInfoObReader::get_next_tablet_sstable_header(
ObCopySSTableInfoRestoreReader::ObCopySSTableInfoRestoreReader()
: is_inited_(false),
restore_base_info_(nullptr),
restore_action_(ObTabletRestoreAction::MAX),
tablet_id_array_(),
meta_index_store_(nullptr),
tablet_index_(0),
@ -1152,6 +1159,7 @@ ObCopySSTableInfoRestoreReader::ObCopySSTableInfoRestoreReader()
int ObCopySSTableInfoRestoreReader::init(
const share::ObLSID &ls_id,
const ObRestoreBaseInfo &restore_base_info,
const ObTabletRestoreAction::ACTION &restore_action,
const common::ObIArray<common::ObTabletID> &tablet_id_array,
backup::ObBackupMetaIndexStoreWrapper &meta_index_store)
{
@ -1163,9 +1171,10 @@ int ObCopySSTableInfoRestoreReader::init(
LOG_WARN("can not init twice", K(ret));
} else if (OB_UNLIKELY(!ls_id.is_valid())
|| OB_UNLIKELY(!restore_base_info.is_valid())
|| OB_UNLIKELY(tablet_id_array.empty())) {
|| OB_UNLIKELY(tablet_id_array.empty())
|| OB_UNLIKELY(!ObTabletRestoreAction::is_valid(restore_action))) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("invalid argument", K(ret), K(ls_id), K(restore_base_info), K(tablet_id_array));
LOG_WARN("invalid argument", K(ret), K(ls_id), K(restore_base_info), K(tablet_id_array), K(restore_action));
} else if (OB_ISNULL(ls_service = MTL(ObLSService *))) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("ls service should not be null", K(ret), KP(ls_service));
@ -1175,6 +1184,7 @@ int ObCopySSTableInfoRestoreReader::init(
LOG_WARN("failed to assign tablet id array", K(ret), K(tablet_id_array));
} else {
restore_base_info_ = &restore_base_info;
restore_action_ = restore_action;
meta_index_store_ = &meta_index_store;
tablet_index_ = 0;
sstable_index_ = 0;
@ -1385,10 +1395,150 @@ int ObCopySSTableInfoRestoreReader::get_tablet_meta_(
LOG_WARN("tablet should not be NULL", K(ret), KP(tablet), K(tablet_id));
} else if (OB_FAIL(tablet->build_migration_tablet_param(tablet_meta))) {
LOG_WARN("failed to build migration tablet param", K(ret), K(tablet_id));
} else if (OB_FAIL(may_update_tablet_meta_(tablet_id, tablet_handle, tablet_meta))) {
LOG_WARN("may update tablet meta failed", K(ret), K(tablet_id));
}
return ret;
}
int ObCopySSTableInfoRestoreReader::may_update_tablet_meta_(
const common::ObTabletID &tablet_id,
ObTabletHandle &tablet_handle,
ObMigrationTabletParam &tablet_meta)
{
int ret = OB_SUCCESS;
if (ObTabletRestoreAction::is_restore_major(restore_action_)) {
bool need_update_storage_schema = false;
backup::ObBackupTabletMeta backup_tablet_meta;
if (OB_FAIL(get_backup_major_tablet_meta_(tablet_id, backup_tablet_meta))) {
LOG_WARN("failed to get major tablet storage schema", K(ret), K(tablet_id));
} else if (OB_FAIL(compare_storage_schema_(tablet_id, tablet_handle, backup_tablet_meta, need_update_storage_schema))) {
LOG_WARN("failed to compare storage schema", K(ret), K(tablet_id));
} else if (need_update_storage_schema) {
tablet_meta.storage_schema_.reset();
if (OB_FAIL(tablet_meta.storage_schema_.init(tablet_meta.allocator_,
backup_tablet_meta.tablet_meta_.storage_schema_))) {
LOG_WARN("failed to init storage schema", K(ret), K(tablet_meta), K(backup_tablet_meta));
}
}
}
return ret;
}
int ObCopySSTableInfoRestoreReader::get_backup_major_tablet_meta_(
const common::ObTabletID &tablet_id,
backup::ObBackupTabletMeta &tablet_meta)
{
int ret = OB_SUCCESS;
backup::ObBackupMetaIndex tablet_meta_index;
share::ObBackupPath backup_path;
if (OB_ISNULL(restore_base_info_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("get invalid args", K(ret), KP(restore_base_info_));
} else {
const ObBackupDest &backup_dest = restore_base_info_->backup_dest_;
const share::ObBackupStorageInfo *storage_info = backup_dest.get_storage_info();
if (OB_FAIL(fetch_backup_major_tablet_meta_index_(tablet_id, tablet_meta_index))) {
LOG_WARN("failed to fetch backup tablet meta index", K(ret), K(tablet_id));
} else if (OB_FAIL(get_backup_tablet_meta_backup_path_(backup_dest, tablet_meta_index, backup_path))) {
LOG_WARN("failed to get tablet backup path", K(ret), K(backup_dest), K(tablet_meta_index));
} else if (OB_FAIL(read_backup_major_tablet_meta_(backup_path, storage_info, tablet_meta_index, tablet_meta))) {
LOG_WARN("failed to read backup major tablet meta", K(ret), K(backup_path), K(tablet_meta_index));
}
}
return ret;
}
int ObCopySSTableInfoRestoreReader::fetch_backup_major_tablet_meta_index_(
const common::ObTabletID &tablet_id,
backup::ObBackupMetaIndex &meta_index)
{
int ret = OB_SUCCESS;
meta_index.reset();
ObBackupDataType backup_data_type;
backup_data_type.set_major_data_backup();
backup::ObBackupMetaType meta_type = backup::ObBackupMetaType::BACKUP_TABLET_META;
if (!tablet_id.is_valid()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("get invalid arg", K(ret), K(tablet_id));
} else if (OB_ISNULL(meta_index_store_)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("should not be null", K(ret));
} else if (OB_FAIL(meta_index_store_->get_backup_meta_index(
backup_data_type, tablet_id, meta_type, meta_index))) {
LOG_WARN("failed to get meta index", K(ret), K(tablet_id));
} else {
LOG_INFO("get backup meta index", K(meta_index));
}
return ret;
}
int ObCopySSTableInfoRestoreReader::get_backup_tablet_meta_backup_path_(
const share::ObBackupDest &backup_dest,
const backup::ObBackupMetaIndex &meta_index,
ObBackupPath &backup_path)
{
int ret = OB_SUCCESS;
ObBackupDataType backup_data_type;
backup_data_type.set_major_data_backup();
if (!backup_dest.is_valid() || !meta_index.is_valid()) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("get invalid args", K(ret), K(meta_index), K(backup_dest));
} else if (OB_FAIL(ObBackupPathUtil::get_macro_block_backup_path(backup_dest,
meta_index.ls_id_, backup_data_type, meta_index.turn_id_,
meta_index.retry_id_, meta_index.file_id_, backup_path))) {
LOG_WARN("failed to get macro block backup path", K(ret), K(meta_index));
} else {
LOG_INFO("get macro block backup path", K(backup_path), K(meta_index));
}
return ret;
}
int ObCopySSTableInfoRestoreReader::read_backup_major_tablet_meta_(
const share::ObBackupPath &backup_path,
const share::ObBackupStorageInfo *storage_info,
const backup::ObBackupMetaIndex &meta_index,
backup::ObBackupTabletMeta &tablet_meta)
{
int ret = OB_SUCCESS;
share::ObBackupDataType backup_data_type;
backup_data_type.set_major_data_backup();
if (OB_FAIL(backup::ObLSBackupRestoreUtil::read_tablet_meta(
backup_path.get_obstr(), storage_info, backup_data_type, meta_index, tablet_meta))) {
LOG_WARN("failed to read tablet meta", K(ret), K(backup_path), K(meta_index));
}
return ret;
}
int ObCopySSTableInfoRestoreReader::compare_storage_schema_(
const common::ObTabletID &tablet_id,
const ObTabletHandle &tablet_handle,
const backup::ObBackupTabletMeta &tablet_meta,
bool &need_update)
{
int ret = OB_SUCCESS;
need_update = false;
const int64_t old_storage_schema_version = tablet_handle.get_obj()->get_storage_schema().schema_version_;
const int64_t new_storage_schema_version = tablet_meta.tablet_meta_.storage_schema_.schema_version_;
if (new_storage_schema_version > old_storage_schema_version) {
need_update = true;
LOG_INFO("storage schema version", K(old_storage_schema_version), K(new_storage_schema_version), K(tablet_id));
#ifdef ERRSIM
const int64_t old_multi_version_start = tablet_handle.get_obj()->get_multi_version_start();
const int64_t old_snapshot_version = tablet_handle.get_obj()->get_snapshot_version();
const int64_t new_multi_version_sstart = tablet_meta.tablet_meta_.multi_version_start_;
const int64_t new_snapshot_version = tablet_meta.tablet_meta_.snapshot_version_;
SERVER_EVENT_SYNC_ADD("storage_ha", "need_update_tablet_schema",
"tablet_id", tablet_id.id(),
"old_storage_schema_version", old_storage_schema_version,
"new_storage_schema_version", new_storage_schema_version,
"old_snapshot_version", old_snapshot_version,
"new_snapshot_version", new_snapshot_version,
"new_mult_version_start", new_multi_version_sstart);
#endif
}
return ret;
}
ObCopyTabletsSSTableInfoObProducer::ObCopyTabletsSSTableInfoObProducer()
: is_inited_(false),

View File

@ -61,7 +61,7 @@ struct ObCopyMacroBlockReaderInitParam final
TO_STRING_KV(K_(tenant_id), K_(ls_id), K_(table_key), KPC_(copy_macro_range_info), K_(src_info),
K_(is_leader_restore), KP_(bandwidth_throttle), KP_(svr_rpc_proxy), KP_(restore_base_info),
KP_(second_meta_index_store), KP_(restore_macro_block_id_mgr));
KP_(meta_index_store), KP_(second_meta_index_store), KP_(restore_macro_block_id_mgr));
uint64_t tenant_id_;
share::ObLSID ls_id_;
@ -72,6 +72,7 @@ struct ObCopyMacroBlockReaderInitParam final
common::ObInOutBandwidthThrottle *bandwidth_throttle_;
obrpc::ObStorageRpcProxy *svr_rpc_proxy_;
const ObRestoreBaseInfo *restore_base_info_;
backup::ObBackupMetaIndexStoreWrapper *meta_index_store_;
backup::ObBackupMetaIndexStoreWrapper *second_meta_index_store_;
ObRestoreMacroBlockIdMgr *restore_macro_block_id_mgr_;
@ -337,6 +338,7 @@ public:
int init(
const share::ObLSID &ls_id,
const ObRestoreBaseInfo &restore_base_info,
const ObTabletRestoreAction::ACTION &restore_action,
const common::ObIArray<common::ObTabletID> &tablet_id_array,
backup::ObBackupMetaIndexStoreWrapper &meta_index_store);
virtual int get_next_sstable_info(
@ -360,11 +362,35 @@ private:
int get_tablet_meta_(
const common::ObTabletID &tablet_id,
ObMigrationTabletParam &tablet_meta);
int may_update_tablet_meta_(
const common::ObTabletID &tablet_id,
ObTabletHandle &tablet_handle,
ObMigrationTabletParam &tablet_meta);
int get_backup_major_tablet_meta_(
const common::ObTabletID &tablet_id,
backup::ObBackupTabletMeta &tablet_meta);
int fetch_backup_major_tablet_meta_index_(
const common::ObTabletID &tablet_id,
backup::ObBackupMetaIndex &meta_index);
int get_backup_tablet_meta_backup_path_(
const share::ObBackupDest &backup_dest,
const backup::ObBackupMetaIndex &meta_index,
share::ObBackupPath &backup_path);
int read_backup_major_tablet_meta_(
const share::ObBackupPath &backup_path,
const share::ObBackupStorageInfo *storage_info,
const backup::ObBackupMetaIndex &meta_index,
backup::ObBackupTabletMeta &tablet_meta);
int compare_storage_schema_(
const common::ObTabletID &tablet_id,
const ObTabletHandle &old_tablet_handle,
const backup::ObBackupTabletMeta &tablet_meta,
bool &need_update);
private:
bool is_inited_;
const ObRestoreBaseInfo *restore_base_info_;
ObTabletRestoreAction::ACTION restore_action_;
common::ObArray<common::ObTabletID> tablet_id_array_;
backup::ObBackupMetaIndexStoreWrapper *meta_index_store_;
int64_t tablet_index_;

View File

@ -38,6 +38,7 @@ ObStorageHATabletsBuilderParam::ObStorageHATabletsBuilderParam()
svr_rpc_proxy_(nullptr),
storage_rpc_(nullptr),
restore_base_info_(nullptr),
restore_action_(ObTabletRestoreAction::MAX),
meta_index_store_(nullptr)
{
}
@ -57,6 +58,7 @@ void ObStorageHATabletsBuilderParam::reset()
svr_rpc_proxy_ = nullptr;
storage_rpc_ = nullptr;
restore_base_info_ = nullptr;
restore_action_ = ObTabletRestoreAction::MAX;
meta_index_store_ = nullptr;
}
@ -72,7 +74,9 @@ bool ObStorageHATabletsBuilderParam::is_valid() const
bool_ret = src_info_.is_valid() && OB_NOT_NULL(bandwidth_throttle_)
&& OB_NOT_NULL(svr_rpc_proxy_) && OB_NOT_NULL(storage_rpc_);
} else {
bool_ret = OB_NOT_NULL(restore_base_info_) && OB_NOT_NULL(meta_index_store_);
bool_ret = OB_NOT_NULL(restore_base_info_)
&& ObTabletRestoreAction::is_valid(restore_action_)
&& OB_NOT_NULL(meta_index_store_);
}
}
return bool_ret;
@ -99,6 +103,7 @@ int ObStorageHATabletsBuilderParam::assign(const ObStorageHATabletsBuilderParam
svr_rpc_proxy_ = param.svr_rpc_proxy_;
storage_rpc_ = param.storage_rpc_;
restore_base_info_ = param.restore_base_info_;
restore_action_ = param.restore_action_;
meta_index_store_ = param.meta_index_store_;
}
return ret;
@ -397,6 +402,8 @@ int ObStorageHATabletsBuilder::build_tablets_sstable_info_(
LOG_WARN("table should not be MEMTABLE", K(ret), K(sstable_info));
} else if (OB_FAIL(param_.ha_table_info_mgr_->add_table_info(sstable_info.tablet_id_, sstable_info))) {
LOG_WARN("failed to add table info", K(ret), K(sstable_info));
} else {
LOG_INFO("add table info", K(sstable_info.tablet_id_), K(sstable_info));
}
return ret;
}
@ -441,7 +448,8 @@ int ObStorageHATabletsBuilder::get_tablets_sstable_restore_reader_(ObICopySSTabl
} else if (FALSE_IT(restore_reader = new (buf) ObCopySSTableInfoRestoreReader())) {
} else if (FALSE_IT(reader = restore_reader)) {
} else if (OB_FAIL(restore_reader->init(param_.ls_->get_ls_id(),
*param_.restore_base_info_, param_.tablet_id_array_, *param_.meta_index_store_))) {
*param_.restore_base_info_, param_.restore_action_,
param_.tablet_id_array_, *param_.meta_index_store_))) {
LOG_WARN("failed to init restore reader", K(ret), K(param_));
}
@ -1525,6 +1533,7 @@ ObStorageHACopySSTableParam::ObStorageHACopySSTableParam()
svr_rpc_proxy_(nullptr),
storage_rpc_(nullptr),
restore_base_info_(nullptr),
meta_index_store_(nullptr),
second_meta_index_store_(nullptr)
{
}
@ -1543,6 +1552,7 @@ void ObStorageHACopySSTableParam::reset()
svr_rpc_proxy_ = nullptr;
storage_rpc_ = nullptr;
restore_base_info_ = nullptr;
meta_index_store_ = nullptr;
second_meta_index_store_ = nullptr;
}
@ -1558,7 +1568,9 @@ bool ObStorageHACopySSTableParam::is_valid() const
bool_ret = src_info_.is_valid() && OB_NOT_NULL(bandwidth_throttle_)
&& OB_NOT_NULL(svr_rpc_proxy_) && OB_NOT_NULL(storage_rpc_);
} else {
bool_ret = OB_NOT_NULL(restore_base_info_) && OB_NOT_NULL(second_meta_index_store_);
bool_ret = OB_NOT_NULL(restore_base_info_)
&& OB_NOT_NULL(meta_index_store_)
&& OB_NOT_NULL(second_meta_index_store_);
}
}
return bool_ret;
@ -1584,6 +1596,7 @@ int ObStorageHACopySSTableParam::assign(const ObStorageHACopySSTableParam &param
svr_rpc_proxy_ = param.svr_rpc_proxy_;
storage_rpc_ = param.storage_rpc_;
restore_base_info_ = param.restore_base_info_;
meta_index_store_ = param.meta_index_store_;
second_meta_index_store_ = param.second_meta_index_store_;
}
return ret;

View File

@ -35,7 +35,7 @@ struct ObStorageHATabletsBuilderParam final
TO_STRING_KV(K_(tenant_id), KPC_(ls), K_(tablet_id_array), K_(src_info), K_(local_rebuild_seq),
K_(need_check_seq), K_(is_leader_restore), K_(need_keep_old_tablet), KP_(ha_table_info_mgr),
KP_(bandwidth_throttle), KP_(svr_rpc_proxy), KP_(storage_rpc));
K_(restore_action), KP_(bandwidth_throttle), KP_(svr_rpc_proxy), KP_(storage_rpc));
uint64_t tenant_id_;
ObLS *ls_;
@ -51,6 +51,7 @@ struct ObStorageHATabletsBuilderParam final
obrpc::ObStorageRpcProxy *svr_rpc_proxy_;
storage::ObStorageRpc *storage_rpc_;
const ObRestoreBaseInfo *restore_base_info_;
ObTabletRestoreAction::ACTION restore_action_;
backup::ObBackupMetaIndexStoreWrapper *meta_index_store_;
DISALLOW_COPY_AND_ASSIGN(ObStorageHATabletsBuilderParam);
@ -225,6 +226,7 @@ struct ObStorageHACopySSTableParam final
obrpc::ObStorageRpcProxy *svr_rpc_proxy_;
storage::ObStorageRpc *storage_rpc_;
const ObRestoreBaseInfo *restore_base_info_;
backup::ObBackupMetaIndexStoreWrapper *meta_index_store_;
backup::ObBackupMetaIndexStoreWrapper *second_meta_index_store_;
DISALLOW_COPY_AND_ASSIGN(ObStorageHACopySSTableParam);

View File

@ -118,6 +118,7 @@ const char *ObTabletRestoreAction::get_action_str(const ACTION &action)
"RESTORE_TABLET_META",
"RESTORE_MINOR",
"RESTORE_MAJOR",
"RESTORE_NONE",
};
STATIC_ASSERT(MAX == ARRAYSIZEOF(action_strs), "action count mismatch");
if (action < 0 || action >= MAX) {
@ -161,6 +162,20 @@ bool ObTabletRestoreAction::is_restore_major(const ACTION &action)
return bool_ret;
}
bool ObTabletRestoreAction::is_restore_none(const ACTION &action)
{
bool bool_ret = false;
if (!is_valid(action)) {
bool_ret = false;
LOG_ERROR("restore action is unexpected", K(action));
} else if (ACTION::RESTORE_NONE != action) {
bool_ret = false;
} else {
bool_ret = true;
}
return bool_ret;
}
bool ObTabletRestoreAction::is_restore_all(const ACTION &action)
{
bool bool_ret = false;

View File

@ -60,12 +60,14 @@ struct ObTabletRestoreAction
RESTORE_TABLET_META = 1,
RESTORE_MINOR = 2,
RESTORE_MAJOR = 3,
RESTORE_NONE = 4,
MAX,
};
static const char *get_action_str(const ACTION &action);
static bool is_valid(const ACTION &action);
static bool is_restore_minor(const ACTION &action);
static bool is_restore_major(const ACTION &action);
static bool is_restore_none(const ACTION &action);
static bool is_restore_all(const ACTION &action);
static bool is_restore_tablet_meta(const ACTION &action);
static int trans_restore_action_to_restore_status(

View File

@ -102,6 +102,7 @@ ObTabletRestoreCtx::ObTabletRestoreCtx()
restore_base_info_(nullptr),
is_leader_(false),
action_(ObTabletRestoreAction::MAX),
meta_index_store_(nullptr),
second_meta_index_store_(nullptr),
replica_type_(ObReplicaType::REPLICA_TYPE_MAX),
ha_table_info_mgr_(nullptr),
@ -122,7 +123,7 @@ bool ObTabletRestoreCtx::is_valid() const
|| ObCopyTabletStatus::TABLET_NOT_EXIST == status_)
&& OB_NOT_NULL(restore_base_info_)
&& ObTabletRestoreAction::is_valid(action_)
&& (!is_leader_ || OB_NOT_NULL(second_meta_index_store_))
&& (!is_leader_ || (OB_NOT_NULL(meta_index_store_) && OB_NOT_NULL(second_meta_index_store_)))
&& ObReplicaTypeCheck::is_replica_type_valid(replica_type_)
&& OB_NOT_NULL(ha_table_info_mgr_);
}
@ -136,6 +137,7 @@ void ObTabletRestoreCtx::reset()
restore_base_info_ = nullptr;
is_leader_ = false;
action_ = ObTabletRestoreAction::MAX;
meta_index_store_ = nullptr;
second_meta_index_store_ = nullptr;
replica_type_ = ObReplicaType::REPLICA_TYPE_MAX;
status_ = ObCopyTabletStatus::MAX_STATUS;
@ -1006,7 +1008,8 @@ int ObInitialTabletGroupRestoreTask::init_ha_tablets_builder_()
LOG_WARN("initial tablet group restore task do not init", K(ret));
} else if (OB_FAIL(ObTabletGroupRestoreUtils::init_ha_tablets_builder(
ctx_->arg_.tenant_id_, ctx_->tablet_id_array_, ctx_->arg_.is_leader_, ctx_->src_,
ls_handle_.get_ls(), &ctx_->arg_.restore_base_info_, meta_index_store_,
ls_handle_.get_ls(), &ctx_->arg_.restore_base_info_,
ctx_->arg_.action_, meta_index_store_,
&ctx_->ha_table_info_mgr_, ha_tablets_builder_))) {
LOG_WARN("failed to init ha tablets builder", K(ret), KPC(ctx_));
}
@ -1202,7 +1205,7 @@ int ObStartTabletGroupRestoreTask::init(
LOG_WARN("failed to get ls", K(ret), KPC(ctx_));
} else if (OB_FAIL(ObTabletGroupRestoreUtils::init_ha_tablets_builder(
ctx_->arg_.tenant_id_, ctx_->tablet_id_array_, ctx_->arg_.is_leader_, ctx_->src_,
ls_handle_.get_ls(), &ctx_->arg_.restore_base_info_, meta_index_store_,
ls_handle_.get_ls(), &ctx_->arg_.restore_base_info_, ctx_->arg_.action_, meta_index_store_,
&ctx_->ha_table_info_mgr_, ha_tablets_builder_))) {
LOG_WARN("failed to init ha tablets builder", K(ret), KPC(ctx_));
} else {
@ -1281,6 +1284,7 @@ int ObStartTabletGroupRestoreTask::generate_tablet_restore_dag_()
param.action_ = ctx_->arg_.action_;
param.ha_table_info_mgr_ = &ctx_->ha_table_info_mgr_;
param.restore_base_info_ = &ctx_->arg_.restore_base_info_;
param.meta_index_store_ = meta_index_store_;
param.second_meta_index_store_ = second_meta_index_store_;
param.tablet_group_ctx_ = &ctx_->tablet_group_ctx_;
@ -1645,6 +1649,7 @@ ObInitTabletRestoreParam::ObInitTabletRestoreParam()
is_leader_(false),
action_(ObTabletRestoreAction::MAX),
restore_base_info_(nullptr),
meta_index_store_(nullptr),
second_meta_index_store_(nullptr),
ha_table_info_mgr_(nullptr),
tablet_group_ctx_(nullptr)
@ -1665,6 +1670,7 @@ void ObInitTabletRestoreParam::reset()
is_leader_ = false;
action_ = ObTabletRestoreAction::MAX;
restore_base_info_ = nullptr;
meta_index_store_ = nullptr;
second_meta_index_store_ = nullptr;
ha_table_info_mgr_ = nullptr;
tablet_group_ctx_ = nullptr;
@ -1679,7 +1685,9 @@ bool ObInitTabletRestoreParam::is_valid() const
&& OB_NOT_NULL(ha_table_info_mgr_);
if (bool_ret) {
if (is_leader_) {
bool_ret = OB_NOT_NULL(restore_base_info_) && OB_NOT_NULL(second_meta_index_store_);
bool_ret = OB_NOT_NULL(restore_base_info_)
&& OB_NOT_NULL(meta_index_store_)
&& OB_NOT_NULL(second_meta_index_store_);
}
}
return bool_ret;
@ -1834,6 +1842,7 @@ int ObTabletRestoreDag::init(
tablet_restore_ctx_.restore_base_info_ = param.restore_base_info_;
tablet_restore_ctx_.action_ = param.action_;
tablet_restore_ctx_.is_leader_ = param.is_leader_;
tablet_restore_ctx_.meta_index_store_ = param.meta_index_store_;
tablet_restore_ctx_.second_meta_index_store_ = param.second_meta_index_store_;
tablet_restore_ctx_.replica_type_ = ls->get_replica_type();
tablet_restore_ctx_.ha_table_info_mgr_ = param.ha_table_info_mgr_;
@ -1965,6 +1974,7 @@ int ObTabletRestoreDag::generate_next_dag(share::ObIDag *&dag)
param.action_ = tablet_restore_ctx_.action_;
param.ha_table_info_mgr_ = tablet_restore_ctx_.ha_table_info_mgr_;
param.restore_base_info_ = tablet_restore_ctx_.restore_base_info_;
param.meta_index_store_ = tablet_restore_ctx_.meta_index_store_;
param.second_meta_index_store_ = tablet_restore_ctx_.second_meta_index_store_;
param.tablet_group_ctx_ = tablet_group_ctx_;
@ -2223,6 +2233,7 @@ int ObTabletRestoreTask::generate_physical_restore_task_(
} else if (FALSE_IT(init_param.ls_ = ls_)) {
} else if (FALSE_IT(init_param.is_leader_restore_ = tablet_restore_ctx_->is_leader_)) {
} else if (FALSE_IT(init_param.restore_base_info_ = tablet_restore_ctx_->restore_base_info_)) {
} else if (FALSE_IT(init_param.meta_index_store_ = tablet_restore_ctx_->meta_index_store_)) {
} else if (FALSE_IT(init_param.second_meta_index_store_ = tablet_restore_ctx_->second_meta_index_store_)) {
} else if (OB_FAIL(tablet_restore_ctx_->ha_table_info_mgr_->get_table_info(tablet_restore_ctx_->tablet_id_,
copy_table_key, init_param.sstable_param_))) {
@ -2398,6 +2409,7 @@ int ObTabletRestoreTask::build_copy_sstable_info_mgr_()
param.tablet_id_ = tablet_restore_ctx_->tablet_id_;
param.is_leader_restore_ = tablet_restore_ctx_->is_leader_;
param.local_rebuild_seq_ = 0;
param.meta_index_store_ = tablet_restore_ctx_->meta_index_store_;
param.second_meta_index_store_ = tablet_restore_ctx_->second_meta_index_store_;
param.need_check_seq_ = false;
param.restore_base_info_ = tablet_restore_ctx_->restore_base_info_;
@ -2429,7 +2441,8 @@ int ObTabletRestoreTask::generate_tablet_copy_finish_task_(
} else if (OB_FAIL(tablet_restore_ctx_->ha_table_info_mgr_->get_tablet_meta(
tablet_restore_ctx_->tablet_id_, src_tablet_meta))) {
LOG_WARN("failed to get src tablet meta", K(ret), KPC(tablet_restore_ctx_));
} else if (OB_FAIL(tablet_copy_finish_task->init(tablet_restore_ctx_->tablet_id_, ls_, reporter, src_tablet_meta))) {
} else if (OB_FAIL(tablet_copy_finish_task->init(tablet_restore_ctx_->tablet_id_, ls_, reporter,
tablet_restore_ctx_->action_, src_tablet_meta))) {
LOG_WARN("failed to init tablet copy finish task", K(ret), KPC(ha_dag_net_ctx_), KPC(tablet_restore_ctx_));
}
return ret;
@ -2468,7 +2481,8 @@ int ObTabletRestoreTask::try_update_tablet_()
LOG_WARN("ls should not be NULL", K(ret), KP(ls), KPC(tablet_restore_ctx_));
} else if (OB_FAIL(ObTabletGroupRestoreUtils::init_ha_tablets_builder(
tablet_restore_ctx_->tenant_id_, tablet_id_array, tablet_restore_ctx_->is_leader_, src_info_,
ls, tablet_restore_ctx_->restore_base_info_, tablet_restore_ctx_->second_meta_index_store_,
ls, tablet_restore_ctx_->restore_base_info_, tablet_restore_ctx_->action_,
tablet_restore_ctx_->second_meta_index_store_,
tablet_restore_ctx_->ha_table_info_mgr_, ha_tablets_builder))) {
LOG_WARN("failed to init ha tablets builder", K(ret), KPC(tablet_restore_ctx_));
} else {
@ -2721,6 +2735,7 @@ int ObTabletGroupRestoreUtils::init_ha_tablets_builder(
const ObStorageHASrcInfo src_info,
ObLS *ls,
const ObRestoreBaseInfo *restore_base_info,
const ObTabletRestoreAction::ACTION &restore_action,
backup::ObBackupMetaIndexStoreWrapper *meta_index_store,
ObStorageHATableInfoMgr *ha_table_info_mgr,
ObStorageHATabletsBuilder &ha_tablets_builder)
@ -2732,6 +2747,7 @@ int ObTabletGroupRestoreUtils::init_ha_tablets_builder(
if (OB_INVALID_ID == tenant_id || tablet_id_array.empty() || OB_ISNULL(ls)
|| (!is_leader_restore && !src_info.is_valid())
|| (is_leader_restore && (OB_ISNULL(restore_base_info) || OB_ISNULL(meta_index_store)))
|| !ObTabletRestoreAction::is_valid(restore_action)
|| OB_ISNULL(ha_table_info_mgr)) {
ret = OB_INVALID_ARGUMENT;
LOG_WARN("init ha tablets builder get unexpected error", K(ret), KP(tenant_id),
@ -2750,6 +2766,7 @@ int ObTabletGroupRestoreUtils::init_ha_tablets_builder(
param.meta_index_store_ = meta_index_store;
param.need_check_seq_ = false;
param.restore_base_info_ = restore_base_info;
param.restore_action_ = restore_action;
param.src_info_ = src_info;
param.storage_rpc_ = ls_service->get_storage_rpc();
param.svr_rpc_proxy_ = ls_service->get_storage_rpc_proxy();

View File

@ -87,7 +87,7 @@ public:
int set_copy_tablet_status(const ObCopyTabletStatus::STATUS &status);
int get_copy_tablet_status(ObCopyTabletStatus::STATUS &status);
VIRTUAL_TO_STRING_KV(K_(tenant_id), K_(ls_id), K_(tablet_id), KPC_(restore_base_info), K_(is_leader),
K_(action), KP_(second_meta_index_store), K_(replica_type), KP_(ha_table_info_mgr), K_(status));
K_(action), KP_(meta_index_store), KP_(second_meta_index_store), K_(replica_type), KP_(ha_table_info_mgr), K_(status));
public:
uint64_t tenant_id_;
@ -97,6 +97,7 @@ public:
const ObRestoreBaseInfo *restore_base_info_;
bool is_leader_;
ObTabletRestoreAction::ACTION action_;
backup::ObBackupMetaIndexStoreWrapper *meta_index_store_;
backup::ObBackupMetaIndexStoreWrapper *second_meta_index_store_;
ObReplicaType replica_type_;
ObStorageHATableInfoMgr *ha_table_info_mgr_;
@ -335,6 +336,7 @@ struct ObInitTabletRestoreParam final
bool is_leader_;
ObTabletRestoreAction::ACTION action_;
const ObRestoreBaseInfo *restore_base_info_;
backup::ObBackupMetaIndexStoreWrapper *meta_index_store_;
backup::ObBackupMetaIndexStoreWrapper *second_meta_index_store_;
ObStorageHATableInfoMgr *ha_table_info_mgr_;
ObHATabletGroupCtx *tablet_group_ctx_;
@ -463,6 +465,7 @@ struct ObTabletGroupRestoreUtils
const ObStorageHASrcInfo src_info,
ObLS *ls,
const ObRestoreBaseInfo *restore_base_info,
const ObTabletRestoreAction::ACTION &restore_action,
backup::ObBackupMetaIndexStoreWrapper *meta_index_store,
ObStorageHATableInfoMgr *ha_table_info_mgr,
ObStorageHATabletsBuilder &ha_tablets_builder);