diff --git a/src/rootserver/backup/ob_backup_data_set_task_mgr.cpp b/src/rootserver/backup/ob_backup_data_set_task_mgr.cpp index 722548a5b2..0c51857c2a 100644 --- a/src/rootserver/backup/ob_backup_data_set_task_mgr.cpp +++ b/src/rootserver/backup/ob_backup_data_set_task_mgr.cpp @@ -491,7 +491,14 @@ int ObBackupSetTaskMgr::backup_user_meta_() } else if (OB_FAIL(do_backup_root_key_())) { LOG_WARN("[DATA_BACKUP]failed to do backup root key", K(ret)); } else if (ls_task.count() == finish_cnt) { - ROOTSERVICE_EVENT_ADD("backup_data", "before_backup_data"); + ROOTSERVICE_EVENT_ADD("backup_data", + "before_backup_data", + "tenant_id", + job_attr_->tenant_id_, + "job_id", + job_attr_->job_id_, + "task_id", + set_task_attr_.task_id_); share::SCN consistent_scn; bool need_change_meta_turn = false; if (OB_FAIL(check_need_change_meta_turn_(ls_task, need_change_meta_turn))) { @@ -542,6 +549,7 @@ int ObBackupSetTaskMgr::calc_consistent_scn_(ObIArray { int ret = OB_SUCCESS; consistent_scn.set_min(); + DEBUG_SYNC(BEFORE_CALC_CONSISTENT_SCN); // let consistent_scn be the biggest max_tablet_checkpoint_scn_ of all the ls and the cur gts. if (OB_FAIL(ObBackupDataScheduler::get_backup_scn(*sql_proxy_, job_attr_->tenant_id_, true, consistent_scn))) { LOG_WARN("failed to get backup scn", K(ret), "tenant_id", job_attr_->tenant_id_); @@ -737,7 +745,14 @@ int ObBackupSetTaskMgr::merge_tablet_to_ls_info_(const share::SCN &consistent_sc } else if (OB_FAIL(generate_ls_tasks_(ls_ids, type))) { LOG_WARN("failed to generate ls tasks", K(ret), K(ls_ids), K(type)); } else { - ROOTSERVICE_EVENT_ADD("backup_data", "after_backup_consistent_scn"); + ROOTSERVICE_EVENT_ADD("backup_data", + "after_backup_consistent_scn", + "tenant_id", + job_attr_->tenant_id_, + "job_id", + job_attr_->job_id_, + "task_id", + set_task_attr_.task_id_); } if (trans_.is_started()) { int tmp_ret = OB_SUCCESS; diff --git a/src/share/ob_debug_sync_point.h b/src/share/ob_debug_sync_point.h index 587a6ce271..960680960d 100755 --- a/src/share/ob_debug_sync_point.h +++ b/src/share/ob_debug_sync_point.h @@ -568,6 +568,7 @@ class ObString; ACT(BEFORE_SEND_PARALLEL_CREATE_TABLE,)\ ACT(BEFORE_DROP_TENANT,)\ ACT(BEFORE_WAIT_SYS_LS_END_SCN,)\ + ACT(BEFORE_CALC_CONSISTENT_SCN,)\ ACT(MAX_DEBUG_SYNC_POINT,) DECLARE_ENUM(ObDebugSyncPoint, debug_sync_point, OB_DEBUG_SYNC_POINT_DEF); diff --git a/src/storage/high_availability/ob_ls_migration.cpp b/src/storage/high_availability/ob_ls_migration.cpp index 36c73cc88f..1eb198ebc8 100644 --- a/src/storage/high_availability/ob_ls_migration.cpp +++ b/src/storage/high_availability/ob_ls_migration.cpp @@ -2834,19 +2834,17 @@ int ObTabletMigrationTask::try_update_tablet_() ls, &ctx_->ha_table_info_mgr_, ha_tablets_builder))) { LOG_WARN("failed to init ha tablets builder", K(ret), KPC(ctx_)); } else { - //Here inner tablet copy data before clog replay, and now just create a new tablet to replace it. - //Data tablet copy data during clog replay, so the data tablet can only be updated. - if (copy_tablet_ctx_->tablet_id_.is_ls_inner_tablet()) { - if (OB_FAIL(ha_tablets_builder.create_or_update_tablets())) { - LOG_WARN("failed to create or update tablets", K(ret), KPC(ctx_)); - } - } else { - if (OB_FAIL(ha_tablets_builder.update_local_tablets())) { - LOG_WARN("failed to create or update tablets", K(ret), KPC(ctx_), KPC(copy_tablet_ctx_)); + if (OB_FAIL(ctx_->ha_table_info_mgr_.remove_tablet_table_info(copy_tablet_ctx_->tablet_id_))) { + if (OB_HASH_NOT_EXIST == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to remove tablet info", K(ret), KPC(copy_tablet_ctx_), KPC(ctx_)); } } if (OB_FAIL(ret)) { + } else if (copy_tablet_ctx_->tablet_id_.is_ls_inner_tablet() && OB_FAIL(ha_tablets_builder.create_or_update_tablets())) { + LOG_WARN("failed to create or update inner tablet", K(ret), KPC(ctx_)); } else if (OB_FAIL(ha_tablets_builder.build_tablets_sstable_info())) { LOG_WARN("failed to build tablets sstable info", K(ret), KPC(ctx_), KPC(copy_tablet_ctx_)); } else if (OB_FAIL(ctx_->ha_table_info_mgr_.check_tablet_table_info_exist(copy_tablet_ctx_->tablet_id_, is_exist))) { @@ -4019,8 +4017,8 @@ int ObTabletGroupMigrationTask::process() LOG_WARN("tablet group migration task do not init", K(ret)); } else if (ctx_->is_failed()) { //do nothing - } else if (OB_FAIL(try_update_local_tablets_())) { - LOG_WARN("failed to try update local tablets", K(ret), KPC(ctx_)); + } else if (OB_FAIL(try_remove_tablets_info_())) { + LOG_WARN("failed to try remove tablets info", K(ret), KPC(ctx_)); } else if (OB_FAIL(build_tablets_sstable_info_())) { LOG_WARN("failed to build tablets sstable info", K(ret)); } else { @@ -4154,7 +4152,7 @@ int ObTabletGroupMigrationTask::build_tablets_sstable_info_() return ret; } -int ObTabletGroupMigrationTask::try_update_local_tablets_() +int ObTabletGroupMigrationTask::try_remove_tablets_info_() { int ret = OB_SUCCESS; bool is_in_retry = false; @@ -4172,13 +4170,11 @@ int ObTabletGroupMigrationTask::try_update_local_tablets_() //do nothing } else if (OB_FAIL(try_remove_tablets_info_())) { LOG_WARN("failed to try remove tablets info", K(ret), KPC(ctx_)); - } else if (OB_FAIL(ha_tablets_builder_.update_local_tablets())) { - LOG_WARN("failed to build tablets sstable info", K(ret), KPC(ctx_)); } return ret; } -int ObTabletGroupMigrationTask::try_remove_tablets_info_() +int ObTabletGroupMigrationTask::remove_tablets_info_() { int ret = OB_SUCCESS; if (!is_inited_) { diff --git a/src/storage/high_availability/ob_ls_migration.h b/src/storage/high_availability/ob_ls_migration.h index 845125481f..11cb5767c0 100644 --- a/src/storage/high_availability/ob_ls_migration.h +++ b/src/storage/high_availability/ob_ls_migration.h @@ -503,8 +503,8 @@ public: private: int build_tablets_sstable_info_(); int generate_tablet_migration_dag_(); - int try_update_local_tablets_(); int try_remove_tablets_info_(); + int remove_tablets_info_(); int record_server_event_(); private: diff --git a/src/storage/high_availability/ob_physical_copy_task.cpp b/src/storage/high_availability/ob_physical_copy_task.cpp index 5cb6ee187c..26593334a7 100644 --- a/src/storage/high_availability/ob_physical_copy_task.cpp +++ b/src/storage/high_availability/ob_physical_copy_task.cpp @@ -1514,7 +1514,7 @@ int ObTabletCopyFinishTask::check_finish_copy_tablet_data_valid_() LOG_WARN("tablet here should only has one", K(ret), KPC(tablet)); } else if (OB_FAIL(ObStorageHATabletBuilderUtil::check_remote_logical_sstable_exist(tablet, is_logical_sstable_exist))) { LOG_WARN("failed to check remote logical sstable exist", K(ret), KPC(tablet)); - } else if (is_logical_sstable_exist && tablet->get_tablet_meta().ha_status_.is_restore_status_full()) { + } else if (is_logical_sstable_exist) { ret = OB_ERR_UNEXPECTED; LOG_WARN("tablet still has remote logical sstable, unexpected !!!", K(ret), KPC(tablet)); } else if (OB_FAIL(tablet->fetch_table_store(table_store_wrapper))) { diff --git a/src/storage/high_availability/ob_storage_ha_reader.cpp b/src/storage/high_availability/ob_storage_ha_reader.cpp index 22b5985642..9d666250aa 100644 --- a/src/storage/high_availability/ob_storage_ha_reader.cpp +++ b/src/storage/high_availability/ob_storage_ha_reader.cpp @@ -1687,11 +1687,6 @@ int ObCopySSTableInfoObProducer::init( } else if (OB_ISNULL(tablet = tablet_handle_.get_obj())) { ret = OB_ERR_UNEXPECTED; LOG_WARN("tablet should not be NULL", K(ret), K(tablet_sstable_info)); - } else if (!tablet_sstable_info.minor_sstable_scn_range_.is_empty() - && tablet->get_tablet_meta().clog_checkpoint_scn_ < tablet_sstable_info.minor_sstable_scn_range_.end_scn_) { - ret = OB_SSTABLE_NOT_EXIST; - LOG_WARN("src tablet clog_checkpoint_scn is smaller than dest needed log ts", - K(ret), K(tablet_sstable_info), KPC(tablet)); } else if (!tablet_sstable_info.ddl_sstable_scn_range_.is_empty()) { if (tablet->get_tablet_meta().get_ddl_sstable_start_scn() < tablet_sstable_info.ddl_sstable_scn_range_.start_scn_) { ret = OB_ERR_UNEXPECTED; @@ -1786,13 +1781,7 @@ int ObCopySSTableInfoObProducer::check_need_copy_sstable_( need_copy_sstable = sstable->get_key().get_snapshot_version() > tablet_sstable_info_.max_major_sstable_snapshot_; } else if (sstable->is_minor_sstable()) { - if (tablet_sstable_info_.minor_sstable_scn_range_.is_empty()) { - need_copy_sstable = false; - } else if (sstable->get_key().scn_range_.end_scn_ <= tablet_sstable_info_.minor_sstable_scn_range_.start_scn_) { - need_copy_sstable = false; - } else { - need_copy_sstable = true; - } + need_copy_sstable = true; } else if (sstable->is_ddl_dump_sstable()) { const SCN ddl_sstable_start_scn = tablet_sstable_info_.ddl_sstable_scn_range_.start_scn_; const SCN ddl_sstable_end_scn = tablet_sstable_info_.ddl_sstable_scn_range_.end_scn_; diff --git a/src/storage/high_availability/ob_storage_ha_tablet_builder.cpp b/src/storage/high_availability/ob_storage_ha_tablet_builder.cpp index cee20e45aa..76c78fa64b 100644 --- a/src/storage/high_availability/ob_storage_ha_tablet_builder.cpp +++ b/src/storage/high_availability/ob_storage_ha_tablet_builder.cpp @@ -531,7 +531,6 @@ int ObStorageHATabletsBuilder::create_or_update_tablet_( ObArenaAllocator allocator("HATabBuilder"); ObTabletHandle local_tablet_hdl; ObTablesHandleArray major_tables; - ObTablesHandleArray remote_logical_table; ObBatchUpdateTableStoreParam param; ObStorageSchema storage_schema; compaction::ObMediumCompactionInfoList medium_info_list; @@ -556,27 +555,14 @@ int ObStorageHATabletsBuilder::create_or_update_tablet_( } else if (OB_FAIL(ls->rebuild_create_tablet(tablet_info.param_, keep_old))) { LOG_WARN("failed to create or update tablet", K(ret), K(tablet_info)); } else if (tablet_info.param_.is_empty_shell() || tablet_info.param_.ha_status_.is_restore_status_undefined()) { - // empty shell or UNDEFINED tablet does not need to create remote logical sstable. + // empty shell or UNDEFINED tablet does not need to reuse any sstable. } else { - if (OB_FAIL(create_tablet_remote_logical_sstable_(allocator, tablet_info.tablet_id_, remote_logical_table))) { - LOG_WARN("failed to create tablet remote logical sstable", K(ret), K(tablet_info)); - } else if (remote_logical_table.empty()) { - //do nothing - } else if (OB_FAIL(param.tables_handle_.assign(remote_logical_table))) { - LOG_WARN("failed to assign tables handle", K(ret), K(remote_logical_table), K(tablet_info)); - } else if (FALSE_IT(param.tablet_meta_ = &tablet_info.param_)) { - } else if (FALSE_IT(param.rebuild_seq_ = ls->get_rebuild_seq())) { - } else if (OB_FAIL(ls->build_ha_tablet_new_table_store(tablet_info.tablet_id_, param))) { - LOG_WARN("failed to build ha tablet new table store", K(ret), K(remote_logical_table), K(tablet_info)); - } - - if (OB_FAIL(ret)) { - } else if (tablet_info.param_.transfer_info_.has_transfer_table_) { + if (tablet_info.param_.transfer_info_.has_transfer_table_) { //do nothing } else if (OB_FAIL(create_tablet_with_major_sstables_(ls, tablet_info, major_tables, storage_schema, medium_info_list))) { LOG_WARN("failed to crete tablet with major sstables", K(ret), KPC(ls), K(tablet_info), K(major_tables)); } else { - LOG_INFO("succeed build ha table new table store", K(tablet_info), K(remote_logical_table)); + LOG_INFO("succeed build ha table new table store", K(tablet_info), K(major_tables)); } } return ret; @@ -857,7 +843,7 @@ int ObStorageHATabletsBuilder::build_copy_tablet_sstable_info_arg_( //minor if (OB_SUCC(ret)) { - if (OB_FAIL(get_remote_logical_minor_scn_range_(minor_sstable_array, tablet, arg.minor_sstable_scn_range_))) { + if (OB_FAIL(get_minor_scn_range_(minor_sstable_array, tablet, arg.minor_sstable_scn_range_))) { LOG_WARN("failed to get sstable max end log ts", K(ret), K(tablet_id), K(param_)); } } @@ -921,7 +907,7 @@ int ObStorageHATabletsBuilder::get_major_sstable_max_snapshot_( return ret; } -int ObStorageHATabletsBuilder::get_remote_logical_minor_scn_range_( +int ObStorageHATabletsBuilder::get_minor_scn_range_( const ObSSTableArray &minor_sstable_array, ObTablet *tablet, ObScnRange &scn_range) @@ -929,8 +915,6 @@ int ObStorageHATabletsBuilder::get_remote_logical_minor_scn_range_( int ret = OB_SUCCESS; scn_range.reset(); ObArray sstables; - scn_range.start_scn_ = ObTabletMeta::INIT_CLOG_CHECKPOINT_SCN; - scn_range.end_scn_ = ObTabletMeta::INIT_CLOG_CHECKPOINT_SCN; if (!is_inited_) { ret = OB_NOT_INIT; @@ -938,30 +922,26 @@ int ObStorageHATabletsBuilder::get_remote_logical_minor_scn_range_( } else if (OB_ISNULL(tablet)) { ret = OB_INVALID_ARGUMENT; LOG_WARN("get remote logical minor scn range get invalid argument", K(ret), KP(tablet)); - } else if (tablet->get_tablet_meta().has_transfer_table()) { - //transfer tablet should copy whole sstable from src; - scn_range.start_scn_ = ObTabletMeta::INIT_CLOG_CHECKPOINT_SCN; - scn_range.end_scn_ = tablet->get_clog_checkpoint_scn(); } else if (minor_sstable_array.count() > 0 && OB_FAIL(minor_sstable_array.get_all_tables(sstables))) { LOG_WARN("failed to get all tables", K(ret), K(param_)); } else { for (int64_t i = 0; OB_SUCC(ret) && i < sstables.count(); ++i) { const ObITable *table = sstables.at(i); - const ObSSTable *sstable = nullptr; if (OB_ISNULL(table)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("sstable should not be NULL", K(ret), KP(table), K(param_)); - } else if (!table->is_minor_sstable()) { + } else if (table->is_remote_logical_minor_sstable()) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("sstable type is unexpected", K(ret), KP(table), K(param_)); - } else if (FALSE_IT(sstable = static_cast(table))) { - } else if (sstable->is_remote_logical_minor_sstable()) { - scn_range.start_scn_ = sstable->get_key().get_start_scn(); - scn_range.end_scn_ = sstable->get_key().get_end_scn(); - break; + LOG_WARN("tablet should not has remote logical minor sstable, unexpected", K(ret), KPC(tablet), KPC(table)); } } + + if (OB_SUCC(ret)) { + //need copy src all minor sstables for tablet meta merge, do not need calculate sstable version range. + scn_range.start_scn_.set_base(); + scn_range.end_scn_.set_max(); + } } return ret; } @@ -1193,230 +1173,6 @@ int ObStorageHATabletsBuilder::remove_uncomplete_tablet_( return ret; } -int ObStorageHATabletsBuilder::create_tablet_remote_logical_sstable_( - common::ObArenaAllocator &allocator, - const common::ObTabletID &tablet_id, - ObTablesHandleArray &tables_handle) -{ - int ret = OB_SUCCESS; - ObTabletHandle tablet_handle; - ObTablet *tablet = nullptr; - SCN start_scn; - SCN end_scn; - ObArray minor_tables; - ObTableHandleV2 table_handle; - - if (!is_inited_) { - ret = OB_NOT_INIT; - LOG_WARN("storage ha tablets builder do not init", K(ret)); - } else if (!tablet_id.is_valid()) { - ret = OB_INVALID_ARGUMENT; - LOG_WARN("create tablet remote logical sstable get invalid argument", K(ret), K(tablet_id)); - } else if (OB_FAIL(param_.ls_->ha_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), KP(tablet), K(tablet_id)); - } else if (tablet->get_tablet_meta().has_transfer_table()) { - LOG_INFO("has transfer table do not create remote logical table", K(tablet_id)); - } else if (FALSE_IT(start_scn = tablet->get_tablet_meta().start_scn_)) { - } else if (FALSE_IT(end_scn = tablet->get_tablet_meta().clog_checkpoint_scn_)) { - } else if (start_scn > end_scn) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("tablet clog start ts is bigger than clog checkpoint ts, unexpected !", - K(ret), K(start_scn), K(end_scn), KPC(tablet)); - } else if (OB_FAIL(tables_handle.get_all_minor_sstables(minor_tables))) { - LOG_WARN("failed to get all minor sstables", K(ret), K(tablet_id), K(tables_handle), KPC(tablet)); - } else { - start_scn = minor_tables.empty() ? - start_scn : minor_tables.at(minor_tables.count() - 1)->get_end_scn(); - if (start_scn >= end_scn|| end_scn == ObTabletMeta::INIT_CLOG_CHECKPOINT_SCN) { - LOG_INFO("local tablet sstable is continue with memtable, no need create remote logical sstable", - K(tablet_id), K(minor_tables), K(start_scn), K(start_scn)); - } else if (OB_FAIL(create_remote_logical_sstable_(allocator, tablet_id, start_scn, end_scn, tablet, table_handle))) { - LOG_WARN("failed to create remote logical sstable", K(ret), K(tablet_id), K(start_scn), K(end_scn), KPC(tablet)); - } else if (OB_FAIL(tables_handle.add_table(table_handle))) { - LOG_WARN("failed to add table handle into tables handle", K(ret), K(table_handle), K(tables_handle)); - } - } - return ret; -} - -int ObStorageHATabletsBuilder::create_remote_logical_sstable_( - common::ObArenaAllocator &arena_allocator, - const common::ObTabletID &tablet_id, - const SCN start_scn, - const SCN end_scn, - ObTablet *tablet, - ObTableHandleV2 &table_handle) -{ - int ret = OB_SUCCESS; - ObTabletCreateSSTableParam create_sstable_param; - void *buf = nullptr; - ObSSTable *sstable = nullptr; - ObArenaAllocator allocator; - ObStorageSchema *storage_schema = nullptr; - if (!is_inited_) { - ret = OB_NOT_INIT; - LOG_WARN("storage ha tablets builder do not init", K(ret)); - } else if (!tablet_id.is_valid() || OB_ISNULL(tablet) || !start_scn.is_valid() || !end_scn.is_valid() || start_scn == end_scn) { - ret = OB_INVALID_ARGUMENT; - LOG_WARN("create remote logical sstable get invalid argument", K(ret), KPC(tablet), K(start_scn), K(end_scn)); - } else if (OB_FAIL(tablet->load_storage_schema(allocator, storage_schema))) { - LOG_WARN("fail to load storage schema failed", K(ret)); - } else if (OB_FAIL(build_remote_logical_sstable_param_(start_scn, end_scn, *storage_schema, - tablet_id, create_sstable_param))) { - LOG_WARN("failed to build remote logical sstable param", K(ret), K(tablet_id), K(start_scn), K(end_scn)); - } else if (OB_ISNULL(buf = arena_allocator.alloc(sizeof(ObSSTable)))) { - ret = OB_ALLOCATE_MEMORY_FAILED; - LOG_WARN("fail to allocate sstable", K(ret)); - } else if (FALSE_IT(sstable = new (buf) ObSSTable())) { - } else if (OB_FAIL(ObTabletCreateDeleteHelper::create_sstable(create_sstable_param, arena_allocator, *sstable))) { - LOG_WARN("failed to create sstable", K(ret), K(create_sstable_param), K(tablet_id)); - } else { - table_handle.set_sstable(sstable, &arena_allocator); - LOG_INFO("succeed to create remote logical sstable", K(tablet_id), K(table_handle), KPC(tablet)); - } - ObTabletObjLoadHelper::free(allocator, storage_schema); - return ret; -} - -//TODO(muwei.ym) put this param in tablet_table_store 4.3 -int ObStorageHATabletsBuilder::build_remote_logical_sstable_param_( - const SCN start_scn, - const SCN end_scn, - const ObStorageSchema &table_schema, - const common::ObTabletID &tablet_id, - ObTabletCreateSSTableParam ¶m) -{ - int ret = OB_SUCCESS; - if (!is_inited_) { - ret = OB_NOT_INIT; - LOG_WARN("storage ha tablets builder do not init", K(ret)); - } else if (!start_scn.is_valid() || !end_scn.is_valid() || start_scn == end_scn - || !table_schema.is_valid() || !tablet_id.is_valid()) { - ret = OB_INVALID_ARGUMENT; - LOG_WARN("build remote logical sstable param get invalid argument", K(ret), K(table_schema), K(tablet_id)); - }else if (OB_FAIL(table_schema.get_encryption_id(param.encrypt_id_))) { - LOG_WARN("fail to get_encryption_id", K(ret), K(table_schema)); - } else { - param.master_key_id_ = table_schema.get_master_key_id(); - MEMCPY(param.encrypt_key_, table_schema.get_encrypt_key_str(), table_schema.get_encrypt_key_len()); - const int64_t multi_version_col_cnt = ObMultiVersionRowkeyHelpper::get_extra_rowkey_col_cnt(); - param.table_key_.table_type_ = ObITable::TableType::REMOTE_LOGICAL_MINOR_SSTABLE; - param.table_key_.tablet_id_ = tablet_id; - param.table_key_.scn_range_.start_scn_ = start_scn; - param.table_key_.scn_range_.end_scn_ = end_scn; - param.max_merged_trans_version_ = INT64_MAX; //Set max merged trans version avoild sstable recycle; - - param.schema_version_ = table_schema.get_schema_version(); - param.create_snapshot_version_ = 0; - param.progressive_merge_round_ = table_schema.get_progressive_merge_round(); - param.progressive_merge_step_ = 0; - - param.table_mode_ = table_schema.get_table_mode_struct(); - param.index_type_ = table_schema.get_index_type(); - param.rowkey_column_cnt_ = table_schema.get_rowkey_column_num() - + ObMultiVersionRowkeyHelpper::get_extra_rowkey_col_cnt(); - param.root_block_addr_.set_none_addr(); - param.data_block_macro_meta_addr_.set_none_addr(); - param.root_row_store_type_ = ObRowStoreType::FLAT_ROW_STORE; - param.latest_row_store_type_ = ObRowStoreType::FLAT_ROW_STORE; - param.data_index_tree_height_ = 0; - param.index_blocks_cnt_ = 0; - param.data_blocks_cnt_ = 0; - param.micro_block_cnt_ = 0; - param.use_old_macro_block_count_ = 0; - param.column_cnt_ = table_schema.get_column_count() + multi_version_col_cnt; - param.data_checksum_ = 0; - param.occupy_size_ = 0; - param.ddl_scn_.set_min(); - param.filled_tx_scn_.set_min(); - param.original_size_ = 0; - param.compressor_type_ = ObCompressorType::NONE_COMPRESSOR; - } - return ret; -} - -int ObStorageHATabletsBuilder::update_local_tablets() -{ - int ret = OB_SUCCESS; - ObLS *ls = nullptr; - ObICopyTabletInfoReader *reader = nullptr; - obrpc::ObCopyTabletInfo tablet_info; - - if (!is_inited_) { - ret = OB_NOT_INIT; - LOG_WARN("storage ha tablets builder do not init", K(ret)); - } else if (OB_ISNULL(ls = param_.ls_)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("log stream should not be NULL", K(ret), KP(ls), K(param_)); - } else if (OB_FAIL(get_tablet_info_reader_(reader))) { - LOG_WARN("failed to get tablet info reader", K(ret), K(param_)); - } else { - while (OB_SUCC(ret)) { - tablet_info.reset(); - if (OB_FAIL(reader->fetch_tablet_info(tablet_info))) { - if (OB_ITER_END == ret) { - ret = OB_SUCCESS; - break; - } else { - LOG_WARN("failed to fetch tablet info", K(ret)); - } - } else if (OB_FAIL(update_local_tablet_(tablet_info, ls))) { - LOG_WARN("failed to create or update tablet", K(ret), K(tablet_info)); - } - } - } - - if (OB_NOT_NULL(reader)) { - free_tablet_info_reader_(reader); - } - return ret; -} - -int ObStorageHATabletsBuilder::update_local_tablet_( - const obrpc::ObCopyTabletInfo &tablet_info, - ObLS *ls) -{ - int ret = OB_SUCCESS; - const bool keep_old = param_.need_keep_old_tablet_; - ObTabletHandle tablet_handle; - ObTablet *tablet = nullptr; - ObBatchUpdateTableStoreParam param; - - if (!is_inited_) { - ret = OB_NOT_INIT; - LOG_WARN("storage ha tablets builder do not init", K(ret)); - } else if (!tablet_info.is_valid() || OB_ISNULL(ls)) { - ret = OB_INVALID_ARGUMENT; - LOG_WARN("create or update tablet get invalid argument", K(ret), K(tablet_info), KP(ls)); - } else if (tablet_info.tablet_id_.is_ls_inner_tablet()) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("tablet should not be inner tablet, can not update", K(ret), K(tablet_info)); - } else if (ObCopyTabletStatus::TABLET_NOT_EXIST == tablet_info.status_) { - //do nothing - } else if (OB_FAIL(ls->ha_get_tablet(tablet_info.tablet_id_, tablet_handle))) { - LOG_WARN("failed to get tablet", K(ret), K(tablet_info)); - } else if (OB_ISNULL(tablet = tablet_handle.get_obj())) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("tablet should not be NULL", K(ret), K(tablet_info), KP(tablet)); - } else if (tablet->get_tablet_meta().ha_status_.is_none()) { - ret = OB_STATE_NOT_MATCH; // TODO(zeyong) restore task mgr think about transfer scn later, rollback to OB_ERR_UNEXPECTED later. - LOG_WARN("local exist tablet data is complete, no need update local tablet", K(ret), KPC(tablet)); - } else if (tablet->get_tablet_meta().start_scn_ == tablet_info.param_.start_scn_) { - //do nothing - } else if (FALSE_IT(param.rebuild_seq_ = ls->get_rebuild_seq())) { - } else if (FALSE_IT(param.update_logical_minor_sstable_ = true)) { - } else if (FALSE_IT(param.start_scn_ = tablet_info.param_.start_scn_)) { - } else if (OB_FAIL(ls->build_ha_tablet_new_table_store(tablet_info.tablet_id_, param))) { - LOG_WARN("failed to build ha tablet new table store", K(ret), K(param), K(tablet_info)); - } else { - LOG_INFO("succeed update ha table new table store", K(tablet_info), K(tablet_info)); - } - return ret; -} - int ObStorageHATabletsBuilder::modified_tablet_info_( obrpc::ObCopyTabletInfo &tablet_info) { @@ -2812,10 +2568,6 @@ int ObStorageHATabletBuilderUtil::check_need_merge_tablet_meta_( need_merge = true; } else if (tablet->get_tablet_meta().clog_checkpoint_scn_ >= src_tablet_meta->clog_checkpoint_scn_) { need_merge = false; - } else if (OB_FAIL(check_remote_logical_sstable_exist(tablet, is_exist))) { - LOG_WARN("failed to check remote logical sstable exist", K(ret), KPC(tablet)); - } else if (!is_exist) { - need_merge = false; } else { need_merge = true; } diff --git a/src/storage/high_availability/ob_storage_ha_tablet_builder.h b/src/storage/high_availability/ob_storage_ha_tablet_builder.h index 87746bf0ba..8b17ee421a 100644 --- a/src/storage/high_availability/ob_storage_ha_tablet_builder.h +++ b/src/storage/high_availability/ob_storage_ha_tablet_builder.h @@ -83,7 +83,6 @@ public: // Otherwise, just update it to UNDEFINED. int update_pending_tablets_with_remote(); int build_tablets_sstable_info(); - int update_local_tablets(); int create_all_tablets_with_4_1_rpc( CopyTabletSimpleInfoMap &simple_info_map); private: @@ -117,7 +116,7 @@ private: int get_major_sstable_max_snapshot_( const ObSSTableArray &major_sstable_array, int64_t &max_snapshot_version); - int get_remote_logical_minor_scn_range_( + int get_minor_scn_range_( const ObSSTableArray &minor_sstable_array, ObTablet *tablet, share::ObScnRange &scn_range); @@ -140,29 +139,6 @@ private: ObTablesHandleArray &tables_handle); int remove_uncomplete_tablet_( const common::ObTabletID &tablet_id); - int create_tablet_remote_logical_sstable_( - common::ObArenaAllocator &allocator, - const common::ObTabletID &tablet_id, - ObTablesHandleArray &tables_handle); - int create_remote_logical_sstable_( - common::ObArenaAllocator &allocator, - const common::ObTabletID &tablet_id, - const share::SCN start_scn, - const share::SCN end_scn, - ObTablet *tablet, - ObTableHandleV2 &table_handle); - int build_remote_logical_sstable_param_( - const share::SCN start_scn, - const share::SCN end_scn, - const ObStorageSchema &table_schema, - const common::ObTabletID &tablet_id, - ObTabletCreateSSTableParam ¶m); - int update_local_tablet_( - const obrpc::ObCopyTabletInfo &tablet_info, - ObLS *ls); - int create_tablet_remote_logical_sstable_( - ObTablet *tablet, - ObTablesHandleArray &tables_handle); int modified_tablet_info_( obrpc::ObCopyTabletInfo &tablet_info); diff --git a/src/storage/high_availability/ob_tablet_group_restore.cpp b/src/storage/high_availability/ob_tablet_group_restore.cpp index 116253cb2d..0226e0e5ca 100644 --- a/src/storage/high_availability/ob_tablet_group_restore.cpp +++ b/src/storage/high_availability/ob_tablet_group_restore.cpp @@ -1232,6 +1232,7 @@ int ObStartTabletGroupRestoreTask::init( &ctx_->ha_table_info_mgr_, ha_tablets_builder_))) { LOG_WARN("failed to init ha tablets builder", K(ret), KPC(ctx_)); } else { + ctx_->ha_table_info_mgr_.reuse(); is_inited_ = true; LOG_INFO("succeed init start tablet group restore task", "ls id", ctx_->arg_.ls_id_, "dag_id", *ObCurTraceId::get_trace_id(), "dag_net_id", ctx_->task_id_, "tablet_id_array", ctx_->tablet_id_array_); @@ -1251,8 +1252,6 @@ int ObStartTabletGroupRestoreTask::process() LOG_WARN("start tablet group retore task do not init", K(ret)); } else if (ctx_->is_failed()) { //do nothing - } else if (OB_FAIL(try_update_local_tablets_())) { - LOG_WARN("failed to try update local tablets", K(ret), KPC(ctx_)); } else if (OB_FAIL(create_tablets_sstable_())) { LOG_WARN("failed to create tablets sstable", K(ret)); } else if (OB_FAIL(generate_tablet_restore_dag_())) { @@ -1384,31 +1383,6 @@ int ObStartTabletGroupRestoreTask::create_tablets_sstable_() return ret; } -int ObStartTabletGroupRestoreTask::try_update_local_tablets_() -{ - int ret = OB_SUCCESS; - bool is_in_retry = false; - ObStartTabletGroupRestoreDag *dag = nullptr; - - if (!is_inited_) { - ret = OB_NOT_INIT; - LOG_WARN("start tablet group restore task do not init", K(ret), KPC(ctx_)); - } else if (OB_ISNULL(dag = static_cast(this->get_dag()))) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("start tablet group restore dag should not be NULL", K(ret), KP(dag)); - } else if (OB_FAIL(dag->check_is_in_retry(is_in_retry))) { - LOG_WARN("failed to check is in retry", K(ret), KP(dag)); - } else if (!is_in_retry) { - //do nothing - } else { - ctx_->ha_table_info_mgr_.reuse(); - if (OB_FAIL(ha_tablets_builder_.update_local_tablets())) { - LOG_WARN("failed to update local tablets", K(ret), KPC(ctx_)); - } - } - return ret; -} - int ObStartTabletGroupRestoreTask::record_server_event_() { int ret = OB_SUCCESS; @@ -2540,18 +2514,17 @@ int ObTabletRestoreTask::try_update_tablet_() 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 { - //Here inner tablet copy data before clog replay, and now just create a new tablet to replace it. - //Data tablet copy data during clog replay, so the data tablet can only be updated. - if (tablet_restore_ctx_->tablet_id_.is_ls_inner_tablet()) { - if (OB_FAIL(ha_tablets_builder.create_or_update_tablets())) { - LOG_WARN("failed to create or update tablets", K(ret), KPC(tablet_restore_ctx_)); - } - } else { - if (OB_FAIL(ha_tablets_builder.update_local_tablets())) { - LOG_WARN("failed to create or update tablets", K(ret), KPC(tablet_restore_ctx_)); + if (OB_FAIL(tablet_restore_ctx_->ha_table_info_mgr_->remove_tablet_table_info(tablet_restore_ctx_->tablet_id_))) { + if (OB_HASH_NOT_EXIST == ret) { + ret = OB_SUCCESS; + } else { + LOG_WARN("failed to remove tablet info", K(ret), KPC(tablet_restore_ctx_)); } } + if (OB_FAIL(ret)) { + } else if (tablet_restore_ctx_->tablet_id_.is_ls_inner_tablet() && OB_FAIL(ha_tablets_builder.create_or_update_tablets())) { + LOG_WARN("failed to create or update inner tablet", K(ret), KPC(tablet_restore_ctx_)); } else if (OB_FAIL(ha_tablets_builder.build_tablets_sstable_info())) { LOG_WARN("failed to build tablets sstable info", K(ret), KPC(tablet_restore_ctx_)); } else if (OB_FAIL(tablet_restore_ctx_->ha_table_info_mgr_->check_tablet_table_info_exist( @@ -2612,7 +2585,6 @@ int ObTabletRestoreTask::check_src_sstable_exist_() { int ret = OB_SUCCESS; ObTablet *tablet = nullptr; - bool is_remote_logical_sstable_exist = false; bool is_major_sstable_exist = false; if (!is_inited_) { @@ -2627,19 +2599,15 @@ int ObTabletRestoreTask::check_src_sstable_exist_() if (table_key.is_major_sstable()) { is_major_sstable_exist = true; } else if (table_key.is_remote_logical_minor_sstable()) { - is_remote_logical_sstable_exist = true; + ret = OB_ERR_UNEXPECTED; + LOG_WARN("copy table key should not contain remote logical minor sstable, unexpected", + K(ret), K(copy_table_key_array_)); } } if (OB_SUCC(ret)) { if (ObTabletRestoreAction::is_restore_all(tablet_restore_ctx_->action_)) { - if (is_remote_logical_sstable_exist - || (!is_major_sstable_exist && tablet->get_tablet_meta().table_store_flag_.with_major_sstable())) { - ret = OB_SSTABLE_NOT_EXIST; - LOG_WARN("src restore sstable do not exist", K(ret), K(copy_table_key_array_), KPC(tablet_restore_ctx_)); - } - } else if (ObTabletRestoreAction::is_restore_minor(tablet_restore_ctx_->action_)) { - if (is_remote_logical_sstable_exist) { + if (!is_major_sstable_exist && tablet->get_tablet_meta().table_store_flag_.with_major_sstable()) { ret = OB_SSTABLE_NOT_EXIST; LOG_WARN("src restore sstable do not exist", K(ret), K(copy_table_key_array_), KPC(tablet_restore_ctx_)); } @@ -2769,9 +2737,10 @@ int ObTabletFinishRestoreTask::update_restore_status_() const ObSSTableArray &major_sstables = table_store_wrapper.get_member()->get_major_sstables(); for (int64_t i = 0; OB_SUCC(ret) && i < minor_sstables.count(); ++i) { const ObITable *table = minor_sstables[i]; - if (table->is_remote_logical_minor_sstable()) { + if (OB_ISNULL(table) || table->is_remote_logical_minor_sstable()) { ret = OB_ERR_UNEXPECTED; - LOG_WARN("tablet still has remote logical sstable, unexpected !!!", K(ret), KPC(table), KPC(tablet)); + LOG_WARN("table should not be NULL or tablet still has remote logical minor sstable, unexpected", + K(ret), KP(table), KPC(tablet)); } } diff --git a/src/storage/high_availability/ob_tablet_group_restore.h b/src/storage/high_availability/ob_tablet_group_restore.h index 4756db5755..f28db2a01e 100644 --- a/src/storage/high_availability/ob_tablet_group_restore.h +++ b/src/storage/high_availability/ob_tablet_group_restore.h @@ -262,7 +262,6 @@ private: int create_tablets_sstable_(); int generate_tablet_restore_dag_(); int record_server_event_(); - int try_update_local_tablets_(); private: bool is_inited_; diff --git a/src/storage/ob_storage_struct.cpp b/src/storage/ob_storage_struct.cpp index 136348d168..0b50a1bf80 100644 --- a/src/storage/ob_storage_struct.cpp +++ b/src/storage/ob_storage_struct.cpp @@ -443,7 +443,6 @@ ObBatchUpdateTableStoreParam::ObBatchUpdateTableStoreParam() errsim_point_info_(), #endif rebuild_seq_(OB_INVALID_VERSION), - update_logical_minor_sstable_(false), is_transfer_replace_(false), start_scn_(SCN::min_scn()), tablet_meta_(nullptr), @@ -456,7 +455,6 @@ void ObBatchUpdateTableStoreParam::reset() { tables_handle_.reset(); rebuild_seq_ = OB_INVALID_VERSION; - update_logical_minor_sstable_ = false; is_transfer_replace_ = false; start_scn_.set_min(); tablet_meta_ = nullptr; @@ -467,8 +465,6 @@ void ObBatchUpdateTableStoreParam::reset() bool ObBatchUpdateTableStoreParam::is_valid() const { return rebuild_seq_ > OB_INVALID_VERSION - && (!update_logical_minor_sstable_ - || (update_logical_minor_sstable_ && start_scn_ > SCN::min_scn() && OB_ISNULL(tablet_meta_))) && ObTabletRestoreStatus::is_valid(restore_status_); } @@ -483,7 +479,6 @@ int ObBatchUpdateTableStoreParam::assign( LOG_WARN("failed to assign tables handle", K(ret), K(param)); } else { rebuild_seq_ = param.rebuild_seq_; - update_logical_minor_sstable_ = param.update_logical_minor_sstable_; is_transfer_replace_ = param.is_transfer_replace_; start_scn_ = param.start_scn_; tablet_meta_ = param.tablet_meta_; diff --git a/src/storage/ob_storage_struct.h b/src/storage/ob_storage_struct.h index ae113b0086..f18cff8ab6 100644 --- a/src/storage/ob_storage_struct.h +++ b/src/storage/ob_storage_struct.h @@ -407,7 +407,7 @@ struct ObBatchUpdateTableStoreParam final int assign(const ObBatchUpdateTableStoreParam ¶m); int get_max_clog_checkpoint_scn(share::SCN &clog_checkpoint_scn) const; - TO_STRING_KV(K_(tables_handle), K_(rebuild_seq), K_(update_logical_minor_sstable), K_(is_transfer_replace), + TO_STRING_KV(K_(tables_handle), K_(rebuild_seq), K_(is_transfer_replace), K_(start_scn), KP_(tablet_meta), K_(update_ddl_sstable), K_(restore_status)); ObTablesHandleArray tables_handle_; @@ -415,7 +415,6 @@ struct ObBatchUpdateTableStoreParam final ObErrsimTransferBackfillPoint errsim_point_info_; #endif int64_t rebuild_seq_; - bool update_logical_minor_sstable_; bool is_transfer_replace_; share::SCN start_scn_; const ObMigrationTabletParam *tablet_meta_; diff --git a/src/storage/tablet/ob_tablet_table_store.cpp b/src/storage/tablet/ob_tablet_table_store.cpp index bf37543343..79b38e0822 100644 --- a/src/storage/tablet/ob_tablet_table_store.cpp +++ b/src/storage/tablet/ob_tablet_table_store.cpp @@ -587,7 +587,7 @@ int ObTabletTableStore::try_cache_local_sstable_meta( if (array_sstable->is_loaded()) { // sstable is already loaded to memory } else if (array_sstable->is_remote_logical_minor_sstable()) { - // no need to cache remote logical minor sstable. + // no need to cache remote logical minor sstable, here only for compatible. } else if (OB_FAIL(load_sstable( array_sstable->get_addr(), array_sstable->is_co_sstable(), sstable_handle))) { LOG_WARN("fail to load sstable", K(ret), KPC(array_sstable)); @@ -2300,9 +2300,6 @@ int ObTabletTableStore::combine_ha_minor_sstables_( common::ObIArray &new_minor_sstables) { int ret = OB_SUCCESS; - //TODO(muwei.ym) remove logical sstable in 4.2 RC3 - //1.ha now will not reuse minor sstable so it need add minor sstable which is from src and end_scn <= clog_checkpoint_scn - //2.old store minor sstables contains remote logical sstable and after clog_checkpoint_scn sstables. SCN max_copy_end_scn; max_copy_end_scn.set_min(); ObArray tmp_minor_sstables; @@ -2320,11 +2317,8 @@ int ObTabletTableStore::combine_ha_minor_sstables_( for (int64_t i = 0; OB_SUCC(ret) && i < old_store_minor_sstables.count(); ++i) { ObITable *table = old_store_minor_sstables.at(i); if (table->is_remote_logical_minor_sstable()) { - if (max_copy_end_scn < table->get_end_scn() && !max_copy_end_scn.is_min()) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("remote logical minor sstable end scn is bigger than max copy end scn, unexpected", - K(ret), K(max_copy_end_scn), KPC(table)); - } + ret = OB_ERR_UNEXPECTED; + LOG_WARN("old store minor sstable contains logical sstable, unexpected", K(ret), K(old_store_minor_sstables)); } else if (table->get_end_scn() <= max_copy_end_scn) { //do nothing } else if (OB_FAIL(tmp_minor_sstables.push_back(table))) { @@ -2407,7 +2401,6 @@ int ObTabletTableStore::check_old_store_minor_sstables_( common::ObIArray &old_store_minor_sstables) { int ret = OB_SUCCESS; - int64_t remote_logical_minor_sstable_count = 0; if (OB_FAIL(check_minor_tables_continue_(old_store_minor_sstables))) { LOG_WARN("failed to check minor tables continue", K(ret), K(old_store_minor_sstables)); @@ -2416,44 +2409,22 @@ int ObTabletTableStore::check_old_store_minor_sstables_( //check old store remote logical minor sstable count should be less than 1 for (int64_t i = 0; OB_SUCC(ret) && i < old_store_minor_sstables.count(); ++i) { ObITable *table = old_store_minor_sstables.at(i); - if (OB_ISNULL(table) || !table->is_multi_version_minor_sstable()) { + if (OB_ISNULL(table) || !table->is_multi_version_minor_sstable() || table->is_remote_logical_minor_sstable()) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table is null or table type is unexpected", K(ret), KPC(table)); - } else if (table->is_remote_logical_minor_sstable()) { - remote_logical_minor_sstable_count++; } } - if (OB_SUCC(ret) && remote_logical_minor_sstable_count > 1) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("old table store remote logical minor sstable count more than 1", K(ret), K(old_store_minor_sstables)); - } return ret; } int ObTabletTableStore::get_ha_mini_minor_sstables_(ObTableStoreIterator &iter) const { int ret = OB_SUCCESS; - int64_t index = 0; - for (int64_t i = 0; OB_SUCC(ret) && i < minor_tables_.count(); ++i) { ObSSTable *table = minor_tables_[i]; if (OB_ISNULL(table)) { ret = OB_ERR_UNEXPECTED; LOG_WARN("table should not be NULL", K(ret), K(minor_tables_), KP(table)); - } else if (table->is_remote_logical_minor_sstable()) { - index = i + 1; - break; - } - } - - for (int64_t i = index; OB_SUCC(ret) && i < minor_tables_.count(); ++i) { - ObSSTable *table = minor_tables_[i]; - if (OB_ISNULL(table)) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("table should not be NULL", K(ret), K(minor_tables_), KP(table)); - } else if (table->is_remote_logical_minor_sstable()) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("tablet table store has multi remote logical minor sstable, unexpected !!!", K(ret), K(minor_tables_)); } else if (OB_FAIL(iter.add_table(table))) { LOG_WARN("failed to push table into minor sstables array", K(ret), KPC(table), K(minor_tables_)); } @@ -2461,66 +2432,6 @@ int ObTabletTableStore::get_ha_mini_minor_sstables_(ObTableStoreIterator &iter) return ret; } -int ObTabletTableStore::update_ha_minor_sstables_( - common::ObArenaAllocator &allocator, - const ObTablet &tablet, - const ObBatchUpdateTableStoreParam ¶m, - const ObTabletTableStore &old_store) -{ - int ret = OB_SUCCESS; - ObArray new_minor_tables; - const ObSSTableArray &old_minor_tables = old_store.minor_tables_; - - if (param.start_scn_ >= tablet.get_clog_checkpoint_scn()) { - //no need keep local minor sstable - LOG_INFO("start scn is bigger than clog checkpoint ts, no need keep local minor sstable", K(old_store)); - } else { - int64_t index = 0; - bool has_remote_logical_sstable = false; - for (int64_t i = 0; i < old_minor_tables.count(); ++i) { - const ObITable *table = old_minor_tables[i]; - if (table->is_remote_logical_minor_sstable()) { - has_remote_logical_sstable = true; - index = i; - break; - } - } - - if (has_remote_logical_sstable) { - ObITable *table = old_minor_tables[index]; - if (!table->is_remote_logical_minor_sstable()) { - ret = OB_ERR_UNEXPECTED; - LOG_WARN("table type is unexpected", K(ret), KPC(table), K(old_store), K(param)); - } else if (param.start_scn_ >= table->get_end_scn()) { - //no need remote logical sstable - index = index + 1; - } else { - ObSSTable *sstable = static_cast(table); - share::ObScnRange new_scn_range; - share::ObScnRange original_scn_range = sstable->get_scn_range(); - new_scn_range.start_scn_ = param.start_scn_; - new_scn_range.end_scn_ = table->get_end_scn(); - sstable->set_scn_range(new_scn_range); - LOG_INFO("cut ha remote logical sstable log ts range", KPC(sstable), K(new_scn_range), K(original_scn_range)); - } - } else { - //local minor sstable contain param.start_scn, reuse local sstable - //index = 0 - } - - if (OB_SUCC(ret)) { - if (OB_FAIL(old_minor_tables.get_all_tables(new_minor_tables))) { - LOG_WARN("failed to get all minor tables", K(ret), K(old_minor_tables)); - } else if (index >= new_minor_tables.count()) { - //reuse nothing, copy from src - } else if (OB_FAIL(minor_tables_.init(allocator, new_minor_tables, index))) { - LOG_WARN("failed to init minor_tables", K(ret), K(new_minor_tables)); - } - } - } - return ret; -} - int ObTabletTableStore::build_ha_minor_tables_( common::ObArenaAllocator &allocator, const ObTablet &tablet, @@ -2529,11 +2440,7 @@ int ObTabletTableStore::build_ha_minor_tables_( const int64_t inc_base_snapshot_version) { int ret = OB_SUCCESS; - if (param.update_logical_minor_sstable_) { - if (OB_FAIL(update_ha_minor_sstables_(allocator, tablet, param, old_store))) { - LOG_WARN("failed to update ha minor sstables", K(ret), K(param), K(old_store)); - } - } else if (param.is_transfer_replace_) { + if (param.is_transfer_replace_) { if (OB_FAIL(replace_transfer_minor_sstables_(allocator, tablet, param, old_store))) { LOG_WARN("failed to replace transfer minor tables", K(ret), K(param), K(old_store)); } diff --git a/src/storage/tablet/ob_tablet_table_store.h b/src/storage/tablet/ob_tablet_table_store.h index dce6c0e8ed..66eb12154c 100644 --- a/src/storage/tablet/ob_tablet_table_store.h +++ b/src/storage/tablet/ob_tablet_table_store.h @@ -294,11 +294,6 @@ private: const ObTablet &tablet, const ObBatchUpdateTableStoreParam ¶m, const ObTabletTableStore &old_store); - int update_ha_minor_sstables_( - common::ObArenaAllocator &allocator, - const ObTablet &tablet, - const ObBatchUpdateTableStoreParam ¶m, - const ObTabletTableStore &old_store); // ddl int pull_ddl_memtables(common::ObArenaAllocator &allocator, const ObTablet &tablet); int build_ddl_sstables(