[CP] fix medium info of the split dest tablet lost.

This commit is contained in:
obdev 2024-10-31 14:14:46 +00:00 committed by ob-robot
parent 4848d5d5d1
commit b85b6becdb
12 changed files with 308 additions and 21 deletions

View File

@ -1480,6 +1480,8 @@ int ObTabletLobWriteDataTask::create_sstables(
int64_t last_minor_idx = -1;
ObFixedArray<ObTablesHandleArray, common::ObIAllocator> 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;
}

View File

@ -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<ObTabletTableStore> 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

View File

@ -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

View File

@ -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));

View File

@ -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()

View File

@ -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

View File

@ -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));

View File

@ -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-----------------------------------
*/

View File

@ -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:

View File

@ -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

View File

@ -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_;

View File

@ -3242,6 +3242,7 @@ int ObTabletTableStore::build_split_new_table_store_(
int64_t inc_base_snapshot_version = -1;
ObSEArray<ObITable *, OB_DEFAULT_SE_ARRAY_COUNT> 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<ObSSTable *>(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;