/** * Copyright (c) 2021 OceanBase * OceanBase CE is licensed under Mulan PubL v2. * You can use this software according to the terms and conditions of the Mulan PubL v2. * You may obtain a copy of Mulan PubL v2 at: * http://license.coscl.org.cn/MulanPubL-2.0 * THIS SOFTWARE IS PROVIDED ON AN "AS IS" BASIS, WITHOUT WARRANTIES OF ANY KIND, * EITHER EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO NON-INFRINGEMENT, * MERCHANTABILITY OR FIT FOR A PARTICULAR PURPOSE. * See the Mulan PubL v2 for more details. */ #ifndef STORAGE_OB_PARTITION_BASE_DATA_BACKUP_H_ #define STORAGE_OB_PARTITION_BASE_DATA_BACKUP_H_ #include "share/ob_define.h" #include "share/restore/ob_restore_args.h" #include "share/scheduler/ob_dag_scheduler.h" #include "storage/ob_i_partition_base_data_reader.h" #include "lib/restore/ob_storage_path.h" #include "lib/thread/ob_dynamic_thread_pool.h" #include "lib/allocator/ob_concurrent_fifo_allocator.h" #include "lib/queue/ob_fixed_queue.h" #include "storage/ob_partition_service_rpc.h" #include "share/backup/ob_backup_struct.h" #include "share/backup/ob_backup_path.h" #include "lib/restore/ob_storage.h" #include "storage/ob_partition_base_data_physical_restore.h" #include "share/schema/ob_schema_getter_guard.h" #include "storage/backup/ob_partition_backup_struct.h" #include "storage/backup/ob_partition_base_data_physical_restore_v2.h" #include "storage/ob_i_partition_component_factory.h" #include "share/backup/ob_backup_file_lock_mgr.h" namespace oceanbase { // TODO(): remove using using namespace common; using namespace common::hash; using namespace share; using namespace blocksstable; using namespace obrpc; using namespace schema; namespace storage { class ObRestoreInfo; class ObPGStorage; class ObPartMigrationTask; class ObMigrateCtx; struct ObBackupSSTableInfo { ObBackupSSTableInfo() : sstable_meta_(), part_list_() {} ~ObBackupSSTableInfo() {} TO_STRING_KV(K_(sstable_meta), K_(part_list)); void reset(); bool is_valid() const; int assign(const ObBackupSSTableInfo& result); blocksstable::ObSSTableBaseMeta sstable_meta_; ObSArray part_list_; }; // macro block backup arg struct ObBackupMacroBlockArg final { ObBackupMacroBlockArg(); void reset(); bool is_valid() const; TO_STRING_KV(K_(fetch_arg), KP_(table_key_ptr), K_(need_copy)); obrpc::ObFetchMacroBlockArg fetch_arg_; const ObITable::TableKey* table_key_ptr_; bool need_copy_; }; class ObPartitionMetaBackupReader { public: ObPartitionMetaBackupReader(); virtual ~ObPartitionMetaBackupReader() {} int init(const share::ObPhysicalBackupArg& arg, const ObPartitionKey& pkey); int read_partition_meta(ObPartitionStoreMeta& partition_store_meta); int read_sstable_pair_list(const uint64_t backup_index, common::ObIArray& pair_list); int read_sstable_meta(const uint64_t backup_index, blocksstable::ObSSTableBaseMeta& sstable_meta); int64_t get_data_size() const { return data_size_; } int read_table_ids(common::ObIArray& table_id_array); int read_table_keys_by_table_id(const uint64_t table_id, ObIArray& table_keys_array); int read_backup_metas(ObPGPartitionStoreMeta*& partition_store_meta, common::ObIArray*& sstable_info_array, common::ObArray*& table_key_array); int fetch_table_ids(hash::ObHashSet& tables_ids); bool is_inited() const { return is_inited_; } private: int read_partition_meta_info(const ObPartitionKey& pkey, const int64_t backup_snapshot_version); int read_table_info(const ObPartitionKey& pkey, const uint64_t table_id, const int64_t multi_version_start, const int64_t version, ObPGStorage* pg_storage, ObFetchTableInfoResult& table_info); int read_all_sstable_metas(); int read_sstable_meta(ObITable::TableKey& table_key, ObBackupSSTableInfo& sstable_info); int prepare_partition_store_meta_info( const ObPGPartitionStoreMeta& partition_store_meta, const ObTablesHandle& handle); int build_backup_sstable_info(const ObSSTable* sstable, ObBackupSSTableInfo& sstable_info); private: bool is_inited_; int64_t data_size_; ObPGPartitionMetaInfo meta_info_; common::ObArray sstable_info_array_; common::ObArray table_key_array_; const share::ObPhysicalBackupArg* arg_; common::ObArenaAllocator meta_allocator_; int64_t table_count_; ObPartitionKey pkey_; DISALLOW_COPY_AND_ASSIGN(ObPartitionMetaBackupReader); }; class ObMacroBlockBackupSyncReader final { public: ObMacroBlockBackupSyncReader(); virtual ~ObMacroBlockBackupSyncReader(); int init(const share::ObPhysicalBackupArg& args, const ObITable::TableKey& table_key, const obrpc::ObFetchMacroBlockArg& macro_arg); int64_t get_data_size() const { return data_size_; } int get_macro_block_meta(blocksstable::ObFullMacroBlockMeta& meta, blocksstable::ObBufferReader& data); void reset(); TO_STRING_KV(K_(is_inited), K_(args), K_(data_size), K_(result_code), K_(is_data_ready), K_(macro_arg), K_(backup_index_tid), K_(full_meta), K_(data)); private: int process(); int get_macro_read_info(const obrpc::ObFetchMacroBlockArg& arg, blocksstable::ObMacroBlockCtx& macro_block_ctx, blocksstable::ObMacroBlockReadInfo& read_info); private: bool is_inited_; const share::ObPhysicalBackupArg* args_; common::ObArenaAllocator allocator_; int64_t data_size_; int32_t result_code_; bool is_data_ready_; obrpc::ObFetchMacroBlockArg macro_arg_; uint64_t backup_index_tid_; blocksstable::ObFullMacroBlockMeta full_meta_; ObMacroBlockHandle macro_handle_; blocksstable::ObBufferReader data_; ObPartitionKey pkey_; ObTableHandle store_handle_; ObSSTable* sstable_; DISALLOW_COPY_AND_ASSIGN(ObMacroBlockBackupSyncReader); }; class ObPartitionBaseDataMetaBackupReader { public: ObPartitionBaseDataMetaBackupReader(); virtual ~ObPartitionBaseDataMetaBackupReader(); int init( const common::ObPartitionKey& pkey, const ObDataStorageInfo& data_info, const ObPhysicalBackupArg& backup_arg); int fetch_partition_meta(ObPGPartitionStoreMeta& partition_store_meta); int fetch_sstable_meta(const uint64_t index_id, blocksstable::ObSSTableBaseMeta& sstable_meta); int64_t get_data_size() const { return reader_.get_data_size(); } int fetch_sstable_pair_list(const uint64_t index_id, common::ObIArray& pair_list); int fetch_all_table_ids(common::ObIArray& table_id_array); int fetch_table_keys(const uint64_t index_id, obrpc::ObFetchTableInfoResult& table_res); TO_STRING_KV(K_(pkey), KP(backup_arg_), K_(last_read_size), K_(partition_store_meta), K_(snapshot_version), K_(schema_version), K_(data_version)); private: int prepare(const common::ObPartitionKey& pkey, const ObDataStorageInfo& data_info); private: bool is_inited_; common::ObPartitionKey pkey_; const ObPhysicalBackupArg* backup_arg_; ObPartitionMetaBackupReader reader_; common::ObArenaAllocator allocator_; int64_t last_read_size_; ObPGPartitionStoreMeta partition_store_meta_; int64_t snapshot_version_; int64_t schema_version_; int64_t data_version_; DISALLOW_COPY_AND_ASSIGN(ObPartitionBaseDataMetaBackupReader); }; class ObPartitionGroupMetaBackupReader; class ObPhysicalBaseMetaBackupReader : public ObIPhysicalBaseMetaReader { public: ObPhysicalBaseMetaBackupReader(); virtual ~ObPhysicalBaseMetaBackupReader() {} int init(ObRestoreInfo& restore_info, const ObPartitionKey& pkey, const uint64_t table_id, ObPartitionGroupMetaBackupReader& reader); virtual int fetch_sstable_meta(blocksstable::ObSSTableBaseMeta& sstable_meta); virtual int fetch_macro_block_list(common::ObIArray& macro_block_list); virtual Type get_type() const { return BASE_DATA_META_BACKUP_READER; } private: bool is_inited_; ObRestoreInfo* restore_info_; ObPartitionGroupMetaBackupReader* reader_; common::ObArenaAllocator allocator_; ObPartitionKey pkey_; uint64_t table_id_; DISALLOW_COPY_AND_ASSIGN(ObPhysicalBaseMetaBackupReader); }; class ObPartitionMacroBlockBackupReader : public ObIPartitionMacroBlockReader { public: ObPartitionMacroBlockBackupReader(); virtual ~ObPartitionMacroBlockBackupReader(); int init(const ObPhysicalBackupArg& backup_arg, const ObIArray& list); virtual int get_next_macro_block(blocksstable::ObFullMacroBlockMeta& meta, blocksstable::ObBufferReader& data, blocksstable::MacroBlockId& src_macro_id); virtual Type get_type() const { return MACRO_BLOCK_BACKUP_READER; } virtual int64_t get_data_size() const { return read_size_; } int64_t get_block_count() const { return readers_.count(); } bool is_inited() { return is_inited_; } private: int schedule_macro_block_task(const ObPhysicalBackupArg& backup_arg, const obrpc::ObFetchMacroBlockArg& arg, const ObITable::TableKey& table_key, ObMacroBlockBackupSyncReader& reader); private: bool is_inited_; common::ObArray macro_list_; int64_t macro_idx_; common::ObArenaAllocator allocator_; common::ObArray readers_; int64_t read_size_; DISALLOW_COPY_AND_ASSIGN(ObPartitionMacroBlockBackupReader); }; class ObPartitionGroupMetaBackupReader { public: ObPartitionGroupMetaBackupReader(); virtual ~ObPartitionGroupMetaBackupReader(); int init(const ObPartitionGroupMeta& pg_meta, const ObPhysicalBackupArg& backup_arg); int get_partition_readers(const ObPartitionArray& partitions, common::ObIArray& partition_reader_array); int fetch_sstable_meta( const uint64_t index_id, const ObPartitionKey& pkey, blocksstable::ObSSTableBaseMeta& sstable_meta); int fetch_sstable_pair_list( const uint64_t index_id, const ObPartitionKey& pkey, common::ObIArray& pair_list); private: int prepare(const ObPartitionGroupMeta& pg_meta, const ObPhysicalBackupArg& backup_arg); private: bool is_inited_; common::ObPGKey pg_key_; const ObPhysicalBackupArg* backup_arg_; int64_t last_read_size_; hash::ObHashMap partition_reader_map_; common::ObArenaAllocator allocator_; DISALLOW_COPY_AND_ASSIGN(ObPartitionGroupMetaBackupReader); }; class ObPGPartitionBaseDataMetaBackupReader : public ObIPGPartitionBaseDataMetaObReader { public: ObPGPartitionBaseDataMetaBackupReader(); virtual ~ObPGPartitionBaseDataMetaBackupReader(); int init(const ObPartitionArray& partitions, ObPartitionGroupMetaBackupReader* reader); int fetch_pg_partition_meta_info(obrpc::ObPGPartitionMetaInfo& partition_meta_info); virtual Type get_type() const { return BASE_DATA_META_OB_BACKUP_READER; } private: bool is_inited_; int64_t reader_index_; common::ObArray partition_reader_array_; DISALLOW_COPY_AND_ASSIGN(ObPGPartitionBaseDataMetaBackupReader); }; class ObBackupFileAppender final { public: ObBackupFileAppender(); virtual ~ObBackupFileAppender(); int open(common::ObInOutBandwidthThrottle& bandwidth_throttle, const share::ObPhysicalBackupArg& backup_arg, const common::ObString& path, const ObBackupFileType type); int close(); bool is_valid() const; int append_meta_index(const common::ObIArray& meta_index_array); int append_macro_index( const common::ObIArray& macro_index_array, ObITable::TableKey& table_key); int append_partition_group_meta( const ObPartitionGroupMeta& pg_meta, ObBackupMetaIndex& meta_index, bool& is_uploaded); int append_partition_meta( const storage::ObPGPartitionStoreMeta& partition_meta, ObBackupMetaIndex& meta_index, bool& is_uploaded); int append_sstable_metas(const common::ObIArray& sstable_info_array, ObBackupMetaIndex& meta_index, bool& is_uploaded); int append_table_keys(const common::ObIArray& table_keys, ObBackupMetaIndex& meta_index, bool& is_uploaded); int append_macroblock_data(blocksstable::ObFullMacroBlockMeta& macro_meta, blocksstable::ObBufferReader& macro_data, ObBackupTableMacroIndex& macro_index); int append_backup_pg_meta_info( const ObBackupPGMetaInfo& pg_meta_info, ObBackupMetaIndex& meta_index, bool& is_uploaded); int sync_upload(); int64_t get_upload_size() { return file_offset_ + data_buffer_.pos(); } TO_STRING_KV(K_(is_opened), K_(file_offset), K_(max_buf_size), KP(backup_arg_)); private: int get_data_version(const ObBackupFileType data_type, uint16& data_version); int is_exist(const common::ObString& uri, bool& exist); template int write(const T& backup_base, int64_t& write_size, bool& is_uploaded); int write_tail(); int open(const common::ObString path); private: const static int64_t MAX_DATA_BUF_LENGTH = 1LL << 22; // 4M const static int64_t MAX_INDEX_BUF_LENGTH = 1LL << 18; // 256KB const static int64_t MAX_IDLE_TIME = 10 * 1000LL * 1000LL; // 10s bool is_opened_; uint64_t file_offset_; int64_t max_buf_size_; ObBackupFileType file_type_; ObSelfBufferWriter tmp_buffer_; ObSelfBufferWriter data_buffer_; ObStorageAppender storage_appender_; ObBackupCommonHeader* common_header_; const share::ObPhysicalBackupArg* backup_arg_; common::ObInOutBandwidthThrottle* bandwidth_throttle_; int32_t inner_error_; DISALLOW_COPY_AND_ASSIGN(ObBackupFileAppender); }; class ObBackupMetaWriter final { public: ObBackupMetaWriter(); virtual ~ObBackupMetaWriter(); int open(common::ObInOutBandwidthThrottle& bandwidth_throttle, ObIArray& task_list); int process(); int close(); TO_STRING_KV(K_(is_inited), K_(task_id), KP_(cp_fty), K_(meta_appender), K_(index_appender)); private: int prepare_appender(common::ObInOutBandwidthThrottle& bandwidth_throttle, const share::ObPhysicalBackupArg& arg); int check_task(const ObIArray& task_list) const; int get_min_snapshot_version(const common::ObIArray& table_key_array, int64_t& snapshot_version); int append_meta_index(const bool need_upload, ObBackupMetaIndex& meta_index); void set_meta_index(const ObPartitionKey& pkey, ObBackupMetaIndex& meta_index); int write_pg_meta(const ObPartitionKey& pgkey, const ObPartitionGroupMeta& pg_meta); int write_partition_meta(const ObPartitionKey& pkey, const ObPGPartitionStoreMeta& partition_meta); int write_sstable_metas(const ObPartitionKey& pkey, const common::ObIArray& sstable_info_array); int write_table_keys(const ObPartitionKey& pkey, const common::ObArray& table_keys); int create_partition_store_meta_info(const ObPGPartitionStoreMeta& partition_store_meta, const common::ObIArray& sstable_info_array, const common::ObIArray& table_keys, ObBackupPartitionStoreMetaInfo& partition_store_meta_info); int write_backup_pg_meta_info(const ObBackupPGMetaInfo& pg_meta_info); private: bool is_inited_; int64_t task_id_; ObIArray* task_list_; ObIPartitionComponentFactory* cp_fty_; ObBackupFileAppender meta_appender_; ObBackupFileAppender index_appender_; ObArray meta_index_array_; DISALLOW_COPY_AND_ASSIGN(ObBackupMetaWriter); }; struct ObBackupMacroBlockInfo final { ObBackupMacroBlockInfo(); bool is_valid() const; TO_STRING_KV(K_(table_key), K_(start_index), K_(cur_block_count), K_(total_block_count)); ObITable::TableKey table_key_; int64_t start_index_; int64_t cur_block_count_; int64_t total_block_count_; }; class ObBackupPhysicalPGCtx { public: class SubTask final { public: SubTask(); virtual ~SubTask(); void reset(); TO_STRING_KV(K_(block_info), K_(block_count)); ObArray block_info_; int64_t block_count_; }; struct MacroIndexMergePoint final { MacroIndexMergePoint(); void reset(); bool is_valid() const; TO_STRING_KV(K_(table_id), K_(sstable_idx), KP_(macro_index_array)); uint64_t table_id_; int64_t sstable_idx_; const ObArray* macro_index_array_; }; struct MacroIndexRetryPoint final { MacroIndexRetryPoint(); void reset(); bool is_valid() const; TO_STRING_KV(K_(table_key), K_(last_idx)); ObITable::TableKey table_key_; int64_t last_idx_; }; public: const static uint64_t DEFAULT_WAIT_TIME = 10 * 1000 * 1000; // 10s const static uint64_t MAX_MACRO_BLOCK_COUNT_PER_TASK = 512; // 1GB per backup data file ObBackupPhysicalPGCtx(); virtual ~ObBackupPhysicalPGCtx(); // TODO() delete this interface later int open(common::ObInOutBandwidthThrottle& bandwidth_throttle, const share::ObPhysicalBackupArg& arg, const common::ObPGKey& pg_key); int open(common::ObInOutBandwidthThrottle& bandwidth_throttle, const share::ObPhysicalBackupArg& arg, const common::ObPGKey& pg_key, const ObBackupDataType& backup_data_type); int init(common::ObInOutBandwidthThrottle& bandwidth_throttle, const share::ObPhysicalBackupArg& arg, const common::ObPGKey& pg_key); int open(); int close(); void reset(); bool is_valid() const; int64_t get_task_count() const; void set_result(const int32_t ret); int get_result() const; int add_backup_macro_block_info(const ObBackupMacroBlockInfo& block_info); int wait_for_turn(const int64_t task_idx); int finish_task(const int64_t task_idx); int fetch_prev_macro_index(const ObPhyRestoreMacroIndexStoreV2& macro_index_store, const ObBackupMacroBlockArg& macro_arg, ObBackupTableMacroIndex& macro_index); int check_table_exist( const ObITable::TableKey& table_key, const ObPhyRestoreMacroIndexStoreV2& macro_index_store, bool& is_exist); bool is_opened() const { return is_opened_; } ObBackupDataType::BackupDataType get_backup_data_type() const { return backup_data_type_.type_; } TO_STRING_KV(K_(macro_block_count), K_(base_task_id), K_(retry_cnt), K_(task_turn), K_(index_merge_point), K_(result), K_(pg_key), K_(table_keys), "task_count", tasks_.count(), K_(macro_index_appender)); common::ObInOutBandwidthThrottle* bandwidth_throttle_; ObArray table_keys_; ObArray tasks_; ObBackupFileAppender macro_index_appender_; int64_t macro_block_count_; int64_t block_count_per_task_; int64_t base_task_id_; int64_t retry_cnt_; private: int fetch_available_sub_task(SubTask*& sub_task); int init_macro_index_appender(); int init_already_backup_data(); // for retry int fetch_retry_points(const ObString& path, const ObString& storage_info); int reuse_already_backup_data(ObBackupMacroBlockInfo& block_info); // for retry int get_tenant_pg_data_path(const ObBackupBaseDataPathInfo& path_info, ObBackupPath& path); int get_macro_block_index_path( const ObBackupBaseDataPathInfo& path_info, const int64_t retry_cnt, ObBackupPath& path); private: common::ObThreadCond cond_; volatile int64_t task_turn_; MacroIndexMergePoint index_merge_point_; common::SpinRWLock lock_; int32_t result_; common::ObPGKey pg_key_; const share::ObPhysicalBackupArg* backup_arg_; bool find_breakpoint_; // make sure retry backup only one breakpoint ObArray retry_points_; bool is_opened_; ObBackupDataType backup_data_type_; bool is_inited_; DISALLOW_COPY_AND_ASSIGN(ObBackupPhysicalPGCtx); }; class ObBackupCopyPhysicalTask : public share::ObITask { public: ObBackupCopyPhysicalTask(); virtual ~ObBackupCopyPhysicalTask(); virtual int generate_next_task(ObITask*& next_task) override; int init(const int64_t task_idx, ObMigrateCtx& ctx); int init(const int64_t task_idx, const ObITable::TableKey& already_backup_table_key, ObMigrateCtx& ctx); virtual int process() override; private: int get_macro_block_backup_reader(const ObIArray& list, const ObPhysicalBackupArg& backup_arg, ObPartitionMacroBlockBackupReader*& reader); int fetch_backup_macro_block_arg(const share::ObPhysicalBackupArg& backup_arg, const ObITable::TableKey& table_key, const int64_t macro_idx, ObBackupMacroBlockArg& macro_arg); int fetch_physical_block_with_retry( const common::ObIArray& list, const int64_t copy_count, const int64_t reuse_count); int backup_physical_block( const common::ObIArray& list, const int64_t copy_count, const int64_t reuse_count); int get_datafile_appender(const ObITable::TableType& table_type, const share::ObPhysicalBackupArg& arg, const common::ObPGKey& pg_key, ObBackupFileAppender& macro_file); int backup_block_data(ObPartitionMacroBlockBackupReader& reader, ObBackupFileAppender& macro_file, ObBackupTableMacroIndex& block_index); int backup_block_index(const int64_t reuse_count, const ObIArray& list, ObIArray& macro_indexs); int reuse_block_index(const ObIArray& list, ObIArray& macro_indexs, int64_t& reuse_count); int calc_migrate_data_statics(const int64_t copy_count, const int64_t reuse_count); private: static const int64_t OB_FETCH_MAJOR_BLOCK_RETRY_INTERVAL = 1 * 1000 * 1000L; // 1s bool is_inited_; ObMigrateCtx* ctx_; int64_t task_idx_; int64_t base_task_id_; // default=0, base_task_id_ + task_idx_ = data file name; ObBackupPhysicalPGCtx* backup_pg_ctx_; ObBackupPhysicalPGCtx::SubTask* sub_task_; ObIPartitionComponentFactory* cp_fty_; ObSSTableMacroBlockChecker checker_; ObITable::TableKey already_backup_table_key_; int64_t output_macro_data_bytes_; int64_t input_macro_data_bytes_; DISALLOW_COPY_AND_ASSIGN(ObBackupCopyPhysicalTask); }; class ObBackupFinishTask : public share::ObITask { public: ObBackupFinishTask(); virtual ~ObBackupFinishTask(); int init(ObMigrateCtx& migrate_ctx); virtual int process() override; private: bool is_inited_; ObMigrateCtx* ctx_; DISALLOW_COPY_AND_ASSIGN(ObBackupFinishTask); }; struct ObBackupMacroData final { static const int64_t BACKUP_MARCO_DATA_VERSION = 1; OB_UNIS_VERSION(BACKUP_MARCO_DATA_VERSION); public: ObBackupMacroData(blocksstable::ObBufferHolder& meta, blocksstable::ObBufferReader& data); TO_STRING_KV(K_(data), K_(meta)); private: blocksstable::ObBufferHolder& meta_; blocksstable::ObBufferReader& data_; DISALLOW_COPY_AND_ASSIGN(ObBackupMacroData); }; template int ObBackupFileAppender::write(const T& backup_base, int64_t& write_size, bool& is_uploaded) { int ret = OB_SUCCESS; const int64_t need_write_size = backup_base.get_serialize_size(); write_size = 0; is_uploaded = false; if (OB_UNLIKELY(!is_opened_)) { ret = OB_NOT_INIT; STORAGE_LOG(WARN, "backup file appender not init", K(ret), K(is_opened_)); } else if (0 == need_write_size) { ret = OB_INVALID_ARGUMENT; STORAGE_LOG(WARN, "ObBackupFileAppender write nothing", K(ret), K(need_write_size)); } else if (data_buffer_.length() + need_write_size > max_buf_size_) { if (OB_FAIL(sync_upload())) { STORAGE_LOG(WARN, "upload buffer fail", K(ret)); } else { is_uploaded = true; } } if (OB_SUCC(ret)) { if (OB_ISNULL(common_header_)) { const int64_t header_len = sizeof(ObBackupCommonHeader); if (data_buffer_.length() + header_len + need_write_size > max_buf_size_) { ret = OB_BUF_NOT_ENOUGH; STORAGE_LOG(WARN, "data buffer not enough", K(ret), K(header_len), K(need_write_size)); } else { char* header_buf = data_buffer_.data(); if (OB_FAIL(data_buffer_.advance_zero(header_len))) { STORAGE_LOG(WARN, "advance failed", K(ret), K(header_len)); } else { common_header_ = reinterpret_cast(header_buf); common_header_->reset(); common_header_->compressor_type_ = ObCompressorType::NONE_COMPRESSOR; common_header_->data_type_ = file_type_; if (OB_FAIL(get_data_version(file_type_, common_header_->data_version_))) { STORAGE_LOG(WARN, "get common header data version fail", K(ret), K_(file_type)); } } } } } if (OB_SUCC(ret)) { int64_t last_pos = data_buffer_.pos(); if (OB_FAIL(data_buffer_.write_serialize(backup_base))) { STORAGE_LOG(WARN, "data buffer write fail", K(ret), K(need_write_size)); } else if (data_buffer_.pos() - last_pos > need_write_size) { ret = OB_ERR_UNEXPECTED; STORAGE_LOG(ERROR, "serialize size must not large than need write size", K(ret), "cur_pos", data_buffer_.pos(), K(last_pos), K(need_write_size), K(backup_base)); } else { write_size = data_buffer_.pos() - last_pos; common_header_->data_length_ += write_size; } } if (OB_FAIL(ret)) { if (OB_SUCCESS == inner_error_) { inner_error_ = ret; } } return ret; } class ObBackupMetaIndexReformer { public: ObBackupMetaIndexReformer(); virtual ~ObBackupMetaIndexReformer(); int init(const ObBackupBaseDataPathInfo& path_info, share::ObIBackupLeaseService& backup_lease_service); int upload_backup_meta_index(const common::ObIArray& backup_meta_indexs); private: int upload_backup_meta_index_(const common::ObIArray& backup_meta_indexs); int64_t get_buffer_size_(const common::ObIArray& backup_meta_index); int write_buf_(const common::ObIArray& backup_meta_index, ObSelfBufferWriter& data_buffer); private: bool is_inited_; share::ObBackupFileSpinLock lock_; share::ObIBackupLeaseService* backup_lease_service_; ObBackupBaseDataPathInfo path_info_; private: DISALLOW_COPY_AND_ASSIGN(ObBackupMetaIndexReformer); }; } // namespace storage } // namespace oceanbase #endif /* SRC_STORAGE_OB_PARTITION_BASE_DATA_BACKUP_H_ */