patch backup &migrate code to open source

This commit is contained in:
mw0
2021-09-27 20:48:34 +08:00
committed by wangzelin.wzl
parent a2c22b06aa
commit 64b53b6a17
41 changed files with 4944 additions and 4232 deletions

View File

@ -84,7 +84,7 @@ void ObPartitionStoreMeta::reset()
create_timestamp_ = 0;
}
int ObPartitionStoreMeta::deep_copy(const ObPartitionStoreMeta& meta)
int ObPartitionStoreMeta::deep_copy(const ObPartitionStoreMeta &meta)
{
int ret = OB_SUCCESS;
@ -138,7 +138,7 @@ bool ObPGPartitionStoreMeta::is_valid() const
return pkey_.is_valid() && multi_version_start_ > 0 && create_schema_version_ >= 0 && create_timestamp_ >= 0;
}
int ObPGPartitionStoreMeta::deep_copy(const ObPGPartitionStoreMeta& meta)
int ObPGPartitionStoreMeta::deep_copy(const ObPGPartitionStoreMeta &meta)
{
int ret = OB_SUCCESS;
if (is_valid()) {
@ -156,7 +156,7 @@ int ObPGPartitionStoreMeta::deep_copy(const ObPGPartitionStoreMeta& meta)
return ret;
}
int ObPGPartitionStoreMeta::copy_from_old_meta(const ObPartitionStoreMeta& meta)
int ObPGPartitionStoreMeta::copy_from_old_meta(const ObPartitionStoreMeta &meta)
{
int ret = OB_SUCCESS;
if (is_valid()) {
@ -224,7 +224,7 @@ void ObPartitionGroupMeta::reset()
restore_schema_version_ = OB_INVALID_TIMESTAMP;
}
int ObPartitionGroupMeta::deep_copy(const ObPartitionGroupMeta& meta)
int ObPartitionGroupMeta::deep_copy(const ObPartitionGroupMeta &meta)
{
int ret = OB_SUCCESS;
@ -261,7 +261,7 @@ int ObPartitionGroupMeta::deep_copy(const ObPartitionGroupMeta& meta)
return ret;
}
int ObPartitionGroupMeta::copy_from_store_meta(const ObPartitionStoreMeta& meta)
int ObPartitionGroupMeta::copy_from_store_meta(const ObPartitionStoreMeta &meta)
{
int ret = OB_SUCCESS;
if (is_valid()) {
@ -288,7 +288,7 @@ int ObPartitionGroupMeta::copy_from_store_meta(const ObPartitionStoreMeta& meta)
return ret;
}
int ObPartitionGroupMeta::get_recover_info_for_flashback(const int64_t, ObRecoverPoint&)
int ObPartitionGroupMeta::get_recover_info_for_flashback(const int64_t, ObRecoverPoint &)
{
return OB_NOT_SUPPORTED;
}
@ -363,7 +363,7 @@ void ObGetMergeTablesResult::reset()
read_base_version_ = 0;
}
int ObGetMergeTablesResult::deep_copy(const ObGetMergeTablesResult& src)
int ObGetMergeTablesResult::deep_copy(const ObGetMergeTablesResult &src)
{
int ret = OB_SUCCESS;
if (!src.is_valid()) {
@ -396,15 +396,13 @@ AddTableParam::AddTableParam()
need_prewarm_(false),
is_daily_merge_(false),
complement_minor_sstable_(nullptr),
backup_snapshot_version_(0),
schema_version_(0)
{}
bool AddTableParam::is_valid() const
{
return (!is_daily_merge_ || (OB_NOT_NULL(table_) || OB_NOT_NULL(complement_minor_sstable_))) &&
max_kept_major_version_number_ >= 0 && multi_version_start_ > ObVersionRange::MIN_VERSION &&
backup_snapshot_version_ >= 0;
max_kept_major_version_number_ >= 0 && multi_version_start_ > ObVersionRange::MIN_VERSION;
}
ObPartitionReadableInfo::ObPartitionReadableInfo()
@ -441,7 +439,7 @@ void ObPartitionReadableInfo::reset()
force_ = false;
}
int ObMigrateStatusHelper::trans_replica_op(const ObReplicaOpType& op_type, ObMigrateStatus& migrate_status)
int ObMigrateStatusHelper::trans_replica_op(const ObReplicaOpType &op_type, ObMigrateStatus &migrate_status)
{
int ret = OB_SUCCESS;
migrate_status = OB_MIGRATE_STATUS_MAX;
@ -503,7 +501,7 @@ int ObMigrateStatusHelper::trans_replica_op(const ObReplicaOpType& op_type, ObMi
return ret;
}
int ObMigrateStatusHelper::trans_fail_status(const ObMigrateStatus& cur_status, ObMigrateStatus& fail_status)
int ObMigrateStatusHelper::trans_fail_status(const ObMigrateStatus &cur_status, ObMigrateStatus &fail_status)
{
int ret = OB_SUCCESS;
fail_status = OB_MIGRATE_STATUS_MAX;
@ -569,7 +567,7 @@ int ObMigrateStatusHelper::trans_fail_status(const ObMigrateStatus& cur_status,
return ret;
}
int ObMigrateStatusHelper::trans_reboot_status(const ObMigrateStatus& cur_status, ObMigrateStatus& reboot_status)
int ObMigrateStatusHelper::trans_reboot_status(const ObMigrateStatus &cur_status, ObMigrateStatus &reboot_status)
{
int ret = OB_SUCCESS;
reboot_status = OB_MIGRATE_STATUS_MAX;
@ -685,7 +683,7 @@ bool ObCreatePGParam::is_valid() const
restore_schema_version_ >= OB_INVALID_TIMESTAMP && migrate_status_ < ObMigrateStatus::OB_MIGRATE_STATUS_MAX;
}
int ObCreatePGParam::assign(const ObCreatePGParam& param)
int ObCreatePGParam::assign(const ObCreatePGParam &param)
{
int ret = OB_SUCCESS;
if (!param.is_valid()) {
@ -715,7 +713,7 @@ int ObCreatePGParam::assign(const ObCreatePGParam& param)
return ret;
}
int ObCreatePGParam::set_storage_info(const ObSavedStorageInfoV2& info)
int ObCreatePGParam::set_storage_info(const ObSavedStorageInfoV2 &info)
{
int ret = OB_SUCCESS;
if (OB_FAIL(info_.deep_copy(info))) {
@ -724,7 +722,7 @@ int ObCreatePGParam::set_storage_info(const ObSavedStorageInfoV2& info)
return ret;
}
int ObCreatePGParam::set_split_info(const ObPartitionSplitInfo& split_info)
int ObCreatePGParam::set_split_info(const ObPartitionSplitInfo &split_info)
{
int ret = OB_SUCCESS;
if (OB_FAIL(split_info_.assign(split_info))) {
@ -747,7 +745,7 @@ void ObCreatePartitionMeta::reset()
id_hash_array_ = NULL;
}
int ObCreatePartitionMeta::extract_from(const ObTableSchema& table_schema)
int ObCreatePartitionMeta::extract_from(const ObTableSchema &table_schema)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!table_schema.is_valid())) {
@ -770,9 +768,9 @@ int ObCreatePartitionMeta::extract_from(const ObTableSchema& table_schema)
return ret;
}
const ObColumnSchemaV2* ObCreatePartitionMeta::get_column_schema(const uint64_t column_id) const
const ObColumnSchemaV2 *ObCreatePartitionMeta::get_column_schema(const uint64_t column_id) const
{
ObColumnSchemaV2* column = NULL;
ObColumnSchemaV2 *column = NULL;
if (NULL != id_hash_array_) {
if (OB_SUCCESS != id_hash_array_->get_refactored(ObColumnIdKey(column_id), column)) {
column = NULL;
@ -796,7 +794,7 @@ int ObCreatePartitionMeta::replace_tenant_id(const uint64_t new_tenant_id)
return ret;
}
int ObCreatePartitionMeta::assign(const ObCreatePartitionMeta& other)
int ObCreatePartitionMeta::assign(const ObCreatePartitionMeta &other)
{
int ret = OB_SUCCESS;
if (OB_FAIL(column_ids_.assign(other.get_store_column_ids()))) {
@ -833,12 +831,12 @@ void ObCreatePartitionParam::reset()
schemas_.reset();
}
int ObCreatePartitionParam::extract_from(const obrpc::ObCreatePartitionArg& arg)
int ObCreatePartitionParam::extract_from(const obrpc::ObCreatePartitionArg &arg)
{
int ret = OB_SUCCESS;
const int64_t table_cnt = arg.table_schemas_.count();
for (int64_t i = 0; OB_SUCC(ret) && i < table_cnt; ++i) {
const ObTableSchema& table_schema = arg.table_schemas_.at(i);
const ObTableSchema &table_schema = arg.table_schemas_.at(i);
ObCreatePartitionMeta partition_schema;
if (OB_FAIL(partition_schema.extract_from(table_schema))) {
STORAGE_LOG(WARN, "failed to extract partition_schema", KR(ret), K(table_schema));
@ -882,7 +880,7 @@ int ObCreatePartitionParam::replace_tenant_id(const uint64_t new_tenant_id)
} else {
const int64_t cnt = schemas_.count();
for (int64_t i = 0; OB_SUCC(ret) && i < cnt; ++i) {
ObCreatePartitionMeta& partition_schema = schemas_.at(i);
ObCreatePartitionMeta &partition_schema = schemas_.at(i);
if (OB_FAIL(partition_schema.replace_tenant_id(new_tenant_id))) {
STORAGE_LOG(WARN, "failed to replace_tenant_id of partition_scheam", KR(ret), K(*this));
}
@ -911,8 +909,8 @@ bool ObRecoveryPointSchemaFilter::is_inited() const
return is_inited_;
}
int ObRecoveryPointSchemaFilter::init(const int64_t tenant_id, const bool is_restore_point, const int64_t tenant_recovery_point_schema_version,
const int64_t tenant_current_schema_version)
int ObRecoveryPointSchemaFilter::init(const int64_t tenant_id, const bool is_restore_point,
const int64_t tenant_recovery_point_schema_version, const int64_t tenant_current_schema_version)
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(is_inited_)) {
@ -928,7 +926,7 @@ int ObRecoveryPointSchemaFilter::init(const int64_t tenant_id, const bool is_res
K(tenant_current_schema_version));
} else {
is_schema_version_same_ = (tenant_recovery_point_schema_version == tenant_current_schema_version);
ObMultiVersionSchemaService& schema_service = ObMultiVersionSchemaService::get_instance();
ObMultiVersionSchemaService &schema_service = ObMultiVersionSchemaService::get_instance();
if (OB_FAIL(ObBackupUtils::retry_get_tenant_schema_guard(
tenant_id, schema_service, tenant_recovery_point_schema_version, recovery_point_schema_guard_))) {
STORAGE_LOG(WARN,
@ -957,12 +955,12 @@ int ObRecoveryPointSchemaFilter::init(const int64_t tenant_id, const bool is_res
return ret;
}
int ObRecoveryPointSchemaFilter::check_partition_exist(const ObPartitionKey pkey, bool& is_exist)
int ObRecoveryPointSchemaFilter::check_partition_exist(const ObPartitionKey pkey, bool &is_exist)
{
int ret = OB_SUCCESS;
bool check_dropped_partition = false;
is_exist = false;
ObSchemaGetterGuard* schema_guard = NULL;
ObSchemaGetterGuard *schema_guard = NULL;
bool is_exist_in_backup_schema = false;
bool is_exist_in_delay_delete = false;
bool is_exist_in_current_schema = false;
@ -1001,7 +999,7 @@ int ObRecoveryPointSchemaFilter::check_partition_exist(const ObPartitionKey pkey
}
int ObRecoveryPointSchemaFilter::check_partition_exist_(
const ObPartitionKey pkey, const bool check_dropped_partition, ObSchemaGetterGuard& schema_guard, bool& is_exist)
const ObPartitionKey pkey, const bool check_dropped_partition, ObSchemaGetterGuard &schema_guard, bool &is_exist)
{
int ret = OB_SUCCESS;
is_exist = false;
@ -1017,11 +1015,11 @@ int ObRecoveryPointSchemaFilter::check_partition_exist_(
return ret;
}
int ObRecoveryPointSchemaFilter::check_table_exist(const uint64_t table_id, bool& is_exist)
int ObRecoveryPointSchemaFilter::check_table_exist(const uint64_t table_id, bool &is_exist)
{
int ret = OB_SUCCESS;
is_exist = false;
ObSchemaGetterGuard* schema_guard = NULL;
ObSchemaGetterGuard *schema_guard = NULL;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
@ -1044,11 +1042,11 @@ int ObRecoveryPointSchemaFilter::check_table_exist(const uint64_t table_id, bool
}
int ObRecoveryPointSchemaFilter::check_table_exist_(
const uint64_t table_id, ObSchemaGetterGuard& schema_guard, bool& is_exist)
const uint64_t table_id, ObSchemaGetterGuard &schema_guard, bool &is_exist)
{
int ret = OB_SUCCESS;
is_exist = false;
const ObTableSchema* table_schema = NULL;
const ObTableSchema *table_schema = NULL;
bool need_skip = false;
if (OB_UNLIKELY(!is_inited_)) {
ret = OB_NOT_INIT;
@ -1061,8 +1059,8 @@ int ObRecoveryPointSchemaFilter::check_table_exist_(
// do nothing
} else if (OB_FAIL(schema_guard.get_table_schema(table_id, table_schema))) {
STORAGE_LOG(WARN, "failed to get table schema", K(ret), K(table_id));
} else if (OB_FAIL(
ObBackupRestoreTableSchemaChecker::check_backup_restore_need_skip_table(table_schema, need_skip, is_restore_point_))) {
} else if (OB_FAIL(ObBackupRestoreTableSchemaChecker::check_backup_restore_need_skip_table(
table_schema, need_skip, is_restore_point_))) {
LOG_WARN("failed to check backup restore need skip table", K(ret), K(table_id));
} else if (!need_skip) {
// do nothing
@ -1072,7 +1070,7 @@ int ObRecoveryPointSchemaFilter::check_table_exist_(
return ret;
}
int ObRecoveryPointSchemaFilter::do_filter_tables(common::ObIArray<uint64_t>& table_ids)
int ObRecoveryPointSchemaFilter::do_filter_tables(common::ObIArray<uint64_t> &table_ids)
{
int ret = OB_SUCCESS;
ObArray<uint64_t> backup_tables;
@ -1114,7 +1112,7 @@ int ObRecoveryPointSchemaFilter::do_filter_tables(common::ObIArray<uint64_t>& ta
}
int ObRecoveryPointSchemaFilter::do_filter_pg_partitions(
const ObPartitionKey& pg_key, common::ObPartitionArray& partitions)
const ObPartitionKey &pg_key, common::ObPartitionArray &partitions)
{
int ret = OB_SUCCESS;
@ -1126,7 +1124,7 @@ int ObRecoveryPointSchemaFilter::do_filter_pg_partitions(
bool filtered = false;
ObPartitionArray exist_partitions;
for (int i = 0; OB_SUCC(ret) && i < partitions.count(); ++i) {
const ObPartitionKey& pkey = partitions.at(i);
const ObPartitionKey &pkey = partitions.at(i);
if (OB_FAIL(check_partition_exist(pkey, is_exist))) {
STORAGE_LOG(WARN, "backup filter check partition failed", K(ret), K(pg_key), K(pkey));
} else if (is_exist) {
@ -1150,12 +1148,12 @@ int ObRecoveryPointSchemaFilter::do_filter_pg_partitions(
}
int ObRecoveryPointSchemaFilter::check_if_table_miss_by_schema(
const ObPartitionKey& pgkey, const hash::ObHashSet<uint64_t>& table_ids)
const ObPartitionKey &pgkey, const hash::ObHashSet<uint64_t> &table_ids)
{
int ret = OB_SUCCESS;
int hash_ret = OB_SUCCESS;
ObArray<uint64_t> schema_tables;
ObSchemaGetterGuard* schema_guard = NULL;
ObSchemaGetterGuard *schema_guard = NULL;
if (OB_UNLIKELY(!is_inited())) {
ret = OB_NOT_INIT;
@ -1172,8 +1170,8 @@ int ObRecoveryPointSchemaFilter::check_if_table_miss_by_schema(
return ret;
}
int ObRecoveryPointSchemaFilter::check_if_table_miss_by_schema_(const ObPartitionKey& pgkey,
const common::hash::ObHashSet<uint64_t>& table_ids, share::schema::ObSchemaGetterGuard& schema_guard)
int ObRecoveryPointSchemaFilter::check_if_table_miss_by_schema_(const ObPartitionKey &pgkey,
const common::hash::ObHashSet<uint64_t> &table_ids, share::schema::ObSchemaGetterGuard &schema_guard)
{
int ret = OB_SUCCESS;
int hash_ret = OB_SUCCESS;
@ -1199,8 +1197,8 @@ int ObRecoveryPointSchemaFilter::check_if_table_miss_by_schema_(const ObPartitio
return ret;
}
int ObRecoveryPointSchemaFilter::get_table_ids_in_pg_(const ObPartitionKey& pgkey,
common::ObIArray<uint64_t>& table_ids, share::schema::ObSchemaGetterGuard& schema_guard)
int ObRecoveryPointSchemaFilter::get_table_ids_in_pg_(const ObPartitionKey &pgkey,
common::ObIArray<uint64_t> &table_ids, share::schema::ObSchemaGetterGuard &schema_guard)
{
int ret = OB_SUCCESS;
common::ObArray<uint64_t> data_table_ids;
@ -1222,7 +1220,7 @@ int ObRecoveryPointSchemaFilter::get_table_ids_in_pg_(const ObPartitionKey& pgke
if (OB_SUCC(ret)) {
for (int64_t i = 0; OB_SUCC(ret) && i < data_table_ids.count(); ++i) {
const uint64_t data_table_id = data_table_ids.at(i);
const ObTableSchema* table_schema = NULL;
const ObTableSchema *table_schema = NULL;
if (OB_FAIL(schema_guard.get_table_schema(data_table_id, table_schema))) {
STORAGE_LOG(WARN, "failed to get table schema", K(ret), K(data_table_id));
@ -1238,8 +1236,8 @@ int ObRecoveryPointSchemaFilter::get_table_ids_in_pg_(const ObPartitionKey& pgke
STORAGE_LOG(WARN, "get local index status fail", K(ret), K(pgkey));
} else {
for (int64_t j = 0; OB_SUCC(ret) && j < index_stats.count(); ++j) {
const ObTableSchema* index_table_schema = NULL;
const ObIndexTableStat& index_table_stat = index_stats.at(j);
const ObTableSchema *index_table_schema = NULL;
const ObIndexTableStat &index_table_stat = index_stats.at(j);
if (INDEX_STATUS_AVAILABLE != index_table_stat.index_status_) {
// filter unavailable index
} else if (OB_FAIL(schema_guard.get_table_schema(index_table_stat.index_id_, index_table_schema))) {
@ -1262,7 +1260,7 @@ int ObRecoveryPointSchemaFilter::get_table_ids_in_pg_(const ObPartitionKey& pgke
/***********************ObBackupRestoreTableSchemaChecker***************************/
int ObBackupRestoreTableSchemaChecker::check_backup_restore_need_skip_table(
const share::schema::ObTableSchema* table_schema, bool& need_skip, const bool is_restore_point)
const share::schema::ObTableSchema *table_schema, bool &need_skip, const bool is_restore_point)
{
int ret = OB_SUCCESS;
ObIndexStatus index_status;
@ -1276,19 +1274,21 @@ int ObBackupRestoreTableSchemaChecker::check_backup_restore_need_skip_table(
} else if (table_schema->is_dropped_schema()) {
STORAGE_LOG(INFO, "table is dropped, skip it", K(table_id));
} else if (FALSE_IT(index_status = table_schema->get_index_status())) {
} else if (table_schema->is_index_table()
&& (is_restore_point ?
!is_final_index_status(index_status, table_schema->is_dropped_schema()) :
ObIndexStatus::INDEX_STATUS_AVAILABLE != index_status)) {
STORAGE_LOG(INFO, "restore table index is not expected status, skip it",
K(is_restore_point), K(index_status), K(*table_schema));
} else if (table_schema->is_index_table() &&
(is_restore_point ? !is_final_index_status(index_status, table_schema->is_dropped_schema())
: ObIndexStatus::INDEX_STATUS_AVAILABLE != index_status)) {
STORAGE_LOG(INFO,
"restore table index is not expected status, skip it",
K(is_restore_point),
K(index_status),
K(*table_schema));
} else {
need_skip = false;
}
return ret;
}
ObRebuildListener::ObRebuildListener(transaction::ObPartitionTransCtxMgr& mgr) : ob_partition_ctx_mgr_(mgr)
ObRebuildListener::ObRebuildListener(transaction::ObPartitionTransCtxMgr &mgr) : ob_partition_ctx_mgr_(mgr)
{
int tmp_ret = OB_SUCCESS;
while (OB_SUCCESS != (tmp_ret = ob_partition_ctx_mgr_.lock_minor_merge_lock())) {
@ -1317,12 +1317,12 @@ bool ObRebuildListener::on_partition_rebuild()
}
int ObRestoreFakeMemberListHelper::fake_restore_member_list(
const int64_t replica_cnt, common::ObMemberList& fake_member_list)
const int64_t replica_cnt, common::ObMemberList &fake_member_list)
{
int ret = OB_SUCCESS;
fake_member_list.reset();
const char* fake_ip = "127.0.0.1";
const char *fake_ip = "127.0.0.1";
int32_t fake_port = 10000;
if (replica_cnt <= 0) {