|
|
|
@ -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 ¯o_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 ¯o_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 ¯o_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> ¯o_id_set)
|
|
|
|
|
common::hash::ObHashSet<MacroBlockId> ¯o_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 ¯o_id,
|
|
|
|
|
MacroBlkIdMap &mark_info,
|
|
|
|
|
common::hash::ObHashSet<MacroBlockId> ¯o_id_set)
|
|
|
|
|
common::hash::ObHashSet<MacroBlockId> ¯o_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> ¯o_id_set)
|
|
|
|
|
common::hash::ObHashSet<MacroBlockId> ¯o_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> ¯o_id_set)
|
|
|
|
|
common::hash::ObHashSet<MacroBlockId> ¯o_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> ¯o_id_set)
|
|
|
|
|
common::hash::ObHashSet<MacroBlockId> ¯o_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> ¯o_id_set)
|
|
|
|
|
common::hash::ObHashSet<MacroBlockId> ¯o_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> ¯o_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> ¯o_id_set)
|
|
|
|
|
common::hash::ObHashSet<MacroBlockId> ¯o_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> ¯o_id_set)
|
|
|
|
|
common::hash::ObHashSet<MacroBlockId> ¯o_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;
|
|
|
|
|
}
|
|
|
|
|