optimize mark_and_sweep

This commit is contained in:
obdev
2023-08-17 11:24:16 +00:00
committed by ob-robot
parent 03d16ad10a
commit 90ae6c4e43
5 changed files with 142 additions and 159 deletions

View File

@ -127,15 +127,6 @@ ObBlockManager::ObBlockManager()
super_block_(),
super_block_buf_holder_(),
default_block_size_(0),
mark_cost_time_(0),
sweep_cost_time_(0),
reserved_count_(0),
hold_count_(0),
pending_free_count_(0),
disk_block_count_(0),
start_time_(0),
last_end_time_(0),
hold_info_(),
marker_status_(),
marker_lock_(),
is_mark_sweep_enabled_(false),
@ -146,9 +137,10 @@ ObBlockManager::ObBlockManager()
bad_block_lock_(),
io_device_(NULL),
blk_seq_generator_(),
alloc_num_(0),
resize_file_lock_(),
is_inited_(false),
is_started_(false),
resize_file_lock_()
is_started_(false)
{
}
@ -177,20 +169,11 @@ int ObBlockManager::init(
} else if (OB_FAIL(super_block_buf_holder_.init(ObServerSuperBlockHeader::OB_MAX_SUPER_BLOCK_SIZE))) {
LOG_WARN("fail to init super block buffer holder, ", K(ret));
} else {
MEMSET(used_macro_cnt_, 0, sizeof(used_macro_cnt_));
mark_cost_time_ = 0;
sweep_cost_time_= 0;
reserved_count_ = 0;
hold_count_ = 0;
pending_free_count_ = 0;
disk_block_count_ = 0;
start_time_ = 0;
last_end_time_ = 0;
hold_info_.reset();
io_device_ = io_device;
super_block_fd_.first_id_ = 0; // super block default fd
super_block_fd_.second_id_ = 0; // super block default fd
default_block_size_ = block_size;
ATOMIC_STORE(&alloc_num_, 0);
is_inited_ = true;
}
@ -288,6 +271,7 @@ void ObBlockManager::destroy()
is_mark_sweep_enabled_ = false;
marker_status_.reset();
blk_seq_generator_.reset();
ATOMIC_STORE(&alloc_num_, 0);
is_inited_ = false;
}
@ -327,6 +311,7 @@ int ObBlockManager::alloc_block(ObMacroBlockHandle &macro_handle)
if (OB_FAIL(macro_handle.set_macro_block_id(macro_id))) {
LOG_ERROR("Failed to set macro block id", K(ret), K(macro_id));
} else {
ATOMIC_AAF(&alloc_num_, 1);
FLOG_INFO("successfully alloc block", K(macro_id));
}
}
@ -708,6 +693,9 @@ int ObBlockManager::inc_ref(const MacroBlockId &macro_id)
} else {
LOG_ERROR("get block_info fail", K(ret), K(macro_id));
}
} else if (OB_UNLIKELY(0 == block_info.ref_cnt_ && is_mark_sweep_enabled())) {
ret = OB_ERR_UNEXPECTED;
LOG_ERROR("ref cnt shouldn't be 0", K(ret), K(macro_id), K(block_info));
}
if (OB_SUCC(ret)) {
@ -789,34 +777,34 @@ int ObBlockManager::get_marker_status(ObMacroBlockMarkerStatus &status)
} else {
SpinRLockGuard guard(marker_lock_);
status = marker_status_;
status.start_time_ = ATOMIC_LOAD(&start_time_);
}
return ret ;
}
void ObBlockManager::update_marker_status(const bool mark_finished)
void ObBlockManager::update_marker_status(const ObMacroBlockMarkerStatus &tmp_status)
{
SpinWLockGuard guard(marker_lock_);
marker_status_.reset();
marker_status_.total_block_count_ = get_total_macro_block_count();
marker_status_.reserved_block_count_ = io_device_->get_reserved_block_count() + reserved_count_;
marker_status_.linked_block_count_ = used_macro_cnt_[ObMacroBlockCommonHeader::LinkedBlock];
marker_status_.index_block_count_ = used_macro_cnt_[ObMacroBlockCommonHeader::SSTableIndex];
marker_status_.ids_block_count_ = used_macro_cnt_[ObMacroBlockCommonHeader::SSTableMacroID];
marker_status_.tmp_file_count_ = used_macro_cnt_[ObMacroBlockCommonHeader::TmpFileData];
marker_status_.data_block_count_ = used_macro_cnt_[ObMacroBlockCommonHeader::SSTableData];
marker_status_.shared_data_block_count_ = used_macro_cnt_[ObMacroBlockCommonHeader::SharedSSTableData];
marker_status_.disk_block_count_ = disk_block_count_;
marker_status_.bloomfiter_count_ = used_macro_cnt_[ObMacroBlockCommonHeader::BloomFilterData];
marker_status_.hold_count_ = hold_count_;
marker_status_.pending_free_count_ = pending_free_count_;
marker_status_.reserved_block_count_ = io_device_->get_reserved_block_count() + tmp_status.reserved_block_count_;
marker_status_.free_count_ = get_free_macro_block_count();
marker_status_.shared_meta_block_count_ = used_macro_cnt_[ObMacroBlockCommonHeader::SharedMetaData];
marker_status_.mark_cost_time_ = mark_cost_time_;
marker_status_.sweep_cost_time_ = sweep_cost_time_;
marker_status_.start_time_ = start_time_;
marker_status_.last_end_time_ = last_end_time_;
marker_status_.hold_info_ = hold_info_;
marker_status_.mark_finished_ = mark_finished;
marker_status_.hold_count_ = tmp_status.hold_count_;
marker_status_.mark_cost_time_ = tmp_status.mark_cost_time_;
marker_status_.sweep_cost_time_ = tmp_status.sweep_cost_time_;
marker_status_.start_time_ = tmp_status.start_time_;
marker_status_.last_end_time_ = tmp_status.last_end_time_;
marker_status_.mark_finished_ = tmp_status.mark_finished_;
if (tmp_status.mark_finished_) {
marker_status_.linked_block_count_ = tmp_status.linked_block_count_;
marker_status_.index_block_count_ = tmp_status.index_block_count_;
marker_status_.ids_block_count_ = tmp_status.ids_block_count_;
marker_status_.tmp_file_count_ = tmp_status.tmp_file_count_;
marker_status_.data_block_count_ = tmp_status.data_block_count_;
marker_status_.shared_data_block_count_ = tmp_status.shared_data_block_count_;
marker_status_.pending_free_count_ = tmp_status.pending_free_count_;
marker_status_.shared_meta_block_count_ = tmp_status.shared_meta_block_count_;
marker_status_.hold_info_ = tmp_status.hold_info_;
}
}
bool ObBlockManager::GetOldestHoldBlockFunctor::operator()(
@ -879,7 +867,7 @@ bool ObBlockManager::CopyBlockToArrayFunctor::operator()(const MacroBlockId &mac
{
int ret = OB_SUCCESS;
if (OB_UNLIKELY(!can_free)) {
ret = OB_ERR_UNEXPECTED;
// ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected error, this block cannot be freed", K(macro_id), K(can_free));
} else if (OB_FAIL(block_ids_.push_back(macro_id))) {
LOG_WARN("fail to push back block id into array", K(ret), K(macro_id));
@ -912,7 +900,8 @@ int ObBlockManager::do_sweep(MacroBlkIdMap &mark_info)
ret = functor.get_ret_code();
LOG_WARN("fail to copy block into pending free list", K(ret));
} else {
for (int64_t i = 0; OB_SUCC(ret) && i < blocks.count(); i++) {
// ignore ret to sweep all blocks
for (int64_t i = 0; i < blocks.count(); i++) {
const MacroBlockId &macro_id = blocks.at(i);
ObBucketHashWLockGuard lock_guard(bucket_lock_, macro_id.hash());
BlockInfo block_info;
@ -940,7 +929,10 @@ void ObBlockManager::mark_and_sweep()
int ret = OB_SUCCESS;
ObHashSet<MacroBlockId> macro_id_set;
MacroBlkIdMap mark_info;
bool mark_finished = true;
ObMacroBlockMarkerStatus tmp_status;
bool skip_mark = false;
// we must assign alloc_num_ before mark_macro_blocks, because it will be set to 0 in this func
int64_t alloc_num = 0;
if (IS_NOT_INIT) {
ret = OB_NOT_INIT;
@ -957,78 +949,69 @@ void ObBlockManager::mark_and_sweep()
} else if (OB_FAIL(macro_id_set.create(MAX(2, block_map_.count())))) {
LOG_WARN("fail to create macro id set", K(ret));
} else {
ATOMIC_SET(&start_time_, ObTimeUtility::fast_current_time());
if (OB_FAIL(mark_macro_blocks(mark_info, macro_id_set))) {//mark
GetPendingFreeBlockFunctor pending_free_functor(mark_info, tmp_status.hold_count_);
tmp_status.start_time_ = ObTimeUtility::fast_current_time();
if (OB_FAIL(block_map_.for_each(pending_free_functor))) {
ret = pending_free_functor.get_ret_code();
LOG_WARN("fail to get pending free blocks", K(ret));
} else if (0 == (alloc_num = ATOMIC_SET(&alloc_num_, 0)) && 0 == mark_info.count()) {
skip_mark = true;
LOG_INFO("no block alloc/free, no need to mark blocks", K(ret));
} else if (OB_FAIL(mark_macro_blocks(mark_info, macro_id_set, tmp_status))) {//mark
if (OB_EAGAIN == ret) {
mark_finished = false;
tmp_status.mark_finished_ = false;
ret = OB_SUCCESS;
// skip marking
} else {
LOG_WARN("fail to mark macro blocks", K(ret));
}
} else {
tmp_status.mark_finished_ = true;
}
if (OB_FAIL(ret)) {
// do nothing
} else {
pending_free_count_ += mark_info.count();
mark_cost_time_ = ObTimeUtility::fast_current_time() - start_time_;
tmp_status.pending_free_count_ += mark_info.count();
tmp_status.mark_cost_time_ = ObTimeUtility::fast_current_time() - tmp_status.start_time_;
//sweep
SpinWLockGuard guard(sweep_lock_);
if (OB_FAIL(do_sweep(mark_info))) {
LOG_WARN("do sweep fail", K(ret));
} else {
last_end_time_ = ObTimeUtility::fast_current_time();
sweep_cost_time_ = last_end_time_ - start_time_ - mark_cost_time_;
} else if (!skip_mark) {
tmp_status.last_end_time_ = ObTimeUtility::fast_current_time();
tmp_status.sweep_cost_time_ = tmp_status.last_end_time_ - tmp_status.start_time_ - tmp_status.mark_cost_time_;
GetOldestHoldBlockFunctor functor(macro_id_set, hold_info_);
if (OB_FAIL(block_map_.for_each(functor))) {
ret = functor.get_ret_code();
GetOldestHoldBlockFunctor hold_info_functor(macro_id_set, tmp_status.hold_info_);
if (OB_FAIL(block_map_.for_each(hold_info_functor))) {
ret = hold_info_functor.get_ret_code();
LOG_WARN("fail to get oldest hold block", K(ret));
} else {
update_marker_status(mark_finished);
FLOG_INFO("finish once mark and sweep", K(ret), K_(marker_status), "map_cnt", block_map_.count());
update_marker_status(tmp_status);
}
}
}
FLOG_INFO("finish once mark and sweep", K(ret), K(alloc_num), K_(marker_status), "map_cnt", block_map_.count());
}
}
macro_id_set.destroy();
}
void ObBlockManager::reset_mark_status()
{
MEMSET(used_macro_cnt_, 0, sizeof(used_macro_cnt_));
reserved_count_ = 0;
hold_count_ = 0;
pending_free_count_ = 0;
disk_block_count_ = 0;
mark_cost_time_ = 0;
sweep_cost_time_ = 0;
hold_info_.reset();
}
int ObBlockManager::mark_macro_blocks(
MacroBlkIdMap &mark_info,
common::hash::ObHashSet<MacroBlockId> &macro_id_set)
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status)
{
int ret = OB_SUCCESS;
omt::ObMultiTenant *omt = GCTX.omt_;
common::ObSEArray<uint64_t, 8> mtl_tenant_ids;
int64_t disk_blk_cnt = 0;
int64_t hold_cnt = 0;
GetPendingFreeBlockFunctor functor(mark_info, disk_blk_cnt, hold_cnt);
if (OB_ISNULL(omt)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected error, omt is nullptr", K(ret), KP(omt));
} else if (OB_FAIL(block_map_.for_each(functor))) {
ret = functor.get_ret_code();
LOG_WARN("fail to get pending free blocks", K(ret));
} else if (FALSE_IT(reset_mark_status())) {
} else if (FALSE_IT(hold_count_ = hold_cnt)) {
} else if (FALSE_IT(disk_block_count_ = disk_blk_cnt)) {
} else if (OB_FAIL(mark_tmp_file_blocks(mark_info, macro_id_set))) {
} else if (OB_FAIL(mark_tmp_file_blocks(mark_info, macro_id_set, tmp_status))) {
LOG_WARN("fail to mark tmp file blocks", K(ret));
} else if (OB_FAIL(mark_server_meta_blocks(mark_info, macro_id_set))) {
} else if (OB_FAIL(mark_server_meta_blocks(mark_info, macro_id_set, tmp_status))) {
LOG_WARN("fail to mark server meta blocks", K(ret));
} else {
omt->get_mtl_tenant_ids(mtl_tenant_ids);
@ -1036,14 +1019,14 @@ int ObBlockManager::mark_macro_blocks(
const uint64_t tenant_id = mtl_tenant_ids.at(i);
MacroBlockId macro_id;
MTL_SWITCH(tenant_id) {
if (OB_FAIL(mark_tenant_blocks(mark_info, macro_id_set))) {
if (OB_FAIL(mark_tenant_blocks(mark_info, macro_id_set, tmp_status))) {
LOG_WARN("fail to mark tenant blocks", K(ret), K(tenant_id));
} else if (OB_FALSE_IT(MTL(ObSharedMacroBlockMgr*)->get_cur_shared_block(macro_id))) {
} else if (OB_FAIL(mark_held_shared_block(macro_id, mark_info, macro_id_set))) {
} else if (OB_FAIL(mark_held_shared_block(macro_id, mark_info, macro_id_set, tmp_status))) {
LOG_WARN("fail to mark shared block held by shared_macro_block_manager", K(ret), K(macro_id));
} else if (OB_FALSE_IT(
MTL(ObTenantCheckpointSlogHandler*)->get_shared_block_reader_writer().get_cur_shared_block(macro_id))) {
} else if (OB_FAIL(mark_held_shared_block(macro_id, mark_info, macro_id_set))) {
} else if (OB_FAIL(mark_held_shared_block(macro_id, mark_info, macro_id_set, tmp_status))) {
LOG_WARN("fail to mark shared block held by shared_reader_writer", K(ret), K(macro_id));
}
}
@ -1055,7 +1038,8 @@ int ObBlockManager::mark_macro_blocks(
int ObBlockManager::mark_held_shared_block(
const MacroBlockId &macro_id,
MacroBlkIdMap &mark_info,
common::hash::ObHashSet<MacroBlockId> &macro_id_set)
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status)
{
int ret = OB_SUCCESS;
@ -1070,15 +1054,16 @@ int ObBlockManager::mark_held_shared_block(
ret = OB_SUCCESS;
}
} else {
hold_count_--;
reserved_count_++;
tmp_status.hold_count_--;
tmp_status.reserved_block_count_++;
}
return ret;
}
int ObBlockManager::mark_tenant_blocks(
MacroBlkIdMap &mark_info,
common::hash::ObHashSet<MacroBlockId> &macro_id_set)
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status)
{
int ret = OB_SUCCESS;
ObTenantCheckpointSlogHandler *ckpt_hdl = MTL(ObTenantCheckpointSlogHandler *);
@ -1086,7 +1071,7 @@ int ObBlockManager::mark_tenant_blocks(
if (OB_ISNULL(t3m) || OB_ISNULL(ckpt_hdl)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected error, t3m or ckpt hdl of mtl is nullptr", K(ret), KP(t3m), KP(ckpt_hdl));
} else if (OB_FAIL(mark_tenant_ckpt_blocks(mark_info, macro_id_set, *ckpt_hdl))) {
} else if (OB_FAIL(mark_tenant_ckpt_blocks(mark_info, macro_id_set, *ckpt_hdl, tmp_status))) {
LOG_WARN("fail to mark tenant meta blocks", K(ret));
} else {
ObArenaAllocator iter_allocator("MarkIter", OB_MALLOC_NORMAL_BLOCK_SIZE, MTL_ID());
@ -1107,9 +1092,9 @@ int ObBlockManager::mark_tenant_blocks(
} else {
LOG_WARN("fail to get next in-memory tablet", K(ret));
}
} else if (OB_FAIL(mark_tablet_meta_blocks(mark_info, handle, macro_id_set))) {
} else if (OB_FAIL(mark_tablet_meta_blocks(mark_info, handle, macro_id_set, tmp_status))) {
LOG_WARN("fail to mark tablet meta blocks", K(ret));
} else if (OB_FAIL(mark_sstable_blocks(mark_info, handle, macro_id_set))) {
} else if (OB_FAIL(mark_sstable_blocks(mark_info, handle, macro_id_set, tmp_status))) {
LOG_WARN("fail to mark tablet blocks", K(ret));
}
}
@ -1121,7 +1106,8 @@ int ObBlockManager::mark_tenant_blocks(
int ObBlockManager::mark_sstable_blocks(
MacroBlkIdMap &mark_info,
ObTabletHandle &handle,
common::hash::ObHashSet<MacroBlockId> &macro_id_set)
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status)
{
int ret = OB_SUCCESS;
ObTableStoreIterator table_store_iter(false, false);
@ -1150,14 +1136,12 @@ int ObBlockManager::mark_sstable_blocks(
} else if (OB_ISNULL(sstable)) {
ret = OB_ERR_UNEXPECTED;
LOG_WARN("unexpected error, sstable is nullptr", K(ret), KP(sstable));
} else if (OB_FAIL(mark_sstable_meta_block(*sstable, mark_info, macro_id_set))) {
} else if (OB_FAIL(mark_sstable_meta_block(*sstable, mark_info, macro_id_set, tmp_status))) {
LOG_WARN("fail to mark sstable meta block", K(ret), KPC(sstable));
} else if (OB_FAIL(sstable->get_meta(sstable_meta_hdl, &safe_allocator))) {
LOG_WARN("fail to get sstable meta", K(ret));
} else {
const ObSSTableMeta &meta = sstable_meta_hdl.get_sstable_meta();
ObMacroBlockCommonHeader::MacroBlockType macro_type;
macro_type = ObMacroBlockCommonHeader::MacroBlockType::SSTableData;
ObMacroIdIterator iterator;
MacroBlockId macro_id;
if (OB_FAIL(meta.get_macro_info().get_data_block_iter(iterator))) {
@ -1181,14 +1165,14 @@ int ObBlockManager::mark_sstable_blocks(
}
} else {
if (sstable->is_small_sstable()) {
macro_type = ObMacroBlockCommonHeader::MacroBlockType::SharedSSTableData;
tmp_status.shared_data_block_count_++;
} else {
tmp_status.data_block_count_++;
}
used_macro_cnt_[macro_type] ++;
hold_count_--;
tmp_status.hold_count_--;
}
}
if (OB_SUCC(ret)) {
macro_type = ObMacroBlockCommonHeader::MacroBlockType::SSTableIndex;
iterator.reset();
if (OB_FAIL(meta.get_macro_info().get_other_block_iter(iterator))) {
LOG_WARN("fail to get other block iterator", K(ret), K(meta));
@ -1207,12 +1191,11 @@ int ObBlockManager::mark_sstable_blocks(
} else if (OB_FAIL(macro_id_set.set_refactored(macro_id))) {
LOG_WARN("fail to put macro id into set", K(ret), K(macro_id));
} else {
used_macro_cnt_[macro_type] ++;
hold_count_--;
tmp_status.index_block_count_++;
tmp_status.hold_count_--;
}
}
if (OB_SUCC(ret)) {
macro_type = ObMacroBlockCommonHeader::MacroBlockType::SSTableMacroID;
iterator.reset();
if (OB_FAIL(meta.get_macro_info().get_linked_block_iter(iterator))) {
LOG_WARN("fail to get linked block iterator", K(ret), K(meta));
@ -1231,8 +1214,8 @@ int ObBlockManager::mark_sstable_blocks(
} else if (OB_FAIL(macro_id_set.set_refactored(macro_id))) {
LOG_WARN("fail to put macro id into set", K(ret), K(macro_id));
} else {
used_macro_cnt_[macro_type] ++;
hold_count_--;
tmp_status.ids_block_count_++;
tmp_status.hold_count_--;
}
}
}
@ -1244,7 +1227,8 @@ int ObBlockManager::mark_sstable_blocks(
int ObBlockManager::mark_tablet_meta_blocks(
MacroBlkIdMap &mark_info,
storage::ObTabletHandle &handle,
common::hash::ObHashSet<MacroBlockId> &macro_id_set)
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status)
{
int ret = OB_SUCCESS;
const ObTablet *tablet = handle.get_obj();
@ -1263,8 +1247,8 @@ int ObBlockManager::mark_tablet_meta_blocks(
ret = OB_SUCCESS;
}
} else {
used_macro_cnt_[ObMacroBlockCommonHeader::SharedMetaData]++;
hold_count_--;
tmp_status.shared_meta_block_count_++;
tmp_status.hold_count_--;
}
}
}
@ -1274,7 +1258,8 @@ int ObBlockManager::mark_tablet_meta_blocks(
int ObBlockManager::mark_sstable_meta_block(
const blocksstable::ObSSTable &sstable,
MacroBlkIdMap &mark_info,
common::hash::ObHashSet<MacroBlockId> &macro_id_set)
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status)
{
int ret = OB_SUCCESS;
const ObMetaDiskAddr &addr = sstable.get_addr();
@ -1292,8 +1277,8 @@ int ObBlockManager::mark_sstable_meta_block(
ret = OB_SUCCESS;
}
} else {
used_macro_cnt_[ObMacroBlockCommonHeader::SharedMetaData]++;
hold_count_--;
tmp_status.shared_meta_block_count_++;
tmp_status.hold_count_--;
}
}
return ret;
@ -1302,7 +1287,8 @@ int ObBlockManager::mark_sstable_meta_block(
int ObBlockManager::mark_tenant_ckpt_blocks(
MacroBlkIdMap &mark_info,
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObTenantCheckpointSlogHandler &hdl)
ObTenantCheckpointSlogHandler &hdl,
ObMacroBlockMarkerStatus &tmp_status)
{
int ret = OB_SUCCESS;
ObArray<MacroBlockId> macro_block_list;
@ -1314,15 +1300,16 @@ int ObBlockManager::mark_tenant_ckpt_blocks(
} else if (OB_FAIL(update_mark_info(macro_block_list, macro_id_set, mark_info))){
LOG_WARN("fail to update mark info", K(ret), K(macro_block_list.count()));
} else {
used_macro_cnt_[ObMacroBlockCommonHeader::LinkedBlock] += macro_block_list.count();
hold_count_ -= macro_block_list.count();
tmp_status.linked_block_count_ += macro_block_list.count();
tmp_status.hold_count_ -= macro_block_list.count();
}
return ret;
}
int ObBlockManager::mark_tmp_file_blocks(
MacroBlkIdMap &mark_info,
common::hash::ObHashSet<MacroBlockId> &macro_id_set)
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status)
{
int ret = OB_SUCCESS;
ObArray<MacroBlockId> macro_block_list;
@ -1334,15 +1321,16 @@ int ObBlockManager::mark_tmp_file_blocks(
} else if (OB_FAIL(update_mark_info(macro_block_list, macro_id_set, mark_info))){
LOG_WARN("fail to update mark info", K(ret), K(macro_block_list.count()));
} else {
used_macro_cnt_[ObMacroBlockCommonHeader::TmpFileData] += macro_block_list.count();
hold_count_ -= macro_block_list.count();
tmp_status.tmp_file_count_ += macro_block_list.count();
tmp_status.hold_count_ -= macro_block_list.count();
}
return ret;
}
int ObBlockManager::mark_server_meta_blocks(
MacroBlkIdMap &mark_info,
common::hash::ObHashSet<MacroBlockId> &macro_id_set)
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status)
{
int ret = OB_SUCCESS;
ObArray<MacroBlockId> macro_block_list;
@ -1354,8 +1342,8 @@ int ObBlockManager::mark_server_meta_blocks(
} else if (OB_FAIL(update_mark_info(macro_block_list, macro_id_set, mark_info))){
LOG_WARN("fail to update mark info", K(ret), K(macro_block_list.count()));
} else {
used_macro_cnt_[ObMacroBlockCommonHeader::LinkedBlock] += macro_block_list.count();
hold_count_ -= macro_block_list.count();
tmp_status.linked_block_count_ += macro_block_list.count();
tmp_status.hold_count_ -= macro_block_list.count();
}
return ret;
}

View File

@ -294,11 +294,9 @@ private:
public:
GetPendingFreeBlockFunctor(
MacroBlkIdMap &blk_map,
int64_t &disk_block_count,
int64_t &hold_count)
: ret_code_(common::OB_SUCCESS),
blk_map_(blk_map),
disk_block_count_(disk_block_count),
hold_count_(hold_count)
{}
~GetPendingFreeBlockFunctor() = default;
@ -309,7 +307,6 @@ private:
private:
int ret_code_;
MacroBlkIdMap &blk_map_;
int64_t &disk_block_count_;
int64_t &hold_count_;
};
@ -333,40 +330,47 @@ private:
private:
int get_macro_block_info(const MacroBlockId &macro_id, ObMacroBlockInfo &macro_block_info) const;
bool is_bad_block(const MacroBlockId &macro_block_id);
void reset_mark_status();
int mark_macro_blocks(
MacroBlkIdMap &mark_info,
common::hash::ObHashSet<MacroBlockId> &macro_id_set);
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status);
int mark_held_shared_block(
const MacroBlockId &macro_id,
MacroBlkIdMap &mark_info,
common::hash::ObHashSet<MacroBlockId> &macro_id_set);
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status);
int mark_tenant_blocks(
MacroBlkIdMap &mark_info,
common::hash::ObHashSet<MacroBlockId> &macro_id_set);
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status);
int mark_sstable_blocks(
MacroBlkIdMap &mark_info,
storage::ObTabletHandle &handle,
common::hash::ObHashSet<MacroBlockId> &macro_id_set);
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status);
int mark_sstable_meta_block(
const blocksstable::ObSSTable &sstable,
MacroBlkIdMap &mark_info,
common::hash::ObHashSet<MacroBlockId> &macro_id_set);
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status);
int mark_tablet_meta_blocks(
MacroBlkIdMap &mark_info,
storage::ObTabletHandle &handle,
common::hash::ObHashSet<MacroBlockId> &macro_id_set);
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status);
int mark_tenant_ckpt_blocks(
MacroBlkIdMap &mark_info,
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
storage::ObTenantCheckpointSlogHandler &hdl);
storage::ObTenantCheckpointSlogHandler &hdl,
ObMacroBlockMarkerStatus &tmp_status);
int mark_tmp_file_blocks(
MacroBlkIdMap &mark_info,
common::hash::ObHashSet<MacroBlockId> &macro_id_set);
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status);
int mark_server_meta_blocks(
MacroBlkIdMap &mark_info,
common::hash::ObHashSet<MacroBlockId> &macro_id_set);
common::hash::ObHashSet<MacroBlockId> &macro_id_set,
ObMacroBlockMarkerStatus &tmp_status);
bool continue_mark();
int do_sweep(MacroBlkIdMap &mark_info);
@ -377,7 +381,7 @@ private:
int update_mark_info(
const MacroBlockId &macro_id,
MacroBlkIdMap &mark_info);
void update_marker_status(const bool mark_finished);
void update_marker_status(const ObMacroBlockMarkerStatus &tmp_status);
void disable_mark_sweep() { ATOMIC_SET(&is_mark_sweep_enabled_, false); }
void enable_mark_sweep() { ATOMIC_SET(&is_mark_sweep_enabled_, true); }
bool is_mark_sweep_enabled() { return ATOMIC_LOAD(&is_mark_sweep_enabled_); }
@ -451,18 +455,6 @@ private:
storage::ObServerSuperBlock super_block_; // read only memory cache
ObSuperBlockBufferHolder super_block_buf_holder_;
int64_t default_block_size_;
// mark status
int64_t used_macro_cnt_[ObMacroBlockCommonHeader::MaxMacroType];
int64_t mark_cost_time_;
int64_t sweep_cost_time_;
int64_t reserved_count_;
int64_t hold_count_;
int64_t pending_free_count_;
int64_t disk_block_count_;
int64_t start_time_;
int64_t last_end_time_;
ObSimpleMacroBlockInfo hold_info_;
ObMacroBlockMarkerStatus marker_status_;
common::SpinRWLock marker_lock_;
@ -478,11 +470,11 @@ private:
common::ObIODevice *io_device_;
ObMacroBlockSeqGenerator blk_seq_generator_;
int64_t alloc_num_;
lib::ObMutex resize_file_lock_;
bool is_inited_;
bool is_started_;
lib::ObMutex resize_file_lock_;
};
class ObServerBlockManager : public ObBlockManager

View File

@ -1122,6 +1122,10 @@ void ObSSTable::dec_macro_ref() const
common::ObArenaAllocator tmp_allocator(common::ObMemAttr(MTL_ID(), "CacheSST"));
ObSafeArenaAllocator safe_allocator(tmp_allocator);
ObSSTableMetaHandle meta_handle;
if (OB_FAIL(dec_used_size())) {// ignore ret
LOG_WARN("fail to dec used size of shared block", K(ret));
}
// ignore ret and decrease ref cnt
if (OB_FAIL(get_meta(meta_handle, &safe_allocator))) {
LOG_ERROR("fail to get sstable meta", K(ret));
} else if (OB_UNLIKELY(!meta_handle.is_valid())) {
@ -1164,10 +1168,6 @@ void ObSSTable::dec_macro_ref() const
}
}
}
if (OB_FAIL(dec_used_size())) {// ignore ret
LOG_ERROR("fail to dec used size of shared block", K(ret));
}
}
int ObSSTable::inc_macro_ref(bool &inc_success) const

View File

@ -169,18 +169,18 @@ TEST_F(TestBlockManager, test_mark_and_sweep)
ret = macro_id_set.create(MAX(2, OB_SERVER_BLOCK_MGR.block_map_.count()));
ASSERT_EQ(OB_SUCCESS, ret);
int64_t safe_ts = ObTimeUtility::current_time();
int64_t disk_blk_cnt = 0;
int64_t hold_cnt = 0;
ObBlockManager::GetPendingFreeBlockFunctor functor(mark_info, disk_blk_cnt, hold_cnt);
ObBlockManager::GetPendingFreeBlockFunctor functor(mark_info, hold_cnt);
ret = OB_SERVER_BLOCK_MGR.block_map_.for_each(functor);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(blk_cnt - 1, mark_info.count());
ret = OB_SERVER_BLOCK_MGR.mark_server_meta_blocks(mark_info, macro_id_set);
ObMacroBlockMarkerStatus tmp_status;
ret = OB_SERVER_BLOCK_MGR.mark_server_meta_blocks(mark_info, macro_id_set, tmp_status);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(blk_cnt - 1, mark_info.count());
ret = OB_SERVER_BLOCK_MGR.mark_tmp_file_blocks(mark_info, macro_id_set);
ret = OB_SERVER_BLOCK_MGR.mark_tmp_file_blocks(mark_info, macro_id_set, tmp_status);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(blk_cnt - 1, mark_info.count());

View File

@ -244,7 +244,7 @@ TEST_F(TestRefCnt, test_1_0_1)
ret = OB_SERVER_BLOCK_MGR.inc_ref(macro_id);
ASSERT_EQ(OB_SUCCESS, ret);
}
count = 10;
count = 9;
while (count--) {
ret = OB_SERVER_BLOCK_MGR.dec_ref(macro_id);
ASSERT_EQ(OB_SUCCESS, ret);
@ -252,9 +252,9 @@ TEST_F(TestRefCnt, test_1_0_1)
ObMacroBlockInfo block_info;
ret = OB_SERVER_BLOCK_MGR.get_macro_block_info(macro_id, block_info);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(0, block_info.ref_cnt_);
ASSERT_EQ(1, block_info.ref_cnt_);
count = 10;
count = 9;
while (count--) {
ret = OB_SERVER_BLOCK_MGR.inc_ref(macro_id);
ASSERT_EQ(OB_SUCCESS, ret);
@ -268,6 +268,9 @@ TEST_F(TestRefCnt, test_1_0_1)
ret = OB_SERVER_BLOCK_MGR.get_macro_block_info(macro_id, block_info);
ASSERT_EQ(OB_SUCCESS, ret);
ASSERT_EQ(1, block_info.ref_cnt_);
ret = OB_SERVER_BLOCK_MGR.dec_ref(macro_id);
ASSERT_NE(OB_SUCCESS, OB_SERVER_BLOCK_MGR.inc_ref(macro_id));
}
/*TEST_F(TestStorageFile, test_not_enough_mem)