add mysqltest case for medium compaction
This commit is contained in:
parent
1511bfcffa
commit
4260921c59
@ -657,6 +657,7 @@ void ObMultipleMerge::report_tablet_stat()
|
||||
// empty query, ignore it
|
||||
} else {
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
bool report_succ = false; /*placeholder*/
|
||||
storage::ObTabletStat tablet_stat;
|
||||
tablet_stat.ls_id_ = access_ctx_->table_store_stat_.ls_id_.id();
|
||||
tablet_stat.tablet_id_ = access_ctx_->table_store_stat_.tablet_id_.id();
|
||||
@ -665,7 +666,7 @@ void ObMultipleMerge::report_tablet_stat()
|
||||
tablet_stat.scan_physical_row_cnt_ = access_ctx_->table_store_stat_.physical_read_cnt_;
|
||||
tablet_stat.scan_micro_block_cnt_ = access_param_->iter_param_.enable_pd_blockscan() ? access_ctx_->table_store_stat_.micro_access_cnt_ : 0;
|
||||
tablet_stat.pushdown_micro_block_cnt_ = access_ctx_->table_store_stat_.pushdown_micro_access_cnt_;
|
||||
if (OB_TMP_FAIL(MTL(storage::ObTenantTabletStatMgr *)->report_stat(tablet_stat))) {
|
||||
if (OB_TMP_FAIL(MTL(storage::ObTenantTabletStatMgr *)->report_stat(tablet_stat, report_succ))) {
|
||||
STORAGE_LOG_RET(WARN, tmp_ret, "failed to report tablet stat", K(tmp_ret), K(tablet_stat));
|
||||
}
|
||||
}
|
||||
|
@ -1178,7 +1178,7 @@ int ObPartitionMinorMerger::merge_partition(
|
||||
STORAGE_LOG(WARN, "failed to close partition merger", K(ret));
|
||||
} else if (ctx.param_.tablet_id_.is_special_merge_tablet()) {
|
||||
// do nothing
|
||||
} else {
|
||||
} else if (is_mini_merge(merge_param.merge_type_)) {
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
if (OB_TMP_FAIL(collect_merge_stat(merge_param.merge_type_, merge_helper, ctx))) {
|
||||
STORAGE_LOG(WARN, "failed to collect merge stat", K(tmp_ret), K(merge_param));
|
||||
|
@ -1200,26 +1200,29 @@ int ObTabletMergeFinishTask::add_sstable_for_merge(ObTabletMergeCtx &ctx)
|
||||
int ObTabletMergeFinishTask::try_report_tablet_stat_after_mini(ObTabletMergeCtx &ctx)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
const share::ObLSID &ls_id = ctx.param_.ls_id_;
|
||||
const ObTabletID &tablet_id = ctx.param_.tablet_id_;
|
||||
|
||||
const ObTransNodeDMLStat &tnode_stat = ctx.tnode_stat_;
|
||||
bool report_succ = false;
|
||||
|
||||
if (tablet_id.is_special_merge_tablet()) {
|
||||
// no need report
|
||||
} else if (ObTabletStat::MERGE_REPORT_MIN_ROW_CNT >= tnode_stat.get_dml_count()) {
|
||||
// insufficient data, skip to report
|
||||
} else {
|
||||
// always report tablet stat whether _enable_adaptive_compaction is true or not for mini compaction
|
||||
ObTabletStat report_stat;
|
||||
report_stat.ls_id_ = ctx.param_.ls_id_.id(),
|
||||
report_stat.tablet_id_ = ctx.param_.tablet_id_.id();
|
||||
report_stat.ls_id_ = ls_id.id();
|
||||
report_stat.tablet_id_ = tablet_id.id();
|
||||
report_stat.merge_cnt_ = 1;
|
||||
report_stat.insert_row_cnt_ = tnode_stat.insert_row_count_;
|
||||
report_stat.update_row_cnt_ = tnode_stat.update_row_count_;
|
||||
report_stat.delete_row_cnt_ = tnode_stat.delete_row_count_;
|
||||
if (OB_FAIL(MTL(ObTenantTabletStatMgr *)->report_stat(report_stat))) {
|
||||
if (OB_FAIL(MTL(ObTenantTabletStatMgr *)->report_stat(report_stat, report_succ))) {
|
||||
STORAGE_LOG(WARN, "failed to report tablet stat", K(ret));
|
||||
}
|
||||
}
|
||||
FLOG_INFO("try report tablet stat", K(ret), K(ls_id), K(tablet_id), K(tnode_stat), K(report_succ));
|
||||
return ret;
|
||||
}
|
||||
|
||||
@ -1294,11 +1297,10 @@ int ObTabletMergeFinishTask::try_schedule_compaction_after_mini(
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
const ObTabletID &tablet_id = ctx.param_.tablet_id_;
|
||||
ObLSID ls_id = ctx.param_.ls_id_;
|
||||
bool enable_adaptive_compaction = MTL(ObTenantTabletScheduler *)->enable_adaptive_compaction();
|
||||
// report tablet stat
|
||||
if (0 == ctx.get_merge_info().get_sstable_merge_info().macro_block_count_) {
|
||||
// empty mini compaction, no need to reprot stat
|
||||
} else if (enable_adaptive_compaction && OB_TMP_FAIL(try_report_tablet_stat_after_mini(ctx))) {
|
||||
} else if (OB_TMP_FAIL(try_report_tablet_stat_after_mini(ctx))) {
|
||||
LOG_WARN("failed to report table stat after mini compaction", K(tmp_ret), K(ls_id), K(tablet_id));
|
||||
}
|
||||
if (OB_TMP_FAIL(ObMediumCompactionScheduleFunc::schedule_tablet_medium_merge(
|
||||
|
@ -556,14 +556,17 @@ void ObTenantTabletStatMgr::destroy()
|
||||
FLOG_INFO("ObTenantTabletStatMgr destroyed!");
|
||||
}
|
||||
|
||||
int ObTenantTabletStatMgr::report_stat(const ObTabletStat &stat)
|
||||
int ObTenantTabletStatMgr::report_stat(
|
||||
const ObTabletStat &stat,
|
||||
bool &succ_report)
|
||||
{
|
||||
int ret = OB_SUCCESS;
|
||||
succ_report = false;
|
||||
|
||||
if (IS_NOT_INIT) {
|
||||
ret = OB_NOT_INIT;
|
||||
LOG_WARN("ObTenantTabletStatMgr not inited", K(ret));
|
||||
} else if (!stat.is_valid()) {
|
||||
} else if (OB_UNLIKELY(!stat.is_valid())) {
|
||||
ret = OB_INVALID_ARGUMENT;
|
||||
LOG_WARN("get invalid arguments", K(ret), K(stat));
|
||||
} else if (!stat.check_need_report()) {
|
||||
@ -580,6 +583,7 @@ int ObTenantTabletStatMgr::report_stat(const ObTabletStat &stat)
|
||||
}
|
||||
} else {
|
||||
report_queue_[pending_cur % DEFAULT_MAX_PENDING_CNT] = stat;
|
||||
succ_report = true;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
@ -717,21 +721,6 @@ int ObTenantTabletStatMgr::fetch_node(ObTabletStreamNode *&node)
|
||||
return ret;
|
||||
}
|
||||
|
||||
void ObTenantTabletStatMgr::dump_tablet_stat_status()
|
||||
{
|
||||
if (REACH_TENANT_TIME_INTERVAL(DUMP_TABLET_STAT_INTERVAL)) {
|
||||
uint64_t start_idx = report_cursor_; // it's OK to dirty read
|
||||
uint64_t end_idx = pending_cursor_;
|
||||
int64_t map_size = stream_map_.size();
|
||||
int64_t stream_node_cnt = stream_pool_.get_allocated_num();
|
||||
|
||||
LOG_INFO("dump_tablet_stat_status",
|
||||
"queue_cnt", end_idx - start_idx, K(start_idx), K(end_idx),
|
||||
"map_size", map_size,
|
||||
"stream_node_cnt", stream_node_cnt);
|
||||
}
|
||||
}
|
||||
|
||||
void ObTenantTabletStatMgr::process_stats()
|
||||
{
|
||||
int tmp_ret = OB_SUCCESS;
|
||||
@ -767,7 +756,6 @@ void ObTenantTabletStatMgr::refresh_all(const int64_t step)
|
||||
|
||||
void ObTenantTabletStatMgr::TabletStatUpdater::runTimerTask()
|
||||
{
|
||||
mgr_.dump_tablet_stat_status();
|
||||
mgr_.process_stats();
|
||||
|
||||
int64_t interval_step = 0;
|
||||
|
@ -291,7 +291,9 @@ public:
|
||||
void stop();
|
||||
void destroy();
|
||||
|
||||
int report_stat(const ObTabletStat &stat);
|
||||
int report_stat(
|
||||
const ObTabletStat &stat,
|
||||
bool &succ_report);
|
||||
int get_latest_tablet_stat(
|
||||
const share::ObLSID &ls_id,
|
||||
const common::ObTabletID &tablet_id,
|
||||
@ -302,7 +304,6 @@ public:
|
||||
common::ObIArray<ObTabletStat> &tablet_stats);
|
||||
void process_stats();
|
||||
void refresh_all(const int64_t step);
|
||||
void dump_tablet_stat_status();
|
||||
private:
|
||||
class TabletStatUpdater : public common::ObTimerTask
|
||||
{
|
||||
|
@ -1484,7 +1484,8 @@ int ObTablet::do_rowkey_exists(
|
||||
} else if (FALSE_IT(store_ctx.tablet_stat_.exist_row_read_table_cnt_ = check_table_cnt)) {
|
||||
} else if (FALSE_IT(store_ctx.tablet_stat_.exist_row_total_table_cnt_ = table_iter.count())) {
|
||||
} else if (MTL(ObTenantTabletScheduler *)->enable_adaptive_compaction()) {
|
||||
if (OB_TMP_FAIL(MTL(ObTenantTabletStatMgr *)->report_stat(store_ctx.tablet_stat_))) {
|
||||
bool report_succ = false; /*placeholder*/
|
||||
if (OB_TMP_FAIL(MTL(ObTenantTabletStatMgr *)->report_stat(store_ctx.tablet_stat_, report_succ))) {
|
||||
LOG_WARN("failed to report tablet stat", K(tmp_ret), K(stat));
|
||||
}
|
||||
}
|
||||
@ -1538,7 +1539,8 @@ int ObTablet::do_rowkeys_exist(ObTableStoreIterator &tables_iter, ObRowsInfo &ro
|
||||
if (0 == access_ctx.table_store_stat_.exist_row_.empty_read_cnt_) {
|
||||
// ROWKEY IN_ROW_CACHE / NOT EXIST
|
||||
} else if (MTL(ObTenantTabletScheduler *)->enable_adaptive_compaction()) {
|
||||
if (OB_TMP_FAIL(MTL(ObTenantTabletStatMgr *)->report_stat(tablet_stat))) {
|
||||
bool report_succ = false; /*placeholder*/
|
||||
if (OB_TMP_FAIL(MTL(ObTenantTabletStatMgr *)->report_stat(tablet_stat, report_succ))) {
|
||||
LOG_WARN("failed to report tablet stat", K(tmp_ret), K(tablet_stat));
|
||||
}
|
||||
}
|
||||
|
@ -101,7 +101,8 @@ void TestTenantTabletStatMgr::report(ObTenantTabletStatMgr *mgr, const ObTabletS
|
||||
{
|
||||
ASSERT_TRUE(NULL != mgr);
|
||||
ASSERT_TRUE(stat.is_valid());
|
||||
ASSERT_EQ(OB_SUCCESS, mgr->report_stat(stat));
|
||||
bool report_succ = false;
|
||||
ASSERT_EQ(OB_SUCCESS, mgr->report_stat(stat, report_succ));
|
||||
}
|
||||
|
||||
void TestTenantTabletStatMgr::batch_report_stat(int64_t report_num)
|
||||
@ -384,7 +385,8 @@ TEST_F(TestTenantTabletStatMgr, basic_tablet_stat_mgr)
|
||||
tablet_stat.scan_logical_row_cnt_ = 100;
|
||||
tablet_stat.scan_physical_row_cnt_ = 100;
|
||||
|
||||
ret = stat_mgr_->report_stat(tablet_stat);
|
||||
bool report_succ = false;
|
||||
ret = stat_mgr_->report_stat(tablet_stat, report_succ);
|
||||
ASSERT_EQ(OB_SUCCESS, ret);
|
||||
stat_mgr_->process_stats();
|
||||
|
||||
|
Loading…
x
Reference in New Issue
Block a user