diff --git a/src/storage/ddl/ob_tablet_lob_split_task.cpp b/src/storage/ddl/ob_tablet_lob_split_task.cpp index b07309a73f..1a8d8ab056 100644 --- a/src/storage/ddl/ob_tablet_lob_split_task.cpp +++ b/src/storage/ddl/ob_tablet_lob_split_task.cpp @@ -1480,6 +1480,8 @@ int ObTabletLobWriteDataTask::create_sstables( int64_t last_minor_idx = -1; ObFixedArray batch_sstables_handle; batch_sstables_handle.set_allocator(&allocator_); + const compaction::ObMergeType merge_type = share::ObSplitSSTableType::SPLIT_MINOR == split_sstable_type ? + compaction::ObMergeType::MINOR_MERGE : compaction::ObMergeType::MAJOR_MERGE; if (OB_FAIL(batch_sstables_handle.prepare_allocate(ctx_->new_lob_tablet_ids_.count()))) { LOG_WARN("init failed", K(ret), K(ctx_->new_lob_tablet_ids_)); } @@ -1553,10 +1555,36 @@ int ObTabletLobWriteDataTask::create_sstables( ctx_->lob_meta_tablet_handle_, ctx_->new_lob_tablet_ids_.at(i), batch_sstables_handle.at(i), - split_sstable_type, + merge_type, false/*can_reuse_macro_block*/))) { LOG_WARN("update table store with batch tables failed", K(ret), K(batch_sstables_handle.at(i)), K(split_sstable_type)); } + if (OB_SUCC(ret) && !is_major_merge_type(merge_type)) { + // build lost mds sstable into tablet. + ObTableHandleV2 mds_table_handle; + ObTablesHandleArray mds_sstables_handle; + common::ObArenaAllocator build_mds_arena("SplitBuildMds", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID()); + if (OB_FAIL(ObTabletSplitUtil::build_lost_medium_mds_sstable( + build_mds_arena, + ctx_->ls_handle_, + ctx_->lob_meta_tablet_handle_, + ctx_->new_lob_tablet_ids_.at(i), + mds_table_handle))) { + LOG_WARN("build lost medium mds sstable failed", K(ret), KPC(param_)); + } else if (OB_UNLIKELY(!mds_table_handle.is_valid())) { + LOG_INFO("no need to fill medium mds sstable", K(ret), KPC(param_)); + } else if (OB_FAIL(mds_sstables_handle.add_table(mds_table_handle))) { + LOG_WARN("add table failed", K(ret)); + } else if (OB_FAIL(ObTabletSplitMergeTask::update_table_store_with_batch_tables( + ctx_->ls_handle_, + ctx_->lob_meta_tablet_handle_, + ctx_->new_lob_tablet_ids_.at(i), + mds_sstables_handle, + compaction::ObMergeType::MDS_MINI_MERGE, + false/*can_reuse_macro_block*/))) { + LOG_WARN("update table store with batch tables failed", K(ret), K(mds_sstables_handle)); + } + } } return ret; } diff --git a/src/storage/ddl/ob_tablet_split_task.cpp b/src/storage/ddl/ob_tablet_split_task.cpp index 752923bb4c..94a2c34a08 100644 --- a/src/storage/ddl/ob_tablet_split_task.cpp +++ b/src/storage/ddl/ob_tablet_split_task.cpp @@ -15,12 +15,16 @@ #include "logservice/ob_log_service.h" #include "share/ob_ddl_common.h" #include "share/scn.h" +#include "storage/compaction/ob_tablet_merge_ctx.h" #include "storage/ob_i_store.h" #include "storage/ob_partition_range_spliter.h" #include "storage/ddl/ob_ddl_merge_task.h" #include "storage/ddl/ob_ddl_clog.h" +#include "storage/tablet/ob_mds_scan_param_helper.h" #include "storage/tablet/ob_tablet_create_sstable_param.h" #include "storage/tablet/ob_tablet_create_delete_helper.h" +#include "storage/tablet/ob_tablet_mds_table_mini_merger.h" +#include "storage/tablet/ob_tablet_medium_info_reader.h" #include "storage/tablet/ob_tablet_split_mds_helper.h" #include "share/scheduler/ob_dag_warning_history_mgr.h" @@ -1330,6 +1334,8 @@ int ObTabletSplitMergeTask::create_sstable( } else { ObArenaAllocator tmp_arena("PartSplitSchema"); const int64_t multi_version_start = context_->tablet_handle_.get_obj()->get_multi_version_start(); + const compaction::ObMergeType merge_type = share::ObSplitSSTableType::SPLIT_MINOR == split_sstable_type ? + compaction::ObMergeType::MINOR_MERGE : compaction::ObMergeType::MAJOR_MERGE; for (int64_t i = 0; OB_SUCC(ret) && i < param_->dest_tablets_id_.count(); i++) { bool is_major_exist = false; const ObTabletID &dest_tablet_id = param_->dest_tablets_id_.at(i); @@ -1368,7 +1374,7 @@ int ObTabletSplitMergeTask::create_sstable( } } // fill empty minor sstable if scn not continous - if (OB_SUCC(ret) && j == src_table_cnt - 1 && share::ObSplitSSTableType::SPLIT_MINOR == split_sstable_type) { + if (OB_SUCC(ret) && j == src_table_cnt - 1 && !is_major_merge_type(merge_type)) { bool need_fill_empty_sstable = false; SCN end_scn; if (OB_FAIL(check_need_fill_empty_sstable(context_->ls_handle_, src_sstable->is_minor_sstable(), src_sstable->get_key(), dest_tablet_id, need_fill_empty_sstable, end_scn))) { @@ -1396,11 +1402,38 @@ int ObTabletSplitMergeTask::create_sstable( context_->tablet_handle_, dest_tablet_id, batch_sstables_handle, - split_sstable_type, + merge_type, param_->can_reuse_macro_block_))) { LOG_WARN("update table store with batch tables failed", K(ret), K(batch_sstables_handle), K(split_sstable_type)); } } + if (OB_SUCC(ret) && !is_major_merge_type(merge_type)) { + // build lost mds sstable after minor merge. + ObTableHandleV2 table_handle; + batch_sstables_handle.reset(); + common::ObArenaAllocator build_mds_arena("SplitBuildMds", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID()); + if (OB_FAIL(ObTabletSplitUtil::build_lost_medium_mds_sstable( + build_mds_arena, + context_->ls_handle_, + context_->tablet_handle_, + dest_tablet_id, + table_handle))) { + LOG_WARN("build lost medium mds sstable failed", K(ret), KPC(param_)); + } else if (OB_UNLIKELY(!table_handle.is_valid())) { + LOG_INFO("no need to fill medium mds sstable", K(ret), + "src_tablet_id", param_->source_tablet_id_, K(dest_tablet_id)); + } else if (OB_FAIL(batch_sstables_handle.add_table(table_handle))) { + LOG_WARN("add table failed", K(ret)); + } else if (OB_FAIL(ObTabletSplitMergeTask::update_table_store_with_batch_tables( + context_->ls_handle_, + context_->tablet_handle_, + dest_tablet_id, + batch_sstables_handle, + compaction::ObMergeType::MDS_MINI_MERGE, + param_->can_reuse_macro_block_))) { + LOG_WARN("update table store with batch tables failed", K(ret), K(batch_sstables_handle)); + } + } } } } @@ -1578,7 +1611,7 @@ int ObTabletSplitMergeTask::update_table_store_with_batch_tables( const ObTabletHandle &src_tablet_handle, const ObTabletID &dst_tablet_id, const ObTablesHandleArray &tables_handle, - const share::ObSplitSSTableType &split_sstable_type, + const compaction::ObMergeType &merge_type, const bool can_reuse_macro_block) { int ret = OB_SUCCESS; @@ -1589,13 +1622,13 @@ int ObTabletSplitMergeTask::update_table_store_with_batch_tables( || !src_tablet_handle.is_valid() || !dst_tablet_id.is_valid() || tables_handle.empty() - || (share::ObSplitSSTableType::SPLIT_MAJOR != split_sstable_type - && share::ObSplitSSTableType::SPLIT_MINOR != split_sstable_type))) { + || !is_valid_merge_type(merge_type))) { ret = OB_INVALID_ARGUMENT; - LOG_WARN("invalid arg", K(ret), K(ls_handle), K(src_tablet_handle), K(dst_tablet_id), K(tables_handle), K(split_sstable_type)); + LOG_WARN("invalid arg", K(ret), K(ls_handle), K(src_tablet_handle), + K(dst_tablet_id), K(tables_handle), K(merge_type)); } else if (OB_FAIL(tables_handle.get_tables(batch_tables))) { LOG_WARN("get batch sstables failed", K(ret)); - } else if (share::ObSplitSSTableType::SPLIT_MINOR == split_sstable_type) { + } else if (!is_major_merge_type(merge_type)) { // minor merge or mds mini merge. if (OB_FAIL(param.tables_handle_.assign(tables_handle))) { LOG_WARN("assign failed", K(ret), K(batch_tables)); } @@ -1628,7 +1661,7 @@ int ObTabletSplitMergeTask::update_table_store_with_batch_tables( } } - if (OB_SUCC(ret) && can_reuse_macro_block && share::ObSplitSSTableType::SPLIT_MAJOR == split_sstable_type) { + if (OB_SUCC(ret) && can_reuse_macro_block && is_major_merge_type(merge_type)) { // iterate all major and minors, to determine the dest restore status. if (OB_FAIL(check_and_determine_restore_status(ls_handle, dst_tablet_id, param.tables_handle_, param.restore_status_))) { LOG_WARN("check and determine restore status failed", K(ret), K(dst_tablet_id)); @@ -1638,7 +1671,7 @@ int ObTabletSplitMergeTask::update_table_store_with_batch_tables( if (OB_SUCC(ret)) { param.tablet_split_param_.snapshot_version_ = src_tablet_handle.get_obj()->get_tablet_meta().snapshot_version_; param.tablet_split_param_.multi_version_start_ = src_tablet_handle.get_obj()->get_multi_version_start(); - param.tablet_split_param_.update_with_major_tables_ = share::ObSplitSSTableType::SPLIT_MAJOR == split_sstable_type; + param.tablet_split_param_.merge_type_ = merge_type; param.rebuild_seq_ = ls_handle.get_ls()->get_rebuild_seq(); if (OB_FAIL(ls_handle.get_ls()->build_tablet_with_batch_tables(dst_tablet_id, param))) { LOG_WARN("failed to update tablet table store", K(ret), K(dst_tablet_id), K(param)); @@ -2713,5 +2746,152 @@ int ObTabletSplitUtil::check_tablet_restore_status( return ret; } +int ObTabletSplitUtil::build_lost_medium_mds_sstable( + common::ObArenaAllocator &allocator, + const ObLSHandle &ls_handle, + const ObTabletHandle &source_tablet_handle, + const ObTabletID &dest_tablet_id, + ObTableHandleV2 &medium_mds_table_handle) +{ + int ret = OB_SUCCESS; + medium_mds_table_handle.reset(); + ObTabletHandle dest_tablet_handle; + if (OB_UNLIKELY(!ls_handle.is_valid() || !source_tablet_handle.is_valid() || !dest_tablet_id.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arg", K(ret), K(ls_handle), K(source_tablet_handle), K(dest_tablet_id)); + } else if (!MTL_TENANT_ROLE_CACHE_IS_RESTORE()) { + LOG_INFO("not restore tenant, no medium info lost", "tenant_id", MTL_ID(), + "source_tablet_id", source_tablet_handle.get_obj()->get_tablet_id(), K(dest_tablet_id)); + } else if (OB_FAIL(ObDDLUtil::ddl_get_tablet(ls_handle, + dest_tablet_id, dest_tablet_handle, ObMDSGetTabletMode::READ_ALL_COMMITED))) { + LOG_WARN("get tablet failed", K(ret), K(dest_tablet_id)); + } else { + const share::ObLSID &ls_id = ls_handle.get_ls()->get_ls_id(); + const ObTabletID &source_tablet_id = source_tablet_handle.get_obj()->get_tablet_id(); + HEAP_VARS_3 ((compaction::ObTabletMergeDagParam, param), + (compaction::ObTabletMergeCtx, tablet_merge_ctx, param, allocator), + (ObTabletMediumInfoReader, medium_info_reader)) { + HEAP_VARS_3 ((ObTableScanParam, scan_param), + (ObTabletDumpMediumMds2MiniOperator, op), + (ObMdsTableMiniMerger, mds_mini_merger)) { + if (OB_FAIL(check_and_build_mds_sstable_merge_ctx(ls_handle, dest_tablet_handle, tablet_merge_ctx))) { + LOG_WARN("prepare medium mds merge ctx failed", K(ret), K(ls_handle), K(dest_tablet_id)); + } else if (tablet_merge_ctx.static_param_.scn_range_.end_scn_.is_base_scn()) { // = 1 + LOG_INFO("no need to build lost mds sstable again", K(ls_id), K(source_tablet_id), K(dest_tablet_id)); + } else if (OB_FAIL(mds_mini_merger.init(tablet_merge_ctx, op))) { + LOG_WARN("fail to init mds mini merger", K(ret), K(tablet_merge_ctx), K(ls_id), K(dest_tablet_id)); + } else if (OB_FAIL(ObMdsScanParamHelper::build_medium_info_scan_param( + allocator, + ls_id, + source_tablet_id, + scan_param))) { + LOG_WARN("fail to build scan param", K(ret), K(ls_id), K(source_tablet_id)); + } else if (OB_FAIL(medium_info_reader.init(*source_tablet_handle.get_obj(), scan_param))) { + LOG_WARN("failed to init medium info reader", K(ret)); + } else { + bool has_medium_info = false; + mds::MdsDumpKV *kv = nullptr; + common::ObArenaAllocator iter_arena("SplitIterMedium", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID()); + while (OB_SUCC(ret)) { + iter_arena.reuse(); + if (OB_FAIL(medium_info_reader.get_next_mds_kv(iter_arena, kv))) { + if (OB_ITER_END != ret) { + LOG_WARN("iter medium mds failed", K(ret), K(ls_id), K(source_tablet_id)); + } else { + ret = OB_SUCCESS; + break; + } + } else if (OB_FAIL(op(*kv))) { + LOG_WARN("write medium row failed", K(ret)); + } else { + kv->mds::MdsDumpKV::~MdsDumpKV(); + iter_arena.free(kv); + kv = nullptr; + has_medium_info = true; + } + } + if (OB_SUCC(ret)) { + if (!has_medium_info) { + LOG_INFO("no need to build lost mds sstable", K(ls_id), K(source_tablet_id), K(dest_tablet_id)); + } else if (OB_FAIL(op.finish())) { + LOG_WARN("finish failed", K(ret)); + } else if (OB_FAIL(mds_mini_merger.generate_mds_mini_sstable(allocator, medium_mds_table_handle))) { + LOG_WARN("fail to generate mds mini sstable with mini merger", K(ret), K(mds_mini_merger)); + } + } + } + } + } + } + return ret; +} + +int ObTabletSplitUtil::check_and_build_mds_sstable_merge_ctx( + const ObLSHandle &ls_handle, + const ObTabletHandle &dest_tablet_handle, + compaction::ObTabletMergeCtx &tablet_merge_ctx) +{ + int ret = OB_SUCCESS; + ObLSService *ls_service = nullptr; + share::SCN end_scn; + if (OB_UNLIKELY(!ls_handle.is_valid() || !dest_tablet_handle.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arg", K(ret), K(ls_handle), K(dest_tablet_handle)); + } else if (OB_FAIL(check_and_determine_mds_end_scn(dest_tablet_handle, end_scn))) { + LOG_WARN("get mds sstable start scn failed", K(ret), K(dest_tablet_handle)); + } else { + compaction::ObStaticMergeParam &static_param = tablet_merge_ctx.static_param_; + static_param.ls_handle_ = ls_handle; + static_param.dag_param_.ls_id_ = ls_handle.get_ls()->get_ls_id(); + static_param.dag_param_.merge_type_ = compaction::ObMergeType::MDS_MINI_MERGE; + static_param.dag_param_.tablet_id_ = dest_tablet_handle.get_obj()->get_tablet_id(); + static_param.pre_warm_param_.type_ = ObPreWarmerType::MEM_PRE_WARM; + tablet_merge_ctx.tablet_handle_ = dest_tablet_handle; + static_param.scn_range_.start_scn_ = SCN::base_scn(); // 1 + static_param.scn_range_.end_scn_ = end_scn; + static_param.version_range_.snapshot_version_ = end_scn.get_val_for_tx(); + static_param.version_range_.multi_version_start_ = dest_tablet_handle.get_obj()->get_multi_version_start(); + static_param.merge_scn_ = end_scn; + static_param.create_snapshot_version_ = 0; + static_param.need_parallel_minor_merge_ = false; + static_param.tablet_transfer_seq_ = dest_tablet_handle.get_obj()->get_transfer_seq(); + tablet_merge_ctx.static_desc_.tablet_transfer_seq_ = dest_tablet_handle.get_obj()->get_transfer_seq(); + + if (OB_FAIL(tablet_merge_ctx.init_tablet_merge_info())) { + LOG_WARN("failed to init tablet merge info", K(ret), K(ls_handle), K(dest_tablet_handle), K(tablet_merge_ctx)); + } + } + return ret; +} + +int ObTabletSplitUtil::check_and_determine_mds_end_scn( + const ObTabletHandle &dest_tablet_handle, + share::SCN &end_scn) +{ + int ret = OB_SUCCESS; + ObTablet *tablet = nullptr; + ObITable *first_mds_sstable = nullptr; + ObTableStoreIterator table_store_iterator; + ObTabletMemberWrapper table_store_wrapper; + if (OB_UNLIKELY(!dest_tablet_handle.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("invalid arg", K(ret), K(dest_tablet_handle)); + } else if (OB_ISNULL(tablet = dest_tablet_handle.get_obj())) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("tablet should not be nullptr", K(ret), K(dest_tablet_handle)); + } else if (OB_FAIL(tablet->get_all_sstables(table_store_iterator))) { + LOG_WARN("get all sstables failed", K(ret)); + } else if (OB_FAIL(tablet->fetch_table_store(table_store_wrapper))) { + LOG_WARN("fetch table store failed", K(ret), KPC(tablet)); + } else if (OB_ISNULL(first_mds_sstable = + table_store_wrapper.get_member()->get_mds_sstables().get_boundary_table(false/*first*/))) { + end_scn = tablet->get_mds_checkpoint_scn(); + } else { + end_scn = first_mds_sstable->get_start_scn(); + } + + return ret; +} + } //end namespace stroage } //end namespace oceanbase diff --git a/src/storage/ddl/ob_tablet_split_task.h b/src/storage/ddl/ob_tablet_split_task.h index bab6e4b8f2..c2de7a0033 100644 --- a/src/storage/ddl/ob_tablet_split_task.h +++ b/src/storage/ddl/ob_tablet_split_task.h @@ -258,7 +258,7 @@ public: const ObTabletHandle &src_tablet_handle, const ObTabletID &dst_tablet_id, const ObTablesHandleArray &tables_handle, - const share::ObSplitSSTableType &split_sstable_type, + const compaction::ObMergeType &merge_type, const bool can_reuse_macro_block); private: int create_sstable( @@ -445,6 +445,21 @@ public: const ObLSHandle &ls_handle, const ObTabletHandle &source_tablet_handle, bool &is_tablet_status_need_to_split); + static int build_lost_medium_mds_sstable( + common::ObArenaAllocator &allocator, + const ObLSHandle &ls_handle, + const ObTabletHandle &source_tablet_handle, + const ObTabletID &dest_tablet_id, + ObTableHandleV2 &medium_mds_table_handle); +private: + static int check_and_build_mds_sstable_merge_ctx( + const ObLSHandle &ls_handle, + const ObTabletHandle &dest_tablet_handle, + compaction::ObTabletMergeCtx &tablet_merge_ctx); + static int check_and_determine_mds_end_scn( + const ObTabletHandle &dest_tablet_handle, + share::SCN &end_scn); + }; } // end namespace storage diff --git a/src/storage/multi_data_source/adapter_define/mds_dump_kv_wrapper.cpp b/src/storage/multi_data_source/adapter_define/mds_dump_kv_wrapper.cpp index 69e0acd4b1..6472346742 100644 --- a/src/storage/multi_data_source/adapter_define/mds_dump_kv_wrapper.cpp +++ b/src/storage/multi_data_source/adapter_define/mds_dump_kv_wrapper.cpp @@ -127,8 +127,8 @@ int MdsDumpKVStorageAdapter::convert_to_mds_row( ret = OB_ALLOCATE_MEMORY_FAILED; MDS_LOG(WARN, "failed to alloc buf for seriaize DumpKVStorageMetaInfo", K(ret), K(meta_info_size), K(allocator.total()), K(allocator.used())); - } else if (meta_info_.serialize(buf, meta_info_size, pos)) { - MDS_LOG(WARN, "failed to serialize DumpKVStorageMetaInfo", K(meta_info_), K(meta_info_size), K(pos)); + } else if (OB_FAIL(meta_info_.serialize(buf, meta_info_size, pos))) { + MDS_LOG(WARN, "failed to serialize DumpKVStorageMetaInfo", K(ret), K(meta_info_), K(meta_info_size), K(pos)); } else if (OB_UNLIKELY(pos != meta_info_size)) { ret = OB_ERR_UNEXPECTED; MDS_LOG(WARN, "unexpected pos with serialize size", K(ret), K(pos), K(meta_info_size)); diff --git a/src/storage/ob_storage_struct.cpp b/src/storage/ob_storage_struct.cpp index 9460737f46..f9111a418e 100644 --- a/src/storage/ob_storage_struct.cpp +++ b/src/storage/ob_storage_struct.cpp @@ -663,7 +663,7 @@ int ObBatchUpdateTableStoreParam::get_max_clog_checkpoint_scn(SCN &clog_checkpoi ObSplitTableStoreParam::ObSplitTableStoreParam() : snapshot_version_(-1), multi_version_start_(-1), - update_with_major_tables_(false) + merge_type_(INVALID_MERGE_TYPE) { } @@ -675,14 +675,15 @@ ObSplitTableStoreParam::~ObSplitTableStoreParam() bool ObSplitTableStoreParam::is_valid() const { return snapshot_version_ > -1 - && multi_version_start_ >= 0; + && multi_version_start_ >= 0 + && is_valid_merge_type(merge_type_); } void ObSplitTableStoreParam::reset() { snapshot_version_ = -1; multi_version_start_ = -1; - update_with_major_tables_ = false; + merge_type_ = INVALID_MERGE_TYPE; } ObPartitionReadableInfo::ObPartitionReadableInfo() diff --git a/src/storage/ob_storage_struct.h b/src/storage/ob_storage_struct.h index 0755799c5a..6f2eee2323 100644 --- a/src/storage/ob_storage_struct.h +++ b/src/storage/ob_storage_struct.h @@ -481,12 +481,12 @@ public: ~ObSplitTableStoreParam(); bool is_valid() const; void reset(); - TO_STRING_KV(K_(snapshot_version), K_(multi_version_start), K_(update_with_major_tables)); + TO_STRING_KV(K_(snapshot_version), K_(multi_version_start), K_(merge_type)); public: int64_t snapshot_version_; int64_t multi_version_start_; - bool update_with_major_tables_; + compaction::ObMergeType merge_type_; }; struct ObBatchUpdateTableStoreParam final diff --git a/src/storage/tablet/ob_tablet.cpp b/src/storage/tablet/ob_tablet.cpp index 2b7790e091..7bc955fb29 100644 --- a/src/storage/tablet/ob_tablet.cpp +++ b/src/storage/tablet/ob_tablet.cpp @@ -906,7 +906,7 @@ int ObTablet::update_restore_status_for_split_(const ObBatchUpdateTableStorePara if (!param.tablet_split_param_.is_valid()) { ret = OB_INVALID_ARGUMENT; LOG_WARN("invalid args", K(ret), K(param)); - } else if (!param.tablet_split_param_.update_with_major_tables_ || ObTabletRestoreStatus::is_full(param.restore_status_)) { + } else if (!is_major_merge_type(param.tablet_split_param_.merge_type_) || ObTabletRestoreStatus::is_full(param.restore_status_)) { // update restore status only when updating major sstables and inputing remote restore status. } else if (!ObTabletRestoreStatus::is_remote(param.restore_status_)) { ret = OB_INVALID_ARGUMENT; @@ -979,7 +979,7 @@ int ObTablet::init_for_sstable_replace( } else if (OB_FAIL(ObStorageSchemaUtil::update_tablet_storage_schema( tablet_meta_.tablet_id_, *allocator_, *old_storage_schema, *storage_schema, storage_schema_addr_.ptr_))) { LOG_WARN("failed to choose and save storage schema", K(ret), K(old_tablet), K(param)); - } else if (is_tablet_split && OB_FAIL(try_update_table_store_flag(param.tablet_split_param_.update_with_major_tables_))) { + } else if (is_tablet_split && OB_FAIL(try_update_table_store_flag(is_major_merge_type(param.tablet_split_param_.merge_type_)))) { LOG_WARN("failed to update table store flag", K(ret), K(param), K(table_store_addr_)); } else if (is_tablet_split && OB_FAIL(update_restore_status_for_split_(param))) { LOG_WARN("update restore status for tablet split failed", K(ret), K(param), KPC(this)); diff --git a/src/storage/tablet/ob_tablet_mds_table_mini_merger.cpp b/src/storage/tablet/ob_tablet_mds_table_mini_merger.cpp index a7e4773b30..ffc0974f6a 100644 --- a/src/storage/tablet/ob_tablet_mds_table_mini_merger.cpp +++ b/src/storage/tablet/ob_tablet_mds_table_mini_merger.cpp @@ -326,6 +326,31 @@ int ObCrossLSMdsMiniMergeOperator::operator()(const mds::MdsDumpKV &kv) return ret; } +int ObTabletDumpMediumMds2MiniOperator::operator()(const mds::MdsDumpKV &kv) +{ + int ret = OB_SUCCESS; + if (OB_UNLIKELY(!is_inited_)) { + ret = OB_NOT_INIT; + LOG_WARN("not inited", K(ret), K_(is_inited)); + } else if (OB_UNLIKELY(!kv.is_valid())) { + ret = OB_INVALID_ARGUMENT; + LOG_WARN("dump kv is invalid", K(ret), K(kv)); + } else { + cur_row_.reuse(); + cur_allocator_.reuse(); + mds::MdsDumpKVStorageAdapter adapter(kv); + if (OB_FAIL(adapter.convert_to_mds_row(cur_allocator_, cur_row_))) { + LOG_WARN("fail to convert MdsDumpKVStorageAdapter to row", K(ret), K(adapter), K(cur_row_)); + } else if (OB_FAIL(row_store_.put_row_into_queue(cur_row_))) { + LOG_WARN("fail to put row into queue", K(ret)); + } else { + LOG_INFO("mds op succeed to add medium mds row", K(ret), K(adapter), K(cur_row_)); + } + } + return ret; +} + + /* ------------------------------------------ObMdsTableMiniMerger----------------------------------- */ diff --git a/src/storage/tablet/ob_tablet_mds_table_mini_merger.h b/src/storage/tablet/ob_tablet_mds_table_mini_merger.h index e257453aef..7874b81b2b 100644 --- a/src/storage/tablet/ob_tablet_mds_table_mini_merger.h +++ b/src/storage/tablet/ob_tablet_mds_table_mini_merger.h @@ -99,6 +99,18 @@ private: share::SCN scan_end_scn_; }; +// to query all medium mds info, and dump them to minor sstable. +class ObTabletDumpMediumMds2MiniOperator : public ObMdsMiniMergeOperator +{ +public: + ObTabletDumpMediumMds2MiniOperator() = default; + virtual ~ObTabletDumpMediumMds2MiniOperator() = default; + virtual int operator()(const mds::MdsDumpKV &kv) override; +protected: + virtual bool for_flush() override { return true; } +}; + + class ObMdsTableMiniMerger { public: diff --git a/src/storage/tablet/ob_tablet_medium_info_reader.cpp b/src/storage/tablet/ob_tablet_medium_info_reader.cpp index da611e6da2..370a7452a2 100644 --- a/src/storage/tablet/ob_tablet_medium_info_reader.cpp +++ b/src/storage/tablet/ob_tablet_medium_info_reader.cpp @@ -183,5 +183,22 @@ int ObTabletMediumInfoReader::get_min_medium_snapshot( return ret; } + +int ObTabletMediumInfoReader::get_next_mds_kv( + common::ObIAllocator &allocator, + mds::MdsDumpKV *&kv) +{ + int ret = OB_SUCCESS; + kv = nullptr; + if (OB_FAIL(iter_.get_next_mds_kv(allocator, kv))) { + if (OB_ITER_END == ret) { + LOG_DEBUG("iter end", K(ret)); + } else { + LOG_WARN("fail to get next mds kv", K(ret)); + } + } + return ret; +} + } // namespace storage } // namespace oceanbase diff --git a/src/storage/tablet/ob_tablet_medium_info_reader.h b/src/storage/tablet/ob_tablet_medium_info_reader.h index 2e57405a47..b361d77ed6 100644 --- a/src/storage/tablet/ob_tablet_medium_info_reader.h +++ b/src/storage/tablet/ob_tablet_medium_info_reader.h @@ -43,6 +43,9 @@ public: int get_min_medium_snapshot( const int64_t last_major_snapshot_version, int64_t &min_medium_snapshot); + int get_next_mds_kv( + common::ObIAllocator &allocator, + mds::MdsDumpKV *&kv); private: bool is_inited_; common::ObArenaAllocator allocator_; diff --git a/src/storage/tablet/ob_tablet_table_store.cpp b/src/storage/tablet/ob_tablet_table_store.cpp index 2554d780d5..76ad76254c 100644 --- a/src/storage/tablet/ob_tablet_table_store.cpp +++ b/src/storage/tablet/ob_tablet_table_store.cpp @@ -3242,6 +3242,7 @@ int ObTabletTableStore::build_split_new_table_store_( int64_t inc_base_snapshot_version = -1; ObSEArray batch_tables; const ObTabletHAStatus &ha_status = tablet.get_tablet_meta().ha_status_; + ObSSTable *new_mds_sstable = nullptr; if (OB_FAIL(param.tables_handle_.get_tables(batch_tables))) { LOG_WARN("get tables failed", K(ret), K(param)); } else if (OB_FAIL(inner_build_major_tables_(allocator, old_store, batch_tables, @@ -3261,7 +3262,12 @@ int ObTabletTableStore::build_split_new_table_store_( LOG_WARN("failed to pull memtable from memtable_mgr", K(ret)); } else if (OB_FAIL(pull_ddl_memtables(allocator, tablet))) { LOG_WARN("pull_ddl_memtables failed", K(ret)); - } else if (OB_FAIL(build_minor_tables(allocator, nullptr/*new_sstable*/, old_store, false/*need_check_sstable*/, -1/*inc_base_snapshot_version*/, ha_status, unused_param, true/*is_mds*/))) { + } else if (is_mds_merge(param.tablet_split_param_.merge_type_) && 1 != batch_tables.count()) { + ret = OB_ERR_UNEXPECTED; + LOG_WARN("unexpected null new mds sstable", K(ret), K(param)); + } else if (is_mds_merge(param.tablet_split_param_.merge_type_) + && OB_FALSE_IT(new_mds_sstable = static_cast(batch_tables.at(0)))) { + } else if (OB_FAIL(build_minor_tables(allocator, new_mds_sstable, old_store, false/*need_check_sstable*/, -1/*inc_base_snapshot_version*/, ha_status, unused_param, true/*is_mds*/))) { LOG_WARN("failed to build mds sstables", K(ret)); } else { is_inited_ = true;