[CP] should not push multi_version_start before reserved_snapshot_clog sync
This commit is contained in:
		| @ -325,9 +325,13 @@ TEST_F(TestMediumInfoReader, pure_mds_table) | |||||||
|     ObTabletMediumInfoReader reader(*tablet); |     ObTabletMediumInfoReader reader(*tablet); | ||||||
|     ASSERT_EQ(OB_SUCCESS, reader.init(allocator)); |     ASSERT_EQ(OB_SUCCESS, reader.init(allocator)); | ||||||
|     int64_t medium_snapshot = 0; |     int64_t medium_snapshot = 0; | ||||||
|     ret = reader.get_min_medium_snapshot(medium_snapshot); |     ret = reader.get_min_medium_snapshot(0, medium_snapshot); | ||||||
|     ASSERT_EQ(OB_SUCCESS, ret); |     ASSERT_EQ(OB_SUCCESS, ret); | ||||||
|     ASSERT_EQ(medium_snapshot, 1); |     ASSERT_EQ(medium_snapshot, 1); | ||||||
|  |  | ||||||
|  |     ret = reader.get_min_medium_snapshot(1, medium_snapshot); | ||||||
|  |     ASSERT_EQ(OB_SUCCESS, ret); | ||||||
|  |     ASSERT_EQ(medium_snapshot, 2); | ||||||
|   } |   } | ||||||
| } | } | ||||||
|  |  | ||||||
| @ -494,7 +498,7 @@ TEST_F(TestMediumInfoReader, mds_table_dump_data_overlap) | |||||||
|     ObTabletMediumInfoReader reader(*tablet); |     ObTabletMediumInfoReader reader(*tablet); | ||||||
|     ASSERT_EQ(OB_SUCCESS, reader.init(allocator)); |     ASSERT_EQ(OB_SUCCESS, reader.init(allocator)); | ||||||
|     int64_t medium_snapshot = 0; |     int64_t medium_snapshot = 0; | ||||||
|     ret = reader.get_min_medium_snapshot(medium_snapshot); |     ret = reader.get_min_medium_snapshot(0, medium_snapshot); | ||||||
|     ASSERT_EQ(OB_SUCCESS, ret); |     ASSERT_EQ(OB_SUCCESS, ret); | ||||||
|     ASSERT_EQ(medium_snapshot, 1); |     ASSERT_EQ(medium_snapshot, 1); | ||||||
|   } |   } | ||||||
| @ -620,7 +624,7 @@ TEST_F(TestMediumInfoReader, mds_table_dump_data_no_overlap) | |||||||
|     ObTabletMediumInfoReader reader(*tablet); |     ObTabletMediumInfoReader reader(*tablet); | ||||||
|     ASSERT_EQ(OB_SUCCESS, reader.init(allocator)); |     ASSERT_EQ(OB_SUCCESS, reader.init(allocator)); | ||||||
|     int64_t medium_snapshot = 0; |     int64_t medium_snapshot = 0; | ||||||
|     ret = reader.get_min_medium_snapshot(medium_snapshot); |     ret = reader.get_min_medium_snapshot(0, medium_snapshot); | ||||||
|     ASSERT_EQ(OB_SUCCESS, ret); |     ASSERT_EQ(OB_SUCCESS, ret); | ||||||
|     ASSERT_EQ(medium_snapshot, 1); |     ASSERT_EQ(medium_snapshot, 1); | ||||||
|   } |   } | ||||||
|  | |||||||
| @ -48,6 +48,22 @@ int ObSortColumnIdArray::build( | |||||||
|   return ret; |   return ret; | ||||||
| } | } | ||||||
|  |  | ||||||
|  | int32_t ObSortColumnIdArray::get_func_from_map( | ||||||
|  |   ObSortColumnIdArray &sort_array, | ||||||
|  |   const int64_t column_id, | ||||||
|  |   int64_t &input_array_idx) | ||||||
|  | { | ||||||
|  |   int ret = OB_SUCCESS; | ||||||
|  |   if (OB_FAIL(sort_array.map_.get_refactored(column_id, input_array_idx))) { | ||||||
|  |     if (OB_HASH_NOT_EXIST == ret) { | ||||||
|  |       ret = OB_ENTRY_NOT_EXIST; | ||||||
|  |     } else { | ||||||
|  |       LOG_WARN("failed to get column id from map", KR(ret), K(column_id)); | ||||||
|  |     } | ||||||
|  |   } | ||||||
|  |   return ret; | ||||||
|  | } | ||||||
|  |  | ||||||
| int ObSortColumnIdArray::build_hash_map( | int ObSortColumnIdArray::build_hash_map( | ||||||
|   const uint64_t tenant_id, |   const uint64_t tenant_id, | ||||||
|   const ObIArray<ObColDesc> &column_descs) |   const ObIArray<ObColDesc> &column_descs) | ||||||
| @ -63,17 +79,25 @@ int ObSortColumnIdArray::build_hash_map( | |||||||
|     } |     } | ||||||
|   } // end of for |   } // end of for | ||||||
|   if (OB_SUCC(ret)) { |   if (OB_SUCC(ret)) { | ||||||
|     get_func_ = [](ObSortColumnIdArray &sort_array, const int64_t column_id, int64_t &array_idx) -> int { |     get_func_ = get_func_from_map; | ||||||
|       int ret = OB_SUCCESS; |   } | ||||||
|       if (OB_FAIL(sort_array.map_.get_refactored(column_id, array_idx))) { |   return ret; | ||||||
|         if (OB_HASH_NOT_EXIST == ret) { | } | ||||||
|           ret = OB_ENTRY_NOT_EXIST; |  | ||||||
|         } else { | int32_t ObSortColumnIdArray::get_func_from_array( | ||||||
|           LOG_WARN("failed to get column id from map", KR(ret), K(column_id)); |   ObSortColumnIdArray &sort_array, | ||||||
|         } |   const int64_t column_id, | ||||||
|       } |   int64_t &input_array_idx) | ||||||
|       return ret; | { | ||||||
|     }; |   int ret = OB_SUCCESS; | ||||||
|  |   const int64_t array_idx = | ||||||
|  |       std::lower_bound(sort_array.array_.begin(), sort_array.array_.end(), | ||||||
|  |                        ObColumnIdToIdx(column_id)) - sort_array.array_.begin(); | ||||||
|  |   if ((sort_array.array_.count() != array_idx) && | ||||||
|  |       (sort_array.array_[array_idx].column_id_ == column_id)) { | ||||||
|  |     input_array_idx = sort_array.array_[array_idx].idx_; | ||||||
|  |   } else { | ||||||
|  |     ret = OB_ENTRY_NOT_EXIST; | ||||||
|   } |   } | ||||||
|   return ret; |   return ret; | ||||||
| } | } | ||||||
| @ -97,20 +121,7 @@ int ObSortColumnIdArray::build_sort_array( | |||||||
|     if (OB_SUCC(ret)) { |     if (OB_SUCC(ret)) { | ||||||
|       std::sort(array_.begin(), array_.end()); |       std::sort(array_.begin(), array_.end()); | ||||||
|       LOG_TRACE("success to sort array", KR(ret), K(array_)); |       LOG_TRACE("success to sort array", KR(ret), K(array_)); | ||||||
|       get_func_ = [](ObSortColumnIdArray &sort_array, const int64_t column_id, int64_t &input_array_idx) -> int { |       get_func_ = get_func_from_array; | ||||||
|         int ret = OB_SUCCESS; |  | ||||||
|         const int64_t array_idx = std::lower_bound( |  | ||||||
|             sort_array.array_.begin(), |  | ||||||
|             sort_array.array_.end(), |  | ||||||
|             ObColumnIdToIdx(column_id)) - sort_array.array_.begin(); |  | ||||||
|         if ((sort_array.array_.count() != array_idx) |  | ||||||
|             && (sort_array.array_[array_idx].column_id_ == column_id)) { |  | ||||||
|           input_array_idx = sort_array.array_[array_idx].idx_; |  | ||||||
|         } else { |  | ||||||
|           ret = OB_ENTRY_NOT_EXIST; |  | ||||||
|         } |  | ||||||
|         return ret; |  | ||||||
|       }; |  | ||||||
|     } |     } | ||||||
|   } |   } | ||||||
|   return ret; |   return ret; | ||||||
|  | |||||||
| @ -83,6 +83,8 @@ public: | |||||||
| private: | private: | ||||||
|   int build_hash_map(const uint64_t tenant_id, const ObIArray<share::schema::ObColDesc> &column_descs); |   int build_hash_map(const uint64_t tenant_id, const ObIArray<share::schema::ObColDesc> &column_descs); | ||||||
|   int build_sort_array(const ObIArray<share::schema::ObColDesc> &column_descs); |   int build_sort_array(const ObIArray<share::schema::ObColDesc> &column_descs); | ||||||
|  |   static int32_t get_func_from_array(ObSortColumnIdArray &sort_array, const int64_t column_id, int64_t &input_array_idx); | ||||||
|  |   static int32_t get_func_from_map(ObSortColumnIdArray &sort_array, const int64_t column_id, int64_t &input_array_idx); | ||||||
|   typedef int32_t (*GET_FUNC)(ObSortColumnIdArray&, const int64_t, int64_t&); |   typedef int32_t (*GET_FUNC)(ObSortColumnIdArray&, const int64_t, int64_t&); | ||||||
|   typedef hash::ObHashMap<int64_t, int64_t> ColIdToIdxMap; |   typedef hash::ObHashMap<int64_t, int64_t> ColIdToIdxMap; | ||||||
|   typedef common::ObSEArray<ObColumnIdToIdx, share::ObTabletReplicaReportColumnMeta::DEFAULT_COLUMN_CNT> ColIdToIdxArray; |   typedef common::ObSEArray<ObColumnIdToIdx, share::ObTabletReplicaReportColumnMeta::DEFAULT_COLUMN_CNT> ColIdToIdxArray; | ||||||
|  | |||||||
| @ -21,8 +21,7 @@ namespace compaction | |||||||
|  |  | ||||||
| ObCompactionScheduleIterator::ObCompactionScheduleIterator( | ObCompactionScheduleIterator::ObCompactionScheduleIterator( | ||||||
|     const bool is_major, |     const bool is_major, | ||||||
|     ObLSGetMod mod, |     ObLSGetMod mod) | ||||||
|     const int64_t batch_tablet_cnt) |  | ||||||
|     : mod_(mod), |     : mod_(mod), | ||||||
|       is_major_(is_major), |       is_major_(is_major), | ||||||
|       scan_finish_(false), |       scan_finish_(false), | ||||||
| @ -31,7 +30,7 @@ ObCompactionScheduleIterator::ObCompactionScheduleIterator( | |||||||
|       ls_idx_(-1), |       ls_idx_(-1), | ||||||
|       tablet_idx_(0), |       tablet_idx_(0), | ||||||
|       schedule_tablet_cnt_(0), |       schedule_tablet_cnt_(0), | ||||||
|       max_batch_tablet_cnt_(batch_tablet_cnt), |       max_batch_tablet_cnt_(0), | ||||||
|       ls_tablet_svr_(nullptr), |       ls_tablet_svr_(nullptr), | ||||||
|       ls_ids_(), |       ls_ids_(), | ||||||
|       tablet_ids_() |       tablet_ids_() | ||||||
|  | |||||||
| @ -33,11 +33,9 @@ namespace compaction | |||||||
| class ObCompactionScheduleIterator | class ObCompactionScheduleIterator | ||||||
| { | { | ||||||
| public: | public: | ||||||
|   static const int64_t SCHEDULE_TABLET_BATCH_CNT = 50 * 1000L; // 5w |  | ||||||
|   ObCompactionScheduleIterator( |   ObCompactionScheduleIterator( | ||||||
|     const bool is_major, |     const bool is_major, | ||||||
|     storage::ObLSGetMod mod = storage::ObLSGetMod::STORAGE_MOD, |     storage::ObLSGetMod mod = storage::ObLSGetMod::STORAGE_MOD); | ||||||
|     const int64_t batch_tablet_cnt = SCHEDULE_TABLET_BATCH_CNT); |  | ||||||
|   ~ObCompactionScheduleIterator() { reset(); } |   ~ObCompactionScheduleIterator() { reset(); } | ||||||
|   int build_iter(const int64_t schedule_batch_size); |   int build_iter(const int64_t schedule_batch_size); | ||||||
|   int get_next_ls(ObLSHandle &ls_handle); |   int get_next_ls(ObLSHandle &ls_handle); | ||||||
|  | |||||||
| @ -68,8 +68,6 @@ int64_t ObBatchFinishCheckStat::to_string(char *buf, const int64_t buf_len) cons | |||||||
| /* | /* | ||||||
|  * ObTenantMediumChecker implement |  * ObTenantMediumChecker implement | ||||||
|  * */ |  * */ | ||||||
| const int64_t ObTenantMediumChecker::MAX_BATCH_CHECK_NUM; |  | ||||||
|  |  | ||||||
| int ObTenantMediumChecker::mtl_init(ObTenantMediumChecker *&tablet_medium_checker) | int ObTenantMediumChecker::mtl_init(ObTenantMediumChecker *&tablet_medium_checker) | ||||||
| { | { | ||||||
|   return tablet_medium_checker->init(); |   return tablet_medium_checker->init(); | ||||||
| @ -254,15 +252,17 @@ int ObTenantMediumChecker::check_medium_finish_schedule() | |||||||
|         tablet_ls_set_.clear(); |         tablet_ls_set_.clear(); | ||||||
|       } |       } | ||||||
|     } |     } | ||||||
|     if (FAILEDx(batch_tablet_ls_infos.reserve(MAX_BATCH_CHECK_NUM))) { |     const int64_t batch_size = MTL(ObTenantTabletScheduler *)->get_schedule_batch_size(); | ||||||
|       LOG_WARN("fail to reserve array", K(ret), "size", MAX_BATCH_CHECK_NUM); |     if (OB_FAIL(ret) || tablet_ls_infos.empty()) { | ||||||
|     } else if (OB_FAIL(finish_tablet_ls_infos.reserve(MAX_BATCH_CHECK_NUM))) { |     } else if (OB_FAIL(batch_tablet_ls_infos.reserve(batch_size))) { | ||||||
|       LOG_WARN("fail to reserve array", K(ret), "size", MAX_BATCH_CHECK_NUM); |       LOG_WARN("fail to reserve array", K(ret), "size", batch_size); | ||||||
|  |     } else if (OB_FAIL(finish_tablet_ls_infos.reserve(batch_size))) { | ||||||
|  |       LOG_WARN("fail to reserve array", K(ret), "size", batch_size); | ||||||
|     } else { |     } else { | ||||||
|       // batch check |       // batch check | ||||||
|       int64_t info_count = tablet_ls_infos.count(); |       int64_t info_count = tablet_ls_infos.count(); | ||||||
|       int64_t start_idx = 0; |       int64_t start_idx = 0; | ||||||
|       int64_t end_idx = min(MAX_BATCH_CHECK_NUM, info_count); |       int64_t end_idx = min(batch_size, info_count); | ||||||
|       int64_t cost_ts = ObTimeUtility::fast_current_time(); |       int64_t cost_ts = ObTimeUtility::fast_current_time(); | ||||||
|       ObBatchFinishCheckStat stat; |       ObBatchFinishCheckStat stat; | ||||||
|       while (start_idx < end_idx) { |       while (start_idx < end_idx) { | ||||||
| @ -272,7 +272,7 @@ int ObTenantMediumChecker::check_medium_finish_schedule() | |||||||
|           LOG_INFO("success to batch check medium finish", K(start_idx), K(end_idx), K(info_count)); |           LOG_INFO("success to batch check medium finish", K(start_idx), K(end_idx), K(info_count)); | ||||||
|         } |         } | ||||||
|         start_idx = end_idx; |         start_idx = end_idx; | ||||||
|         end_idx = min(start_idx + MAX_BATCH_CHECK_NUM, info_count); |         end_idx = min(start_idx + batch_size, info_count); | ||||||
|       } |       } | ||||||
|       cost_ts = ObTimeUtility::fast_current_time() - cost_ts; |       cost_ts = ObTimeUtility::fast_current_time() - cost_ts; | ||||||
|       ADD_COMPACTION_EVENT( |       ADD_COMPACTION_EVENT( | ||||||
|  | |||||||
| @ -216,11 +216,11 @@ ObTenantTabletScheduler::ObTenantTabletScheduler() | |||||||
|    minor_ls_tablet_iter_(false/*is_major*/), |    minor_ls_tablet_iter_(false/*is_major*/), | ||||||
|    medium_ls_tablet_iter_(true/*is_major*/), |    medium_ls_tablet_iter_(true/*is_major*/), | ||||||
|    gc_sst_tablet_iter_(false/*is_major*/), |    gc_sst_tablet_iter_(false/*is_major*/), | ||||||
|    schedule_tablet_batch_size_(0), |  | ||||||
|    error_tablet_cnt_(0), |    error_tablet_cnt_(0), | ||||||
|    loop_cnt_(0), |    loop_cnt_(0), | ||||||
|    prohibit_medium_map_(), |    prohibit_medium_map_(), | ||||||
|    timer_task_mgr_() |    timer_task_mgr_(), | ||||||
|  |    batch_size_mgr_() | ||||||
| { | { | ||||||
|   STATIC_ASSERT(static_cast<int64_t>(NO_MAJOR_MERGE_TYPE_CNT) == ARRAYSIZEOF(MERGE_TYPES), "merge type array len is mismatch"); |   STATIC_ASSERT(static_cast<int64_t>(NO_MAJOR_MERGE_TYPE_CNT) == ARRAYSIZEOF(MERGE_TYPES), "merge type array len is mismatch"); | ||||||
| } | } | ||||||
| @ -259,7 +259,7 @@ int ObTenantTabletScheduler::init() | |||||||
| { | { | ||||||
|   int ret = OB_SUCCESS; |   int ret = OB_SUCCESS; | ||||||
|   int64_t schedule_interval = ObTenantTabletSchedulerTaskMgr::DEFAULT_COMPACTION_SCHEDULE_INTERVAL; |   int64_t schedule_interval = ObTenantTabletSchedulerTaskMgr::DEFAULT_COMPACTION_SCHEDULE_INTERVAL; | ||||||
|   int64_t schedule_batch_size = ObCompactionScheduleIterator::SCHEDULE_TABLET_BATCH_CNT; |   int64_t schedule_batch_size = ObScheduleBatchSizeMgr::DEFAULT_TABLET_BATCH_CNT; | ||||||
|  |  | ||||||
|   { |   { | ||||||
|     omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); |     omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); | ||||||
| @ -289,7 +289,7 @@ int ObTenantTabletScheduler::init() | |||||||
|     LOG_WARN("Fail to create prohibit medium ls id map", K(ret)); |     LOG_WARN("Fail to create prohibit medium ls id map", K(ret)); | ||||||
|   } else { |   } else { | ||||||
|     timer_task_mgr_.set_scheduler_interval(schedule_interval); |     timer_task_mgr_.set_scheduler_interval(schedule_interval); | ||||||
|     schedule_tablet_batch_size_ = schedule_batch_size; |     batch_size_mgr_.set_tablet_batch_size(schedule_batch_size); | ||||||
|     is_inited_ = true; |     is_inited_ = true; | ||||||
|   } |   } | ||||||
|   return ret; |   return ret; | ||||||
| @ -328,7 +328,7 @@ int ObTenantTabletScheduler::reload_tenant_config() | |||||||
|       LOG_INFO("cache min data version", "old_data_version", cached_data_version, "new_data_version", compat_version); |       LOG_INFO("cache min data version", "old_data_version", cached_data_version, "new_data_version", compat_version); | ||||||
|     } |     } | ||||||
|     int64_t merge_schedule_interval = ObTenantTabletSchedulerTaskMgr::DEFAULT_COMPACTION_SCHEDULE_INTERVAL; |     int64_t merge_schedule_interval = ObTenantTabletSchedulerTaskMgr::DEFAULT_COMPACTION_SCHEDULE_INTERVAL; | ||||||
|     int64_t schedule_batch_size = ObCompactionScheduleIterator::SCHEDULE_TABLET_BATCH_CNT; |     int64_t schedule_batch_size = ObScheduleBatchSizeMgr::DEFAULT_TABLET_BATCH_CNT; | ||||||
|     { |     { | ||||||
|       omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); |       omt::ObTenantConfigGuard tenant_config(TENANT_CONF(MTL_ID())); | ||||||
|       if (tenant_config.is_valid()) { |       if (tenant_config.is_valid()) { | ||||||
| @ -341,9 +341,8 @@ int ObTenantTabletScheduler::reload_tenant_config() | |||||||
|     } // end of ObTenantConfigGuard |     } // end of ObTenantConfigGuard | ||||||
|     if (OB_FAIL(timer_task_mgr_.restart_scheduler_timer_task(merge_schedule_interval))) { |     if (OB_FAIL(timer_task_mgr_.restart_scheduler_timer_task(merge_schedule_interval))) { | ||||||
|       LOG_WARN("failed to restart scheduler timer", K(ret)); |       LOG_WARN("failed to restart scheduler timer", K(ret)); | ||||||
|     } else if (schedule_tablet_batch_size_ != schedule_batch_size) { |     } else { | ||||||
|       schedule_tablet_batch_size_ = schedule_batch_size; |       batch_size_mgr_.set_tablet_batch_size(schedule_batch_size); | ||||||
|       LOG_INFO("succeeded to reload new merge schedule tablet batch cnt", K(schedule_tablet_batch_size_)); |  | ||||||
|     } |     } | ||||||
|   } |   } | ||||||
|   return ret; |   return ret; | ||||||
| @ -394,7 +393,7 @@ int ObTenantTabletScheduler::update_upper_trans_version_and_gc_sstable() | |||||||
|   if (IS_NOT_INIT) { |   if (IS_NOT_INIT) { | ||||||
|     ret = OB_NOT_INIT; |     ret = OB_NOT_INIT; | ||||||
|     LOG_WARN("ObTenantTabletScheduler not init", K(ret)); |     LOG_WARN("ObTenantTabletScheduler not init", K(ret)); | ||||||
|   } else if (OB_FAIL(gc_sst_tablet_iter_.build_iter(schedule_tablet_batch_size_))) { |   } else if (OB_FAIL(gc_sst_tablet_iter_.build_iter(get_schedule_batch_size()))) { | ||||||
|     LOG_WARN("failed to init iterator", K(ret)); |     LOG_WARN("failed to init iterator", K(ret)); | ||||||
|   } |   } | ||||||
|  |  | ||||||
| @ -429,7 +428,7 @@ int ObTenantTabletScheduler::schedule_all_tablets_minor() | |||||||
|   if (IS_NOT_INIT) { |   if (IS_NOT_INIT) { | ||||||
|     ret = OB_NOT_INIT; |     ret = OB_NOT_INIT; | ||||||
|     LOG_WARN("The ObTenantTabletScheduler has not been inited", K(ret)); |     LOG_WARN("The ObTenantTabletScheduler has not been inited", K(ret)); | ||||||
|   } else if (OB_FAIL(minor_ls_tablet_iter_.build_iter(schedule_tablet_batch_size_))) { |   } else if (OB_FAIL(minor_ls_tablet_iter_.build_iter(get_schedule_batch_size()))) { | ||||||
|     LOG_WARN("failed to init iterator", K(ret)); |     LOG_WARN("failed to init iterator", K(ret)); | ||||||
|   } else { |   } else { | ||||||
|     LOG_INFO("start schedule all tablet minor merge", K(minor_ls_tablet_iter_)); |     LOG_INFO("start schedule all tablet minor merge", K(minor_ls_tablet_iter_)); | ||||||
| @ -1641,7 +1640,7 @@ int ObTenantTabletScheduler::schedule_all_tablets_medium() | |||||||
|         LOG_WARN("failed to add suspect info", K(tmp_ret)); |         LOG_WARN("failed to add suspect info", K(tmp_ret)); | ||||||
|       } |       } | ||||||
|     } |     } | ||||||
|   } else if (OB_FAIL(medium_ls_tablet_iter_.build_iter(schedule_tablet_batch_size_))) { |   } else if (OB_FAIL(medium_ls_tablet_iter_.build_iter(get_schedule_batch_size()))) { | ||||||
|     LOG_WARN("failed to init ls iterator", K(ret)); |     LOG_WARN("failed to init ls iterator", K(ret)); | ||||||
|   } else { |   } else { | ||||||
|     bool all_ls_weak_read_ts_ready = true; |     bool all_ls_weak_read_ts_ready = true; | ||||||
|  | |||||||
| @ -23,6 +23,7 @@ | |||||||
| #include "lib/hash/ob_hashset.h" | #include "lib/hash/ob_hashset.h" | ||||||
| #include "storage/compaction/ob_tenant_tablet_scheduler_task_mgr.h" | #include "storage/compaction/ob_tenant_tablet_scheduler_task_mgr.h" | ||||||
| #include "storage/compaction/ob_compaction_schedule_iterator.h" | #include "storage/compaction/ob_compaction_schedule_iterator.h" | ||||||
|  | #include "share/compaction/ob_schedule_batch_size_mgr.h" | ||||||
|  |  | ||||||
| namespace oceanbase | namespace oceanbase | ||||||
| { | { | ||||||
| @ -277,6 +278,7 @@ public: | |||||||
|   int schedule_next_round_for_leader( |   int schedule_next_round_for_leader( | ||||||
|     const ObIArray<compaction::ObTabletCheckInfo> &tablet_ls_infos, |     const ObIArray<compaction::ObTabletCheckInfo> &tablet_ls_infos, | ||||||
|     const ObIArray<compaction::ObTabletCheckInfo> &finish_tablet_ls_infos); |     const ObIArray<compaction::ObTabletCheckInfo> &finish_tablet_ls_infos); | ||||||
|  |   OB_INLINE int64_t get_schedule_batch_size() const { return batch_size_mgr_.get_schedule_batch_size(); } | ||||||
| private: | private: | ||||||
|   friend struct ObTenantTabletSchedulerTaskMgr; |   friend struct ObTenantTabletSchedulerTaskMgr; | ||||||
|   int schedule_next_medium_for_leader( |   int schedule_next_medium_for_leader( | ||||||
| @ -363,11 +365,11 @@ private: | |||||||
|   ObCompactionScheduleIterator minor_ls_tablet_iter_; |   ObCompactionScheduleIterator minor_ls_tablet_iter_; | ||||||
|   ObCompactionScheduleIterator medium_ls_tablet_iter_; |   ObCompactionScheduleIterator medium_ls_tablet_iter_; | ||||||
|   ObCompactionScheduleIterator gc_sst_tablet_iter_; |   ObCompactionScheduleIterator gc_sst_tablet_iter_; | ||||||
|   int64_t schedule_tablet_batch_size_; |  | ||||||
|   int64_t error_tablet_cnt_; // for diagnose |   int64_t error_tablet_cnt_; // for diagnose | ||||||
|   int64_t loop_cnt_; |   int64_t loop_cnt_; | ||||||
|   ObProhibitScheduleMediumMap prohibit_medium_map_; |   ObProhibitScheduleMediumMap prohibit_medium_map_; | ||||||
|   ObTenantTabletSchedulerTaskMgr timer_task_mgr_; |   ObTenantTabletSchedulerTaskMgr timer_task_mgr_; | ||||||
|  |   ObScheduleBatchSizeMgr batch_size_mgr_; | ||||||
| }; | }; | ||||||
|  |  | ||||||
| } // namespace compaction | } // namespace compaction | ||||||
|  | |||||||
| @ -166,8 +166,8 @@ int ObLSReservedSnapshotMgr::update_min_reserved_snapshot_for_leader(const int64 | |||||||
|     if (0 == dependent_tablet_set_.size()) { |     if (0 == dependent_tablet_set_.size()) { | ||||||
|       if (new_snapshot_version < min_reserved_snapshot_) { |       if (new_snapshot_version < min_reserved_snapshot_) { | ||||||
|         ret = OB_INVALID_ARGUMENT; |         ret = OB_INVALID_ARGUMENT; | ||||||
|         LOG_WARN("failed to update min reserved snapshot", K(ret), K(new_snapshot_version), |         LOG_WARN("failed to update min reserved snapshot", K(ret), "ls_id", ls_->get_ls_id(), | ||||||
|             K(min_reserved_snapshot_)); |           K(new_snapshot_version), K(min_reserved_snapshot_)); | ||||||
|       } else if (new_snapshot_version > min_reserved_snapshot_) { |       } else if (new_snapshot_version > min_reserved_snapshot_) { | ||||||
|         // update min_reserved_snapshot and send clog |         // update min_reserved_snapshot and send clog | ||||||
|         min_reserved_snapshot_ = new_snapshot_version; |         min_reserved_snapshot_ = new_snapshot_version; | ||||||
| @ -204,7 +204,7 @@ int ObLSReservedSnapshotMgr::try_sync_reserved_snapshot( | |||||||
|     LOG_WARN("invalid argument", K(ret), K(new_reserved_snapshot)); |     LOG_WARN("invalid argument", K(ret), K(new_reserved_snapshot)); | ||||||
|   } else if (update_flag) { |   } else if (update_flag) { | ||||||
|     if (OB_FAIL(update_min_reserved_snapshot_for_leader(new_reserved_snapshot))) { |     if (OB_FAIL(update_min_reserved_snapshot_for_leader(new_reserved_snapshot))) { | ||||||
|       LOG_WARN("failed to update min_reserved_snapshot", K(ret), K(new_reserved_snapshot)); |       LOG_WARN("failed to update min_reserved_snapshot", K(ret), "ls_id", ls_->get_ls_id(), K(new_reserved_snapshot)); | ||||||
|     } |     } | ||||||
|   } else if (OB_FAIL(sync_clog(new_reserved_snapshot))) { |   } else if (OB_FAIL(sync_clog(new_reserved_snapshot))) { | ||||||
|     LOG_WARN("failed to send update reserved snapshot log", K(ret), K(new_reserved_snapshot)); |     LOG_WARN("failed to send update reserved snapshot log", K(ret), K(new_reserved_snapshot)); | ||||||
| @ -226,7 +226,7 @@ int ObLSReservedSnapshotMgr::sync_clog(const int64_t new_reserved_snapshot) | |||||||
|   } else { |   } else { | ||||||
|     ObMutexGuard guard(sync_clog_lock_); |     ObMutexGuard guard(sync_clog_lock_); | ||||||
|     if (OB_FAIL(try_update_for_leader(new_reserved_snapshot, nullptr/*allocator*/))) { |     if (OB_FAIL(try_update_for_leader(new_reserved_snapshot, nullptr/*allocator*/))) { | ||||||
|       LOG_WARN("failed to send update reserved snapshot log", K(ret), K(new_reserved_snapshot)); |       LOG_WARN("failed to send update reserved snapshot log", K(ret), "ls_id", ls_->get_ls_id(), K(new_reserved_snapshot)); | ||||||
|     } |     } | ||||||
|   } |   } | ||||||
|   return ret; |   return ret; | ||||||
|  | |||||||
| @ -4432,21 +4432,25 @@ int ObTablet::get_kept_snapshot_info( | |||||||
|     ObTabletMediumInfoReader medium_info_reader(*this); |     ObTabletMediumInfoReader medium_info_reader(*this); | ||||||
|     if (OB_FAIL(medium_info_reader.init(arena_allocator))) { |     if (OB_FAIL(medium_info_reader.init(arena_allocator))) { | ||||||
|       LOG_WARN("failed to init medium info reader", K(ret)); |       LOG_WARN("failed to init medium info reader", K(ret)); | ||||||
|     } else if (OB_FAIL(medium_info_reader.get_min_medium_snapshot(min_medium_snapshot))) { |     } else if (OB_FAIL(medium_info_reader.get_min_medium_snapshot(max_merged_snapshot, min_medium_snapshot))) { | ||||||
|       LOG_WARN("failed to get min medium snapshot", K(ret), K(tablet_id)); |       LOG_WARN("failed to get min medium snapshot", K(ret), K(tablet_id)); | ||||||
|     } |     } | ||||||
|   } |   } | ||||||
|  |  | ||||||
|   // for compat, if receive ls_reserved_snapshot clog, should consider ls.get_min_reserved_snapshot() |  | ||||||
|   if (min_reserved_snapshot_on_ls > 0) { |  | ||||||
|     ls_min_reserved_snapshot = min_reserved_snapshot_on_ls; |  | ||||||
|   } |  | ||||||
|   if (OB_SUCC(ret)) { |   if (OB_SUCC(ret)) { | ||||||
|  |     bool use_multi_version_start_on_tablet = false; | ||||||
|     const int64_t old_min_reserved_snapshot = min_reserved_snapshot; |     const int64_t old_min_reserved_snapshot = min_reserved_snapshot; | ||||||
|     snapshot_info.update_by_smaller_snapshot(ObStorageSnapshotInfo::SNAPSHOT_FOR_LS_RESERVED, ls_min_reserved_snapshot); |     if (min_reserved_snapshot_on_ls > 0) { | ||||||
|     snapshot_info.update_by_smaller_snapshot(ObStorageSnapshotInfo::SNAPSHOT_FOR_MIN_MEDIUM, min_medium_snapshot); |       snapshot_info.update_by_smaller_snapshot(ObStorageSnapshotInfo::SNAPSHOT_FOR_LS_RESERVED, ls_min_reserved_snapshot); | ||||||
|     if (snapshot_info.snapshot_ < get_multi_version_start()) { |       snapshot_info.update_by_smaller_snapshot(ObStorageSnapshotInfo::SNAPSHOT_FOR_MIN_MEDIUM, min_medium_snapshot); | ||||||
|       // tablet already had large multi_version_start |       if (snapshot_info.snapshot_ < get_multi_version_start()) { | ||||||
|  |         use_multi_version_start_on_tablet = true; | ||||||
|  |       } | ||||||
|  |     } else { | ||||||
|  |       // if not sync ls_reserved_snapshot yet, should use multi_version_start on tablet | ||||||
|  |       use_multi_version_start_on_tablet = true; | ||||||
|  |     } | ||||||
|  |     if (use_multi_version_start_on_tablet) { | ||||||
|       snapshot_info.snapshot_type_ = ObStorageSnapshotInfo::SNAPSHOT_MULTI_VERSION_START_ON_TABLET; |       snapshot_info.snapshot_type_ = ObStorageSnapshotInfo::SNAPSHOT_MULTI_VERSION_START_ON_TABLET; | ||||||
|       snapshot_info.snapshot_ = get_multi_version_start(); |       snapshot_info.snapshot_ = get_multi_version_start(); | ||||||
|     } |     } | ||||||
| @ -4467,7 +4471,7 @@ int ObTablet::get_kept_snapshot_info( | |||||||
|     ret = OB_ERR_UNEXPECTED; |     ret = OB_ERR_UNEXPECTED; | ||||||
|     LOG_WARN("snapshot info is invalid", KR(ret), K(snapshot_info)); |     LOG_WARN("snapshot info is invalid", KR(ret), K(snapshot_info)); | ||||||
|   } |   } | ||||||
|   LOG_DEBUG("get multi version start", "ls_id", get_tablet_meta().ls_id_, K(tablet_id), |   LOG_TRACE("get multi version start", "ls_id", get_tablet_meta().ls_id_, K(tablet_id), | ||||||
|       K(snapshot_info), K(min_reserved_snapshot), K(get_tablet_meta()), |       K(snapshot_info), K(min_reserved_snapshot), K(get_tablet_meta()), | ||||||
|       K(min_medium_snapshot), K(min_reserved_snapshot_on_ls), K(max_merged_snapshot)); |       K(min_medium_snapshot), K(min_reserved_snapshot_on_ls), K(max_merged_snapshot)); | ||||||
|   return ret; |   return ret; | ||||||
|  | |||||||
| @ -210,7 +210,9 @@ int ObTabletMediumInfoReader::get_specified_medium_info( | |||||||
| } | } | ||||||
|  |  | ||||||
| // temp solution, TODO(@xianzhi) | // temp solution, TODO(@xianzhi) | ||||||
| int ObTabletMediumInfoReader::get_min_medium_snapshot(int64_t &min_medium_snapshot) | int ObTabletMediumInfoReader::get_min_medium_snapshot( | ||||||
|  |   const int64_t last_major_snapshot_version, | ||||||
|  |   int64_t &min_medium_snapshot) | ||||||
| { | { | ||||||
|   int ret = OB_SUCCESS; |   int ret = OB_SUCCESS; | ||||||
|   ObArenaAllocator tmp_allocator; |   ObArenaAllocator tmp_allocator; | ||||||
| @ -225,7 +227,7 @@ int ObTabletMediumInfoReader::get_min_medium_snapshot(int64_t &min_medium_snapsh | |||||||
|       } else { |       } else { | ||||||
|         LOG_WARN("failed to get medium info", K(ret)); |         LOG_WARN("failed to get medium info", K(ret)); | ||||||
|       } |       } | ||||||
|     } else { |     } else if (tmp_key.get_medium_snapshot() > last_major_snapshot_version) { | ||||||
|       min_medium_snapshot = tmp_key.get_medium_snapshot(); |       min_medium_snapshot = tmp_key.get_medium_snapshot(); | ||||||
|       break; |       break; | ||||||
|     } |     } | ||||||
|  | |||||||
| @ -42,7 +42,9 @@ public: | |||||||
|       const compaction::ObMediumCompactionInfoKey &key, |       const compaction::ObMediumCompactionInfoKey &key, | ||||||
|       compaction::ObMediumCompactionInfo &medium_info); |       compaction::ObMediumCompactionInfo &medium_info); | ||||||
|  |  | ||||||
|   int get_min_medium_snapshot(int64_t &min_medium_snapshot); |   int get_min_medium_snapshot( | ||||||
|  |     const int64_t last_major_snapshot_version, | ||||||
|  |     int64_t &min_medium_snapshot); | ||||||
|   int get_max_medium_snapshot(int64_t &max_medium_snapshot); |   int get_max_medium_snapshot(int64_t &max_medium_snapshot); | ||||||
| private: | private: | ||||||
|   int advance_mds_iter(); |   int advance_mds_iter(); | ||||||
|  | |||||||
| @ -35,12 +35,13 @@ public: | |||||||
|   MockObCompactionScheduleIterator(const int64_t batch_tablet_cnt) |   MockObCompactionScheduleIterator(const int64_t batch_tablet_cnt) | ||||||
|     : ObCompactionScheduleIterator( |     : ObCompactionScheduleIterator( | ||||||
|       true/*is_major, no meaning*/, |       true/*is_major, no meaning*/, | ||||||
|       ObLSGetMod::STORAGE_MOD, |       ObLSGetMod::STORAGE_MOD), | ||||||
|       batch_tablet_cnt), |  | ||||||
|       mock_tablet_id_cnt_(0), |       mock_tablet_id_cnt_(0), | ||||||
|       error_tablet_idx_(-1), |       error_tablet_idx_(-1), | ||||||
|       errno_(OB_SUCCESS) |       errno_(OB_SUCCESS) | ||||||
|   {} |   { | ||||||
|  |     max_batch_tablet_cnt_ = batch_tablet_cnt; | ||||||
|  |   } | ||||||
|   virtual int get_cur_ls_handle(ObLSHandle &ls_handle) override |   virtual int get_cur_ls_handle(ObLSHandle &ls_handle) override | ||||||
|   { |   { | ||||||
|     return OB_SUCCESS; |     return OB_SUCCESS; | ||||||
|  | |||||||
		Reference in New Issue
	
	Block a user
	 yangqise7en
					yangqise7en