提交 b1ec5d35 编写于 作者: G godyangfight 提交者: LINGuanRen

patch backup related bug fix to open source

上级 2da7b7f4
...@@ -56,9 +56,9 @@ ObBackupAutoDeleteExpiredData::ObBackupAutoDeleteExpiredData() ...@@ -56,9 +56,9 @@ ObBackupAutoDeleteExpiredData::ObBackupAutoDeleteExpiredData()
ObBackupAutoDeleteExpiredData::~ObBackupAutoDeleteExpiredData() ObBackupAutoDeleteExpiredData::~ObBackupAutoDeleteExpiredData()
{} {}
int ObBackupAutoDeleteExpiredData::init(common::ObServerConfig& cfg, ObMySQLProxy& sql_proxy, int ObBackupAutoDeleteExpiredData::init(common::ObServerConfig &cfg, ObMySQLProxy &sql_proxy,
share::schema::ObMultiVersionSchemaService& schema_service, ObBackupDataClean& backup_data_clean, share::schema::ObMultiVersionSchemaService &schema_service, ObBackupDataClean &backup_data_clean,
share::ObIBackupLeaseService& backup_lease_service) share::ObIBackupLeaseService &backup_lease_service)
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
const int backup_auto_delete_thread_cnt = 1; const int backup_auto_delete_thread_cnt = 1;
...@@ -156,7 +156,7 @@ void ObBackupAutoDeleteExpiredData::run3() ...@@ -156,7 +156,7 @@ void ObBackupAutoDeleteExpiredData::run3()
} }
int ObBackupAutoDeleteExpiredData::check_can_auto_handle_backup( int ObBackupAutoDeleteExpiredData::check_can_auto_handle_backup(
const bool is_auto, const int64_t backup_recovery_window, bool& can_auto_delete) const bool is_auto, const int64_t backup_recovery_window, bool &can_auto_delete)
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
can_auto_delete = true; can_auto_delete = true;
...@@ -172,7 +172,7 @@ int ObBackupAutoDeleteExpiredData::check_can_auto_handle_backup( ...@@ -172,7 +172,7 @@ int ObBackupAutoDeleteExpiredData::check_can_auto_handle_backup(
} }
int ObBackupAutoDeleteExpiredData::get_last_succeed_delete_obsolete_snapshot( int ObBackupAutoDeleteExpiredData::get_last_succeed_delete_obsolete_snapshot(
int64_t& last_succ_delete_obsolete_snapshot) int64_t &last_succ_delete_obsolete_snapshot)
{ {
int ret = OB_SUCCESS; int ret = OB_SUCCESS;
ObBackupInfoManager backup_info_manager; ObBackupInfoManager backup_info_manager;
...@@ -221,6 +221,10 @@ int ObBackupAutoDeleteExpiredData::schedule_auto_delete_expired_data(const int64 ...@@ -221,6 +221,10 @@ int ObBackupAutoDeleteExpiredData::schedule_auto_delete_expired_data(const int64
int64_t last_succ_delete_obsolete_snapshot = 0; int64_t last_succ_delete_obsolete_snapshot = 0;
const int64_t now_ts = ObTimeUtil::current_time(); const int64_t now_ts = ObTimeUtil::current_time();
ObBackupDataCleanScheduler backup_data_clean_scheduler; ObBackupDataCleanScheduler backup_data_clean_scheduler;
const int64_t MAX_INTERVAL = 24L * 60L * 60L * 1000L * 1000L; // 24h
const int64_t AUTO_CLEAN_INTERVAL =
(backup_recovery_window / 2) < MAX_INTERVAL ? (backup_recovery_window / 2) : MAX_INTERVAL;
if (!is_inited_) { if (!is_inited_) {
ret = OB_NOT_INIT; ret = OB_NOT_INIT;
LOG_WARN("backup auto delete expired data do not init", K(ret)); LOG_WARN("backup auto delete expired data do not init", K(ret));
...@@ -229,7 +233,7 @@ int ObBackupAutoDeleteExpiredData::schedule_auto_delete_expired_data(const int64 ...@@ -229,7 +233,7 @@ int ObBackupAutoDeleteExpiredData::schedule_auto_delete_expired_data(const int64
LOG_WARN("schedule auto delete expired data get invalid argument", K(ret), K(backup_recovery_window)); LOG_WARN("schedule auto delete expired data get invalid argument", K(ret), K(backup_recovery_window));
} else if (OB_FAIL(get_last_succeed_delete_obsolete_snapshot(last_succ_delete_obsolete_snapshot))) { } else if (OB_FAIL(get_last_succeed_delete_obsolete_snapshot(last_succ_delete_obsolete_snapshot))) {
LOG_WARN("failed to get last succ delete obsolete snapshot", K(ret), K(last_succ_delete_obsolete_snapshot)); LOG_WARN("failed to get last succ delete obsolete snapshot", K(ret), K(last_succ_delete_obsolete_snapshot));
} else if (now_ts - last_succ_delete_obsolete_snapshot < backup_recovery_window / 2) { } else if (now_ts - last_succ_delete_obsolete_snapshot < AUTO_CLEAN_INTERVAL) {
switch_delete_obsolete_action(); switch_delete_obsolete_action();
if (delete_obsolete_action_ != ObBackupDeleteObsoleteAction::NONE) { if (delete_obsolete_action_ != ObBackupDeleteObsoleteAction::NONE) {
wakeup(); wakeup();
......
...@@ -30,18 +30,18 @@ public: ...@@ -30,18 +30,18 @@ public:
ObPhyRestoreMacroIndexStoreV2(); ObPhyRestoreMacroIndexStoreV2();
virtual ~ObPhyRestoreMacroIndexStoreV2(); virtual ~ObPhyRestoreMacroIndexStoreV2();
void reset(); void reset();
int init(const int64_t backup_task_id, const share::ObPhysicalRestoreArg& arg, int init(const int64_t backup_task_id, const share::ObPhysicalRestoreArg &arg,
const ObReplicaRestoreStatus& restore_status); const ObReplicaRestoreStatus &restore_status);
int init(const int64_t backup_task_id, const common::ObPartitionKey& pkey, const share::ObPhysicalBackupArg& arg, int init(const int64_t backup_task_id, const common::ObPartitionKey &pkey, const share::ObPhysicalBackupArg &arg,
const ObBackupDataType& backup_data_type); const ObBackupDataType &backup_data_type);
int get_macro_index_array( int get_macro_index_array(
const ObITable::TableKey& table_key, const common::ObArray<ObBackupTableMacroIndex>*& index_list) const; const ObITable::TableKey &table_key, const common::ObArray<ObBackupTableMacroIndex> *&index_list) const;
int get_macro_index( int get_macro_index(
const ObITable::TableKey& table_key, const int64_t sstable_idx, ObBackupTableMacroIndex& macro_index) const; const ObITable::TableKey &table_key, const int64_t sstable_idx, ObBackupTableMacroIndex &macro_index) const;
int get_sstable_pair_list( int get_sstable_pair_list(
const ObITable::TableKey& table_key, common::ObIArray<blocksstable::ObSSTablePair>& pair_list) const; const ObITable::TableKey &table_key, common::ObIArray<blocksstable::ObSSTablePair> &pair_list) const;
int get_major_macro_index_array( int get_major_macro_index_array(
const uint64_t index_table_id, const common::ObArray<ObBackupTableMacroIndex>*& index_list) const; const uint64_t index_table_id, const common::ObArray<ObBackupTableMacroIndex> *&index_list) const;
int64_t get_backup_task_id() const int64_t get_backup_task_id() const
{ {
return backup_task_id_; return backup_task_id_;
...@@ -51,31 +51,33 @@ public: ...@@ -51,31 +51,33 @@ public:
return PHY_RESTORE_MACRO_INDEX_STORE_V2; return PHY_RESTORE_MACRO_INDEX_STORE_V2;
} }
virtual bool is_inited() const; virtual bool is_inited() const;
int check_table_exist(const ObITable::TableKey &table_key, bool &is_exist) const;
TO_STRING_KV(K_(is_inited)); TO_STRING_KV(K_(is_inited));
private: private:
int init_major_macro_index(const common::ObPartitionKey& backup_pg_key, const ObBackupBaseDataPathInfo& path_info, int init_major_macro_index(const common::ObPartitionKey &backup_pg_key, const ObBackupBaseDataPathInfo &path_info,
const ObReplicaRestoreStatus& restore_status); const ObReplicaRestoreStatus &restore_status);
int init_major_macro_index(const common::ObPartitionKey& backup_pg_key, const ObSimpleBackupSetPath& simple_path, int init_major_macro_index(const common::ObPartitionKey &backup_pg_key, const ObSimpleBackupSetPath &simple_path,
const ObReplicaRestoreStatus& restore_status); const ObReplicaRestoreStatus &restore_status);
int init_minor_macro_index(const int64_t backup_task_id, const common::ObPartitionKey& backup_pg_key,
const ObBackupBaseDataPathInfo& path_info, const ObReplicaRestoreStatus& restore_status); int init_minor_macro_index(const int64_t backup_task_id, const common::ObPartitionKey &backup_pg_key,
int init_minor_macro_index(const int64_t backup_task_id, const common::ObPartitionKey& backup_pg_key, const ObBackupBaseDataPathInfo &path_info, const ObReplicaRestoreStatus &restore_status);
const ObSimpleBackupSetPath& simple_path, const ObReplicaRestoreStatus& restore_status); int init_minor_macro_index(const int64_t backup_task_id, const common::ObPartitionKey &backup_pg_key,
const ObSimpleBackupSetPath &simple_path, const ObReplicaRestoreStatus &restore_status);
int add_sstable_index( int add_sstable_index(
const ObITable::TableKey& table_key, const common::ObIArray<ObBackupTableMacroIndex>& index_list); const ObITable::TableKey &table_key, const common::ObIArray<ObBackupTableMacroIndex> &index_list);
int init_one_file(const ObString& path, const ObString& storage_info); int init_one_file(const ObString &path, const ObString &storage_info);
int get_table_key_ptr(const ObITable::TableKey& table_key, const ObITable::TableKey*& table_key_ptr); int get_table_key_ptr(const ObITable::TableKey &table_key, const ObITable::TableKey *&table_key_ptr);
private: private:
static const int64_t BUCKET_SIZE = 100000; // 10w static const int64_t BUCKET_SIZE = 100000; // 10w
typedef common::hash::ObHashMap<ObITable::TableKey, common::ObArray<ObBackupTableMacroIndex>*> MacroIndexMap; typedef common::hash::ObHashMap<ObITable::TableKey, common::ObArray<ObBackupTableMacroIndex> *> MacroIndexMap;
bool is_inited_; bool is_inited_;
common::ObArenaAllocator allocator_; common::ObArenaAllocator allocator_;
MacroIndexMap index_map_; MacroIndexMap index_map_;
int64_t backup_task_id_; int64_t backup_task_id_;
ObArray<ObITable::TableKey*> table_keys_ptr_; ObArray<ObITable::TableKey *> table_keys_ptr_;
DISALLOW_COPY_AND_ASSIGN(ObPhyRestoreMacroIndexStoreV2); DISALLOW_COPY_AND_ASSIGN(ObPhyRestoreMacroIndexStoreV2);
}; };
...@@ -84,32 +86,32 @@ public: ...@@ -84,32 +86,32 @@ public:
ObPartitionBaseDataMetaRestoreReaderV2(); ObPartitionBaseDataMetaRestoreReaderV2();
virtual ~ObPartitionBaseDataMetaRestoreReaderV2(); virtual ~ObPartitionBaseDataMetaRestoreReaderV2();
void reset(); void reset();
int init(const common::ObPartitionKey& pkey, const ObPhysicalRestoreArg& restore_info, int init(const common::ObPartitionKey &pkey, const ObPhysicalRestoreArg &restore_info,
const ObPhyRestoreMacroIndexStoreV2& macro_indexs, const ObBackupPartitionStoreMetaInfo& backup_pg_meta_info, const ObPhyRestoreMacroIndexStoreV2 &macro_indexs, const ObBackupPartitionStoreMetaInfo &backup_pg_meta_info,
const int64_t schema_version); const int64_t schema_version);
int fetch_partition_meta(ObPGPartitionStoreMeta& partition_store_meta); int fetch_partition_meta(ObPGPartitionStoreMeta &partition_store_meta);
int fetch_sstable_meta(const ObITable::TableKey& table_key, blocksstable::ObSSTableBaseMeta& sstable_meta); int fetch_sstable_meta(const ObITable::TableKey &table_key, blocksstable::ObSSTableBaseMeta &sstable_meta);
int fetch_sstable_pair_list( int fetch_sstable_pair_list(
const ObITable::TableKey& table_key, common::ObIArray<blocksstable::ObSSTablePair>& pair_list); const ObITable::TableKey &table_key, common::ObIArray<blocksstable::ObSSTablePair> &pair_list);
int fetch_all_table_ids(common::ObIArray<uint64_t>& table_id_array); int fetch_all_table_ids(common::ObIArray<uint64_t> &table_id_array);
int fetch_table_keys(const uint64_t index_id, obrpc::ObFetchTableInfoResult& table_res); int fetch_table_keys(const uint64_t index_id, obrpc::ObFetchTableInfoResult &table_res);
TO_STRING_KV( TO_STRING_KV(
K_(pkey), K_(restore_info), K_(last_read_size), K_(partition_store_meta), K_(data_version), K_(schema_version)); K_(pkey), K_(restore_info), K_(last_read_size), K_(partition_store_meta), K_(data_version), K_(schema_version));
private: private:
int prepare(const common::ObPartitionKey& pkey); int prepare(const common::ObPartitionKey &pkey);
int trans_table_key(const ObITable::TableKey& table_Key, ObITable::TableKey& backup_table_key); int trans_table_key(const ObITable::TableKey &table_Key, ObITable::TableKey &backup_table_key);
int get_backup_sstable_meta_info( int get_backup_sstable_meta_info(
const ObITable::TableKey& backup_table_key, const ObBackupSSTableMetaInfo*& backup_sstable_meta_info); const ObITable::TableKey &backup_table_key, const ObBackupSSTableMetaInfo *&backup_sstable_meta_info);
int get_backup_table_keys(const uint64_t backup_index_id, common::ObIArray<ObITable::TableKey>& table_keys); int get_backup_table_keys(const uint64_t backup_index_id, common::ObIArray<ObITable::TableKey> &table_keys);
int do_filter_tables(common::ObIArray<uint64_t>& table_ids); int do_filter_tables(common::ObIArray<uint64_t> &table_ids);
private: private:
bool is_inited_; bool is_inited_;
common::ObPartitionKey pkey_; common::ObPartitionKey pkey_;
const ObPhysicalRestoreArg* restore_info_; const ObPhysicalRestoreArg *restore_info_;
const ObPhyRestoreMacroIndexStoreV2* macro_indexs_; const ObPhyRestoreMacroIndexStoreV2 *macro_indexs_;
const ObBackupPartitionStoreMetaInfo* partition_store_meta_info_; const ObBackupPartitionStoreMetaInfo *partition_store_meta_info_;
int64_t last_read_size_; int64_t last_read_size_;
ObPGPartitionStoreMeta partition_store_meta_; ObPGPartitionStoreMeta partition_store_meta_;
int64_t data_version_; int64_t data_version_;
...@@ -123,10 +125,10 @@ public: ...@@ -123,10 +125,10 @@ public:
ObPhysicalBaseMetaRestoreReaderV2(); ObPhysicalBaseMetaRestoreReaderV2();
virtual ~ObPhysicalBaseMetaRestoreReaderV2() virtual ~ObPhysicalBaseMetaRestoreReaderV2()
{} {}
int init(common::ObInOutBandwidthThrottle& bandwidth_throttle, const ObPhysicalRestoreArg& restore_info, int init(common::ObInOutBandwidthThrottle &bandwidth_throttle, const ObPhysicalRestoreArg &restore_info,
const ObITable::TableKey& table_key, ObIPartitionGroupMetaRestoreReader& reader); const ObITable::TableKey &table_key, ObIPartitionGroupMetaRestoreReader &reader);
virtual int fetch_sstable_meta(blocksstable::ObSSTableBaseMeta& sstable_meta); virtual int fetch_sstable_meta(blocksstable::ObSSTableBaseMeta &sstable_meta);
virtual int fetch_macro_block_list(common::ObIArray<blocksstable::ObSSTablePair>& macro_block_list); virtual int fetch_macro_block_list(common::ObIArray<blocksstable::ObSSTablePair> &macro_block_list);
virtual Type get_type() const virtual Type get_type() const
{ {
return BASE_DATA_META_RESTORE_READER_V1; return BASE_DATA_META_RESTORE_READER_V1;
...@@ -134,10 +136,10 @@ public: ...@@ -134,10 +136,10 @@ public:
private: private:
bool is_inited_; bool is_inited_;
const ObPhysicalRestoreArg* restore_info_; const ObPhysicalRestoreArg *restore_info_;
ObIPartitionGroupMetaRestoreReader* reader_; ObIPartitionGroupMetaRestoreReader *reader_;
common::ObArenaAllocator allocator_; common::ObArenaAllocator allocator_;
common::ObInOutBandwidthThrottle* bandwidth_throttle_; common::ObInOutBandwidthThrottle *bandwidth_throttle_;
ObITable::TableKey table_key_; ObITable::TableKey table_key_;
DISALLOW_COPY_AND_ASSIGN(ObPhysicalBaseMetaRestoreReaderV2); DISALLOW_COPY_AND_ASSIGN(ObPhysicalBaseMetaRestoreReaderV2);
}; };
...@@ -147,11 +149,11 @@ public: ...@@ -147,11 +149,11 @@ public:
ObPartitionMacroBlockRestoreReaderV2(); ObPartitionMacroBlockRestoreReaderV2();
virtual ~ObPartitionMacroBlockRestoreReaderV2(); virtual ~ObPartitionMacroBlockRestoreReaderV2();
void reset(); void reset();
int init(common::ObInOutBandwidthThrottle& bandwidth_throttle, common::ObIArray<ObMigrateArgMacroBlockInfo>& list, int init(common::ObInOutBandwidthThrottle &bandwidth_throttle, common::ObIArray<ObMigrateArgMacroBlockInfo> &list,
const ObPhysicalRestoreArg& restore_info, const ObPhyRestoreMacroIndexStoreV2& macro_indexs, const ObPhysicalRestoreArg &restore_info, const ObPhyRestoreMacroIndexStoreV2 &macro_indexs,
const ObITable::TableKey& table_key); const ObITable::TableKey &table_key);
virtual int get_next_macro_block(blocksstable::ObFullMacroBlockMeta& meta, blocksstable::ObBufferReader& data, virtual int get_next_macro_block(blocksstable::ObFullMacroBlockMeta &meta, blocksstable::ObBufferReader &data,
blocksstable::MacroBlockId& src_macro_id); blocksstable::MacroBlockId &src_macro_id);
virtual Type get_type() const virtual Type get_type() const
{ {
return MACRO_BLOCK_RESTORE_READER_V2; return MACRO_BLOCK_RESTORE_READER_V2;
...@@ -163,8 +165,8 @@ public: ...@@ -163,8 +165,8 @@ public:
private: private:
int trans_macro_block( int trans_macro_block(
const uint64_t table_id, blocksstable::ObMacroBlockMetaV2& meta, blocksstable::ObBufferReader& data); const uint64_t table_id, blocksstable::ObMacroBlockMetaV2 &meta, blocksstable::ObBufferReader &data);
int get_macro_block_path(const ObBackupTableMacroIndex& macro_index, share::ObBackupPath& path); int get_macro_block_path(const ObBackupTableMacroIndex &macro_index, share::ObBackupPath &path);
private: private:
bool is_inited_; bool is_inited_;
...@@ -174,11 +176,12 @@ private: ...@@ -174,11 +176,12 @@ private:
uint64_t table_id_; uint64_t table_id_;
ObSimpleBackupSetPath simple_path_; ObSimpleBackupSetPath simple_path_;
ObBackupBaseDataPathInfo backup_path_info_; ObBackupBaseDataPathInfo backup_path_info_;
const ObPhyRestoreMacroIndexStoreV2* macro_indexs_; const ObPhyRestoreMacroIndexStoreV2 *macro_indexs_;
common::ObInOutBandwidthThrottle* bandwidth_throttle_; common::ObInOutBandwidthThrottle *bandwidth_throttle_;
ObPartitionKey backup_pgkey_; ObPartitionKey backup_pgkey_;
ObITable::TableKey backup_table_key_; ObITable::TableKey backup_table_key_;
common::ObArenaAllocator allocator_; common::ObArenaAllocator allocator_;
const ObPhysicalRestoreArg *restore_info_;
DISALLOW_COPY_AND_ASSIGN(ObPartitionMacroBlockRestoreReaderV2); DISALLOW_COPY_AND_ASSIGN(ObPartitionMacroBlockRestoreReaderV2);
}; };
...@@ -187,49 +190,49 @@ public: ...@@ -187,49 +190,49 @@ public:
ObPartitionGroupMetaRestoreReaderV2(); ObPartitionGroupMetaRestoreReaderV2();
virtual ~ObPartitionGroupMetaRestoreReaderV2(); virtual ~ObPartitionGroupMetaRestoreReaderV2();
void reset(); void reset();
int init(common::ObInOutBandwidthThrottle& bandwidth_throttle, const ObPhysicalRestoreArg& restore_info, int init(common::ObInOutBandwidthThrottle &bandwidth_throttle, const ObPhysicalRestoreArg &restore_info,
const ObPhyRestoreMetaIndexStore& meta_indexs, const ObPhyRestoreMacroIndexStoreV2& macro_indexs); const ObPhyRestoreMetaIndexStore &meta_indexs, const ObPhyRestoreMacroIndexStoreV2 &macro_indexs);
virtual int fetch_partition_group_meta(ObPartitionGroupMeta& pg_meta); virtual int fetch_partition_group_meta(ObPartitionGroupMeta &pg_meta);
virtual int64_t get_data_size() const virtual int64_t get_data_size() const
{ {
return reader_.get_data_size(); return reader_.get_data_size();
} }
virtual int fetch_sstable_meta(const ObITable::TableKey& table_key, blocksstable::ObSSTableBaseMeta& sstable_meta); virtual int fetch_sstable_meta(const ObITable::TableKey &table_key, blocksstable::ObSSTableBaseMeta &sstable_meta);
virtual int fetch_sstable_pair_list( virtual int fetch_sstable_pair_list(
const ObITable::TableKey& table_key, common::ObIArray<blocksstable::ObSSTablePair>& pair_list); const ObITable::TableKey &table_key, common::ObIArray<blocksstable::ObSSTablePair> &pair_list);
virtual Type get_type() const virtual Type get_type() const
{ {
return PG_META_RESTORE_READER_V2; return PG_META_RESTORE_READER_V2;
}; };
int get_partition_readers(const ObPartitionArray& partitions, int get_partition_readers(const ObPartitionArray &partitions,
common::ObIArray<ObPartitionBaseDataMetaRestoreReaderV2*>& partition_reader_array); common::ObIArray<ObPartitionBaseDataMetaRestoreReaderV2 *> &partition_reader_array);
int get_restore_tenant_id(uint64_t& tenant_id); int get_restore_tenant_id(uint64_t &tenant_id);
int get_restore_schema_version(int64_t& schema_version); int get_restore_schema_version(int64_t &schema_version);
private: private:
int prepare(const ObPhysicalRestoreArg& restore_info, const ObPhyRestoreMacroIndexStoreV2& macro_indexs); int prepare(const ObPhysicalRestoreArg &restore_info, const ObPhyRestoreMacroIndexStoreV2 &macro_indexs);
int prepare_pg_meta(const ObPhysicalRestoreArg& restore_info); int prepare_pg_meta(const ObPhysicalRestoreArg &restore_info);
int create_pg_partition_if_need(const ObPhysicalRestoreArg& restore_info, const ObPartitionGroupMeta& backup_pg_meta); int create_pg_partition_if_need(const ObPhysicalRestoreArg &restore_info, const ObPartitionGroupMeta &backup_pg_meta);
int read_partition_meta(const ObPartitionKey& pkey, const ObPhysicalRestoreArg& restore_info, int read_partition_meta(const ObPartitionKey &pkey, const ObPhysicalRestoreArg &restore_info,
ObPGPartitionStoreMeta& partition_store_meta); ObPGPartitionStoreMeta &partition_store_meta);
int trans_backup_pgmeta(const ObPhysicalRestoreArg& restore_info, ObPartitionGroupMeta& backup_pg_meta); int trans_backup_pgmeta(const ObPhysicalRestoreArg &restore_info, ObPartitionGroupMeta &backup_pg_meta);
int check_backup_partitions_in_pg(const ObPhysicalRestoreArg& restore_info, ObPartitionGroupMeta& backup_pg_meta); int check_backup_partitions_in_pg(const ObPhysicalRestoreArg &restore_info, ObPartitionGroupMeta &backup_pg_meta);
int get_backup_partition_meta_info(const ObPartitionKey& pkey, const ObPhysicalRestoreArg& restore_info, int get_backup_partition_meta_info(const ObPartitionKey &pkey, const ObPhysicalRestoreArg &restore_info,
const ObBackupPartitionStoreMetaInfo*& backup_partition_meta_info); const ObBackupPartitionStoreMetaInfo *&backup_partition_meta_info);
int trans_from_backup_partitions(const ObPhysicalRestoreArg& restore_info, int trans_from_backup_partitions(const ObPhysicalRestoreArg &restore_info,
const common::ObPartitionArray& backup_partitions, common::ObPartitionArray& current_partitions); const common::ObPartitionArray &backup_partitions, common::ObPartitionArray &current_partitions);
int trans_to_backup_partitions(const ObPhysicalRestoreArg& restore_info, int trans_to_backup_partitions(const ObPhysicalRestoreArg &restore_info,
const common::ObPartitionArray& current_partitions, common::ObPartitionArray& backup_partitions); const common::ObPartitionArray &current_partitions, common::ObPartitionArray &backup_partitions);
int do_filter_pg_partitions(const ObPGKey& pg_key, ObPartitionArray& partitions); int do_filter_pg_partitions(const ObPGKey &pg_key, ObPartitionArray &partitions);
private: private:
typedef hash::ObHashMap<ObPartitionKey, ObPartitionBaseDataMetaRestoreReaderV2*> MetaReaderMap; typedef hash::ObHashMap<ObPartitionKey, ObPartitionBaseDataMetaRestoreReaderV2 *> MetaReaderMap;
bool is_inited_; bool is_inited_;
const ObPhysicalRestoreArg* restore_info_; const ObPhysicalRestoreArg *restore_info_;
const ObPhyRestoreMetaIndexStore* meta_indexs_; const ObPhyRestoreMetaIndexStore *meta_indexs_;
ObPGMetaPhysicalReader reader_; ObPGMetaPhysicalReader reader_;
ObPartitionGroupMeta pg_meta_; ObPartitionGroupMeta pg_meta_;
common::ObInOutBandwidthThrottle* bandwidth_throttle_; common::ObInOutBandwidthThrottle *bandwidth_throttle_;
int64_t last_read_size_; int64_t last_read_size_;
MetaReaderMap partition_reader_map_; MetaReaderMap partition_reader_map_;
common::ObArenaAllocator allocator_; common::ObArenaAllocator allocator_;
...@@ -244,22 +247,22 @@ public: ...@@ -244,22 +247,22 @@ public:
ObPGPartitionBaseDataMetaRestoreReaderV2(); ObPGPartitionBaseDataMetaRestoreReaderV2();
virtual ~ObPGPartitionBaseDataMetaRestoreReaderV2(); virtual ~ObPGPartitionBaseDataMetaRestoreReaderV2();
void reset(); void reset();
int init(const ObPartitionArray& partitions, ObPartitionGroupMetaRestoreReaderV2* reader); int init(const ObPartitionArray &partitions, ObPartitionGroupMetaRestoreReaderV2 *reader);
int fetch_pg_partition_meta_info(obrpc::ObPGPartitionMetaInfo& partition_meta_info); int fetch_pg_partition_meta_info(obrpc::ObPGPartitionMetaInfo &partition_meta_info);
virtual Type get_type() const virtual Type get_type() const
{ {
return BASE_DATA_META_OB_RESTORE_READER_V2; return BASE_DATA_META_OB_RESTORE_READER_V2;
} }
private: private:
int check_sstable_table_ids_in_table(const ObPartitionKey& pkey, const common::ObIArray<uint64_t>& table_ids); int check_sstable_table_ids_in_table(const ObPartitionKey &pkey, const common::ObIArray<uint64_t> &table_ids);
int check_sstable_ids_contain_schema_table_id(const hash::ObHashSet<uint64_t>& table_id_set, int check_sstable_ids_contain_schema_table_id(const hash::ObHashSet<uint64_t> &table_id_set,
const uint64_t schema_table_id, schema::ObSchemaGetterGuard& schema_guard); const uint64_t schema_table_id, schema::ObSchemaGetterGuard &schema_guard);
private: private:
bool is_inited_; bool is_inited_;
int64_t reader_index_; int64_t reader_index_;
common::ObArray<ObPartitionBaseDataMetaRestoreReaderV2*> partition_reader_array_; common::ObArray<ObPartitionBaseDataMetaRestoreReaderV2 *> partition_reader_array_;
int64_t schema_version_; int64_t schema_version_;
DISALLOW_COPY_AND_ASSIGN(ObPGPartitionBaseDataMetaRestoreReaderV2); DISALLOW_COPY_AND_ASSIGN(ObPGPartitionBaseDataMetaRestoreReaderV2);
}; };
......
...@@ -813,8 +813,12 @@ int ObPartGroupBackupTask::check_before_backup() ...@@ -813,8 +813,12 @@ int ObPartGroupBackupTask::check_before_backup()
ObExternBackupInfoMgr extern_backup_info_mgr; ObExternBackupInfoMgr extern_backup_info_mgr;
ObClusterBackupDest cluster_backup_dest; ObClusterBackupDest cluster_backup_dest;
ObBackupDest backup_dest; ObBackupDest backup_dest;
ObFakeBackupLeaseService fake_backup_lease_service; ObBackupPath path;
ObExternBackupInfo last_backup_info; ObStorageUtil util(false /*need retry*/);
bool is_exist = false;
// For nfs 4.2 may has bug, which makes open wrong file handle
// There just check file exist
if (!is_inited_) { if (!is_inited_) {
ret = OB_NOT_INIT; ret = OB_NOT_INIT;
STORAGE_LOG(WARN, "not inited", K(ret)); STORAGE_LOG(WARN, "not inited", K(ret));
...@@ -828,17 +832,13 @@ int ObPartGroupBackupTask::check_before_backup() ...@@ -828,17 +832,13 @@ int ObPartGroupBackupTask::check_before_backup()
STORAGE_LOG(WARN, "failed to set backup dest", K(ret), K(backup_arg)); STORAGE_LOG(WARN, "failed to set backup dest", K(ret), K(backup_arg));
} else if (OB_FAIL(cluster_backup_dest.set(backup_dest, backup_arg.incarnation_))) { } else if (OB_FAIL(cluster_backup_dest.set(backup_dest, backup_arg.incarnation_))) {
STORAGE_LOG(WARN, "failed to set cluster backup dest", K(ret), K(backup_dest)); STORAGE_LOG(WARN, "failed to set cluster backup dest", K(ret), K(backup_dest));
} else if (OB_FAIL(extern_backup_info_mgr.init(tenant_id, cluster_backup_dest, fake_backup_lease_service))) { } else if (OB_FAIL(ObBackupPathUtil::get_tenant_data_backup_info_path(cluster_backup_dest, tenant_id, path))) {
STORAGE_LOG(WARN, "failed to init extern backup info", K(ret), K(cluster_backup_dest)); LOG_WARN("failed to get tenant data backup info path", K(ret), K(backup_dest));
} else if (OB_FAIL(extern_backup_info_mgr.get_last_info(last_backup_info))) { } else if (OB_FAIL(util.is_exist(path.get_ptr(), backup_arg.storage_info_, is_exist))) {
STORAGE_LOG(WARN, "failed to get last info", K(ret), K(cluster_backup_dest)); LOG_WARN("failed to check extern backup file info exist", K(ret), K(path), K(backup_dest));
} else if (!last_backup_info.is_valid() || ObExternBackupInfo::DOING != last_backup_info.status_ || } else if (!is_exist) {
last_backup_info.backup_data_version_ != backup_arg.backup_data_version_ ||
last_backup_info.backup_schema_version_ != backup_arg.backup_schema_version_ ||
last_backup_info.backup_type_ != backup_arg.backup_type_ ||
last_backup_info.inc_backup_set_id_ != backup_arg.backup_set_id_) {
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
LOG_WARN("last backup info is unexpected", K(ret), K(last_backup_info), K(backup_arg)); LOG_WARN("extern backup info is not exist", K(ret), K(backup_arg), K(path));
} }
} }
return ret; return ret;
......
...@@ -3110,16 +3110,8 @@ int ObBackupPhysicalPGCtx::check_table_exist( ...@@ -3110,16 +3110,8 @@ int ObBackupPhysicalPGCtx::check_table_exist(
if (OB_UNLIKELY(!macro_index_store.is_inited() || !table_key.is_valid())) { if (OB_UNLIKELY(!macro_index_store.is_inited() || !table_key.is_valid())) {
ret = OB_INVALID_ARGUMENT; ret = OB_INVALID_ARGUMENT;
STORAGE_LOG(WARN, "invalid argument", K(ret), K(macro_index_store), K(table_key)); STORAGE_LOG(WARN, "invalid argument", K(ret), K(macro_index_store), K(table_key));
} else if (OB_FAIL(macro_index_store.get_macro_index_array(table_key, macro_index_array))) { } else if (OB_FAIL(macro_index_store.check_table_exist(table_key, is_exist))) {
if (OB_HASH_NOT_EXIST == ret) { STORAGE_LOG(WARN, "failed to check table exist", K(ret), K(table_key));
is_exist = false;
ret = OB_SUCCESS;
}
} else if (OB_ISNULL(macro_index_array)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("macro index array should not be NULL", K(ret), K(table_key));
} else {
is_exist = true;
} }
return ret; return ret;
} }
...@@ -3340,6 +3332,7 @@ int ObBackupCopyPhysicalTask::process() ...@@ -3340,6 +3332,7 @@ int ObBackupCopyPhysicalTask::process()
} }
} }
} }
STORAGE_LOG(INFO, "reuse backup macro count", K(block_info), K(copy_count), K(reuse_count));
} }
} }
if (OB_FAIL(ret)) { if (OB_FAIL(ret)) {
......
...@@ -3260,8 +3260,12 @@ int ObPartGroupMigrationTask::check_before_backup() ...@@ -3260,8 +3260,12 @@ int ObPartGroupMigrationTask::check_before_backup()
ObExternBackupInfoMgr extern_backup_info_mgr; ObExternBackupInfoMgr extern_backup_info_mgr;
ObClusterBackupDest cluster_backup_dest; ObClusterBackupDest cluster_backup_dest;
ObBackupDest backup_dest; ObBackupDest backup_dest;
ObFakeBackupLeaseService fake_backup_lease_service; ObBackupPath path;
ObExternBackupInfo last_backup_info; ObStorageUtil util(false /*need retry*/);
bool is_exist = false;
// For nfs 4.2 may has bug, which makes open wrong file handle
// There just check file exist
if (!is_inited_) { if (!is_inited_) {
ret = OB_NOT_INIT; ret = OB_NOT_INIT;
STORAGE_LOG(WARN, "not inited", K(ret)); STORAGE_LOG(WARN, "not inited", K(ret));
...@@ -3275,17 +3279,13 @@ int ObPartGroupMigrationTask::check_before_backup() ...@@ -3275,17 +3279,13 @@ int ObPartGroupMigrationTask::check_before_backup()
STORAGE_LOG(WARN, "failed to set backup dest", K(ret), K(backup_arg)); STORAGE_LOG(WARN, "failed to set backup dest", K(ret), K(backup_arg));
} else if (OB_FAIL(cluster_backup_dest.set(backup_dest, backup_arg.incarnation_))) { } else if (OB_FAIL(cluster_backup_dest.set(backup_dest, backup_arg.incarnation_))) {
STORAGE_LOG(WARN, "failed to set cluster backup dest", K(ret), K(backup_dest)); STORAGE_LOG(WARN, "failed to set cluster backup dest", K(ret), K(backup_dest));
} else if (OB_FAIL(extern_backup_info_mgr.init(tenant_id, cluster_backup_dest, fake_backup_lease_service))) { } else if (OB_FAIL(ObBackupPathUtil::get_tenant_data_backup_info_path(cluster_backup_dest, tenant_id, path))) {
STORAGE_LOG(WARN, "failed to init extern backup info", K(ret), K(cluster_backup_dest)); LOG_WARN("failed to get tenant data backup info path", K(ret), K(backup_dest));
} else if (OB_FAIL(extern_backup_info_mgr.get_last_info(last_backup_info))) { } else if (OB_FAIL(util.is_exist(path.get_ptr(), backup_arg.storage_info_, is_exist))) {
STORAGE_LOG(WARN, "failed to get last info", K(ret), K(cluster_backup_dest)); LOG_WARN("failed to check extern backup file info exist", K(ret), K(path), K(backup_dest));
} else if (!last_backup_info.is_valid() || ObExternBackupInfo::DOING != last_backup_info.status_ || } else if (!is_exist) {
last_backup_info.backup_data_version_ != backup_arg.backup_data_version_ ||
last_backup_info.backup_schema_version_ != backup_arg.backup_schema_version_ ||
last_backup_info.backup_type_ != backup_arg.backup_type_ ||
last_backup_info.inc_backup_set_id_ != backup_arg.backup_set_id_) {
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
LOG_WARN("last backup info is unexpected", K(ret), K(last_backup_info), K(backup_arg)); LOG_WARN("extern backup info is not exist", K(ret), K(backup_arg), K(path));
} }
} }
return ret; return ret;
...@@ -5785,7 +5785,8 @@ int ObBackupDag::init(const ObBackupDataType &backup_data_type, ObMigrateCtx &ct ...@@ -5785,7 +5785,8 @@ int ObBackupDag::init(const ObBackupDataType &backup_data_type, ObMigrateCtx &ct
ctx.replica_op_arg_.backup_arg_, ctx.replica_op_arg_.backup_arg_,
ctx.replica_op_arg_.key_))) { ctx.replica_op_arg_.key_))) {
LOG_WARN("failed to init physical backup ctx", K(ret), K(ctx.replica_op_arg_.backup_arg_)); LOG_WARN("failed to init physical backup ctx", K(ret), K(ctx.replica_op_arg_.backup_arg_));
} else if (ctx.replica_op_arg_.backup_arg_.is_incremental_backup() && OB_ISNULL(ctx.macro_indexs_)) { } else if (ctx.replica_op_arg_.backup_arg_.is_incremental_backup() && OB_ISNULL(ctx.macro_indexs_) &&
backup_data_type.is_major_backup()) {
ObPhyRestoreMacroIndexStoreV2 *phy_restore_macro_index_v2 = NULL; ObPhyRestoreMacroIndexStoreV2 *phy_restore_macro_index_v2 = NULL;
if (OB_ISNULL(phy_restore_macro_index_v2 = MIGRATOR.get_cp_fty()->get_phy_restore_macro_index_v2())) { if (OB_ISNULL(phy_restore_macro_index_v2 = MIGRATOR.get_cp_fty()->get_phy_restore_macro_index_v2())) {
ret = OB_ERR_UNEXPECTED; ret = OB_ERR_UNEXPECTED;
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册