From f18e9987e96996d17c64f287c6a505c7089f6481 Mon Sep 17 00:00:00 2001 From: discord9 Date: Fri, 7 Nov 2025 15:40:48 +0800 Subject: [PATCH 01/10] feat: gc worker only on local region Signed-off-by: discord9 feat: gc scheduler wip: gc trigger Signed-off-by: discord9 feat: dn file removal rate Signed-off-by: discord9 feat: trigger gc with stats(WIP) Signed-off-by: discord9 chore Signed-off-by: discord9 also move files ref manifest to store-api Signed-off-by: discord9 feat: basic gc trigger impl Signed-off-by: discord9 wip: handle file ref change Signed-off-by: discord9 refactor: use region ids Signed-off-by: discord9 fix: retry using related regions Signed-off-by: discord9 chore: rm unused Signed-off-by: discord9 fix: update file reference type in GC worker Signed-off-by: discord9 feat: dn gc limiter Signed-off-by: discord9 rename Signed-off-by: discord9 feat: gc scheduler retry with outdated regions Signed-off-by: discord9 feat: use real object store purger Signed-off-by: discord9 wip: add to metasrv Signed-off-by: discord9 feat: add to metasrv Signed-off-by: discord9 feat: datanode gc worker handler Signed-off-by: discord9 fix: no partition col fix Signed-off-by: discord9 fix: RegionId json deser workaround Signed-off-by: discord9 fix: find access layer Signed-off-by: discord9 fix: on host dn Signed-off-by: discord9 fix: stat dedup Signed-off-by: discord9 refactor: rm load-based Signed-off-by: discord9 chore: aft rebase fix Signed-off-by: discord9 feat: not full scan Signed-off-by: discord9 chore: after rebase fix Signed-off-by: discord9 feat: clean tracker Signed-off-by: discord9 after rebase fix Signed-off-by: discord9 clippy Signed-off-by: discord9 refactor: split gc scheduler Signed-off-by: discord9 feat: smaller linger time Signed-off-by: discord9 feat: parallel region gc instr Signed-off-by: discord9 chore: rename Signed-off-by: discord9 chore: rename Signed-off-by: discord9 enable is false Signed-off-by: discord9 feat: update removed files precisely Signed-off-by: discord9 all default to false&use local file purger Signed-off-by: discord9 feat: not evict if gc enabled Signed-off-by: discord9 per review Signed-off-by: discord9 fix: pass gc config in mito&test: after truncate gc Signed-off-by: discord9 WIP: one more test Signed-off-by: discord9 test: basic compact Signed-off-by: discord9 test: compact with ref Signed-off-by: discord9 refactor: for easier mock Signed-off-by: discord9 docs: explain race condition Signed-off-by: discord9 feat: gc region procedure Signed-off-by: discord9 refactor: ctx send gc/ref instr with procedure Signed-off-by: discord9 fix: config deser to default Signed-off-by: discord9 refactor: gc report Signed-off-by: discord9 wip: async index file rm Signed-off-by: discord9 fixme? Signed-off-by: discord9 typo Signed-off-by: discord9 more ut Signed-off-by: discord9 test: more mock test Signed-off-by: discord9 more Signed-off-by: discord9 refactor: split mock test Signed-off-by: discord9 clippy Signed-off-by: discord9 refactor: rm stuff Signed-off-by: discord9 test: mock add gc report per region Signed-off-by: discord9 fix: stricter table failure condition Signed-off-by: discord9 sutff Signed-off-by: discord9 feat: can do different table gc same time&more todos Signed-off-by: discord9 after rebase check Signed-off-by: discord9 --- src/meta-srv/src/gc.rs | 9 + src/meta-srv/src/gc/candidate.rs | 85 ++++ src/meta-srv/src/gc/ctx.rs | 7 +- src/meta-srv/src/gc/handler.rs | 409 ++++++++++++++++ src/meta-srv/src/gc/mock.rs | 380 +++++++++++++++ src/meta-srv/src/gc/mock/basic.rs | 169 +++++++ src/meta-srv/src/gc/mock/candidate_select.rs | 370 +++++++++++++++ src/meta-srv/src/gc/mock/con.rs | 461 +++++++++++++++++++ src/meta-srv/src/gc/mock/config.rs | 196 ++++++++ src/meta-srv/src/gc/mock/err_handle.rs | 241 ++++++++++ src/meta-srv/src/gc/mock/full_list.rs | 273 +++++++++++ src/meta-srv/src/gc/mock/integration.rs | 225 +++++++++ src/meta-srv/src/gc/mock/misc.rs | 135 ++++++ src/meta-srv/src/gc/mock/retry.rs | 240 ++++++++++ src/meta-srv/src/gc/options.rs | 168 +++++++ src/meta-srv/src/gc/scheduler.rs | 197 ++++++++ src/meta-srv/src/gc/tracker.rs | 106 +++++ src/meta-srv/src/metasrv.rs | 8 + src/meta-srv/src/metasrv/builder.rs | 18 + src/mito2/src/gc.rs | 4 + src/mito2/src/manifest/action.rs | 1 + src/mito2/src/manifest/manager.rs | 2 - src/mito2/src/sst/file_purger.rs | 2 +- src/mito2/src/sst/file_ref.rs | 6 +- 24 files changed, 3704 insertions(+), 8 deletions(-) create mode 100644 src/meta-srv/src/gc/handler.rs create mode 100644 src/meta-srv/src/gc/mock.rs create mode 100644 src/meta-srv/src/gc/mock/basic.rs create mode 100644 src/meta-srv/src/gc/mock/candidate_select.rs create mode 100644 src/meta-srv/src/gc/mock/con.rs create mode 100644 src/meta-srv/src/gc/mock/config.rs create mode 100644 src/meta-srv/src/gc/mock/err_handle.rs create mode 100644 src/meta-srv/src/gc/mock/full_list.rs create mode 100644 src/meta-srv/src/gc/mock/integration.rs create mode 100644 src/meta-srv/src/gc/mock/misc.rs create mode 100644 src/meta-srv/src/gc/mock/retry.rs create mode 100644 src/meta-srv/src/gc/options.rs create mode 100644 src/meta-srv/src/gc/scheduler.rs create mode 100644 src/meta-srv/src/gc/tracker.rs diff --git a/src/meta-srv/src/gc.rs b/src/meta-srv/src/gc.rs index f44cc084e287..95cda01a991f 100644 --- a/src/meta-srv/src/gc.rs +++ b/src/meta-srv/src/gc.rs @@ -22,7 +22,16 @@ use store_api::storage::RegionId; mod candidate; mod ctx; +mod handler; +#[cfg(test)] +mod mock; +mod options; mod procedure; +mod scheduler; +mod tracker; + +pub(crate) use options::GcSchedulerOptions; +pub(crate) use scheduler::{GcScheduler, GcTickerRef}; pub(crate) type Region2Peers = HashMap)>; diff --git a/src/meta-srv/src/gc/candidate.rs b/src/meta-srv/src/gc/candidate.rs index ac52d9f81f97..1195e13b6902 100644 --- a/src/meta-srv/src/gc/candidate.rs +++ b/src/meta-srv/src/gc/candidate.rs @@ -23,6 +23,7 @@ use store_api::storage::RegionId; use table::metadata::TableId; use crate::error::Result; +use crate::gc::scheduler::GcScheduler; /// Represents a region candidate for GC with its priority score. #[derive(Debug, Clone, PartialEq, Eq)] @@ -46,3 +47,87 @@ impl GcCandidate { self.score.into_inner() } } + +impl GcScheduler { + /// Calculate GC priority score for a region based on various metrics. + fn calculate_gc_score(&self, region_stat: &RegionStat) -> f64 { + let sst_count_score = region_stat.sst_num as f64 * self.config.sst_count_weight; + + let file_remove_cnt_score = match ®ion_stat.region_manifest { + RegionManifestInfo::Mito { + file_removed_cnt, .. + } => *file_removed_cnt as f64 * self.config.file_removed_cnt_weight, + // Metric engine doesn't have file_removal_rate, also this should be unreachable since metrics engine doesn't support gc + RegionManifestInfo::Metric { .. } => 0.0, + }; + + sst_count_score + file_remove_cnt_score + } + + /// Filter and score regions that are candidates for GC, grouped by table. + pub(crate) async fn select_gc_candidates( + &self, + table_to_region_stats: &HashMap>, + ) -> Result>> { + let mut table_candidates: HashMap> = HashMap::new(); + let now = Instant::now(); + + for (table_id, region_stats) in table_to_region_stats { + let mut candidates = Vec::new(); + + for region_stat in region_stats { + if region_stat.role != RegionRole::Leader { + continue; + } + + // Skip regions that are too small + if region_stat.approximate_bytes < self.config.min_region_size_threshold { + continue; + } + + // Skip regions that are in cooldown period + if let Some(gc_info) = self.region_gc_tracker.lock().await.get(®ion_stat.id) + && now.duration_since(gc_info.last_gc_time) < self.config.gc_cooldown_period + { + debug!("Skipping region {} due to cooldown", region_stat.id); + continue; + } + + let score = self.calculate_gc_score(region_stat); + + debug!( + "Region {} (table {}) has GC score {:.4}", + region_stat.id, table_id, score + ); + + // Only consider regions with a meaningful score + if score > 0.0 { + candidates.push(GcCandidate::new(region_stat.id, score, region_stat.clone())); + } + } + + // Sort candidates by score in descending order and take top N + candidates.sort_by(|a, b| b.score.cmp(&a.score)); + let top_candidates: Vec = candidates + .into_iter() + .take(self.config.regions_per_table_threshold) + .collect(); + + if !top_candidates.is_empty() { + info!( + "Selected {} GC candidates for table {} (top {} out of all qualified)", + top_candidates.len(), + table_id, + self.config.regions_per_table_threshold + ); + table_candidates.insert(*table_id, top_candidates); + } + } + + info!( + "Selected GC candidates for {} tables", + table_candidates.len() + ); + Ok(table_candidates) + } +} diff --git a/src/meta-srv/src/gc/ctx.rs b/src/meta-srv/src/gc/ctx.rs index 848e7f2c283f..cfeff1cc1059 100644 --- a/src/meta-srv/src/gc/ctx.rs +++ b/src/meta-srv/src/gc/ctx.rs @@ -12,8 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -// TODO(discord9): remove this once gc scheduler is fully merged -#![allow(unused)] + use std::collections::{HashMap, HashSet}; use std::time::Duration; @@ -35,6 +34,10 @@ use table::metadata::TableId; use crate::cluster::MetaPeerClientRef; use crate::error::{self, Result, TableMetadataManagerSnafu, UnexpectedSnafu}; use crate::gc::Region2Peers; + +use crate::cluster::MetaPeerClientRef; +use crate::error; +use crate::error::{Result, TableMetadataManagerSnafu}; use crate::gc::procedure::GcRegionProcedure; use crate::handler::HeartbeatMailbox; use crate::service::mailbox::{Channel, MailboxRef}; diff --git a/src/meta-srv/src/gc/handler.rs b/src/meta-srv/src/gc/handler.rs new file mode 100644 index 000000000000..39f4df44097b --- /dev/null +++ b/src/meta-srv/src/gc/handler.rs @@ -0,0 +1,409 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::Arc; +use std::sync::atomic::{AtomicUsize, Ordering}; +use std::time::Instant; + +use common_meta::peer::Peer; +use common_telemetry::{debug, error, info, warn}; +use futures::StreamExt; +use itertools::Itertools; +use snafu::OptionExt as _; +use store_api::storage::{FileRefsManifest, GcReport, RegionId}; +use table::metadata::TableId; +use tokio::time::sleep; + +use crate::error::{self, RegionRouteNotFoundSnafu, Result, UnexpectedSnafu}; +use crate::gc::candidate::GcCandidate; +use crate::gc::scheduler::{GcJobReport, GcScheduler, TableGcReport}; +use crate::gc::tracker::RegionGcInfo; + +impl GcScheduler { + /// Iterate through all region stats, find region that might need gc, and send gc instruction to + /// the corresponding datanode with improved parallel processing and retry logic. + pub(crate) async fn trigger_gc(&self) -> Result { + let start_time = Instant::now(); + info!("Starting GC cycle"); + + // Step 1: Get all region statistics + let table_to_region_stats = self.ctx.get_table_to_region_stats().await?; + info!( + "Fetched region stats for {} tables", + table_to_region_stats.len() + ); + + // Step 2: Select GC candidates based on our scoring algorithm + let per_table_candidates = self.select_gc_candidates(&table_to_region_stats).await?; + + if per_table_candidates.is_empty() { + info!("No GC candidates found, skipping GC cycle"); + return Ok(Default::default()); + } + + // Step 3: Process tables concurrently with limited parallelism + let report = self.process_tables_concurrently(per_table_candidates).await; + + let duration = start_time.elapsed(); + info!( + "Finished GC cycle. Processed {} tables ({} successful). Duration: {:?}", + report.processed_tables, + report.table_reports.len(), + duration + ); + debug!("Detailed GC Job Report: {report:#?}"); + + Ok(report) + } + + /// Process GC for a single table with all its candidate regions. + pub(crate) async fn process_table_gc( + &self, + table_id: TableId, + candidates: Vec, + ) -> Result { + info!( + "Starting GC for table {} with {} candidate regions", + table_id, + candidates.len() + ); + + let mut report = TableGcReport { + table_id, + ..Default::default() + }; + + // Step 1: Get table route information + // if is logic table, can simply pass. + let (phy_table_id, table_peer) = self.ctx.get_table_route(table_id).await?; + + if phy_table_id != table_id { + return Ok(report); + } + + let region_to_peer = table_peer + .region_routes + .iter() + .filter_map(|r| { + r.leader_peer + .as_ref() + .map(|peer| (r.region.id, peer.clone())) + }) + .collect::>(); + + // Step 2: Determine related regions for file reference fetching + let candidate_region_ids: Vec = candidates.iter().map(|c| c.region_id).collect(); + let related_region_ids = self.find_related_regions(&candidate_region_ids).await?; + + // Step 3: Get file references for related regions + let file_refs_manifest = self + .ctx + .get_file_references( + &related_region_ids, + ®ion_to_peer, + self.config.mailbox_timeout, + ) + .await?; + + // Step 4: Filter out candidates that don't have file references available + let total_candidates = candidates.len(); + let mut valid_candidates = Vec::new(); + for candidate in candidates { + // Check if we have file references for this region + if file_refs_manifest + .manifest_version + .contains_key(&candidate.region_id) + { + // Check if this peer's addr were successfully obtained + if region_to_peer.contains_key(&candidate.region_id) { + valid_candidates.push(candidate); + } else { + UnexpectedSnafu { + violated: format!("Missing peer info for region {}", candidate.region_id), + } + .fail()?; + } + } else { + error!( + "Missing file references entry for region {}", + candidate.region_id + ); + UnexpectedSnafu { + violated: format!( + "Missing file references entry for region {}", + candidate.region_id + ), + } + .fail()?; + } + } + + // Step 5: Process each valid candidate region with retry logic + let valid_candidates_count = valid_candidates.len(); + let successful_regions = Arc::new(AtomicUsize::new(0)); + let failed_regions = Arc::new(AtomicUsize::new(0)); + let reports = Arc::new(tokio::sync::Mutex::new(Vec::new())); + let err_reports = Arc::new(tokio::sync::Mutex::new(Vec::new())); + + let file_refs_manifest = Arc::new(file_refs_manifest); + let region_to_peer = Arc::new(region_to_peer); + + futures::stream::iter(valid_candidates) + .for_each_concurrent(Some(self.config.region_gc_concurrency), |candidate| { + let region_id = candidate.region_id; + let file_refs_manifest = file_refs_manifest.clone(); + let region_to_peer = region_to_peer.clone(); + let successful_regions = successful_regions.clone(); + let failed_regions = failed_regions.clone(); + let region_gc_tracker = self.region_gc_tracker.clone(); + let reports = reports.clone(); + let err_reports = err_reports.clone(); + + async move { + match self + .process_region_gc_with_retry( + candidate, + &file_refs_manifest, + ®ion_to_peer, + ) + .await + { + Ok((report, used_full_listing)) => { + reports.lock().await.push(report); + successful_regions.fetch_add(1, Ordering::Relaxed); + // Update GC tracker + let mut gc_tracker = region_gc_tracker.lock().await; + let now = Instant::now(); + let gc_info = gc_tracker + .entry(region_id) + .or_insert_with(|| RegionGcInfo::new(now)); + gc_info.last_gc_time = now; + if used_full_listing { + gc_info.last_full_listing_time = Some(now); + } + } + Err(e) => { + failed_regions.fetch_add(1, Ordering::Relaxed); + error!("Failed to GC region {} after all retries: {}", region_id, e); + // TODO: collect errors into table gc report + err_reports + .lock() + .await + .push((region_id, format!("{:#?}", e))); + } + } + } + }) + .await; + + let successful_regions = successful_regions.load(Ordering::Relaxed); + + info!( + "Completed GC for table {}: {}/{} regions successful ({} skipped due to missing file references)", + table_id, + successful_regions, + valid_candidates_count, + total_candidates - valid_candidates_count + ); + + report.success_regions = reports.lock().await.drain(..).collect(); + report.failed_regions = err_reports.lock().await.drain(..).collect(); + Ok(report) + } + + /// Find related regions that might share files with the candidate regions. + /// Currently returns the same regions since repartition is not implemented yet. + /// TODO(discord9): When repartition is implemented, this should also find parent/child regions + /// that might share files with the candidate regions. + pub(crate) async fn find_related_regions( + &self, + candidate_region_ids: &[RegionId], + ) -> Result> { + Ok(candidate_region_ids.to_vec()) + } + + /// Process GC for a single region with retry logic. + /// Returns the GC report and a boolean indicating whether full file listing was used. + pub(crate) async fn process_region_gc_with_retry( + &self, + candidate: GcCandidate, + file_refs_manifest: &FileRefsManifest, + // TODO(discord9): maybe also refresh region_to_peer mapping if needed? + region_to_peer: &HashMap, + ) -> Result<(GcReport, bool)> { + let region_id = candidate.region_id; + + // TODO(discord9): Select the best peer based on GC load + // for now gc worker need to be run from datanode that hosts the region + // this limit might be lifted in the future + let mut peer = self.get_region_peer(region_id, region_to_peer)?; + + let mut retries = 0; + let mut current_manifest = file_refs_manifest.clone(); + // Final report for recording all deleted files + let mut final_report = GcReport::default(); + + // Determine if we should use full file listing for this region + let should_use_full_listing = self.should_use_full_listing(region_id).await; + + loop { + match self + .ctx + .gc_regions( + peer.clone(), + region_id, + ¤t_manifest, + should_use_full_listing, + self.config.mailbox_timeout, + ) + .await + { + Ok(report) => { + match self + .handle_gc_success(region_id, report, &mut final_report, region_to_peer) + .await? + { + None => return Ok((final_report, should_use_full_listing)), + Some(refreshed_manifest) => { + current_manifest = refreshed_manifest; + } + } + } + // Retryable errors: refresh file references and retry with backoff + Err(e) if e.is_retryable() => { + // TODO(discord9): might do it on table level instead + let (refreshed_manifest, refreshed_peer) = self + .handle_gc_retry(region_id, &mut retries, e, region_to_peer) + .await?; + current_manifest = refreshed_manifest; + peer = refreshed_peer; + } + Err(e) => { + error!( + "Non-retryable error during GC for region {}: {}", + region_id, e + ); + return Err(e); + } + } + } + } + + /// Handle successful GC report, checking if retry is needed for outdated regions. + async fn handle_gc_success( + &self, + region_id: RegionId, + report: GcReport, + final_report: &mut GcReport, + region_to_peer: &HashMap, + ) -> Result> { + if report.need_retry_regions.is_empty() { + final_report.merge(report); + debug!( + "Successfully completed GC for region {} with report: {final_report:?}", + region_id + ); + // note that need_retry_regions should be empty here + // since no more outdated regions + final_report.need_retry_regions.clear(); + Ok(None) + } else { + // retry outdated regions if needed + let refreshed_manifest = self + .refresh_file_refs_for( + &report.need_retry_regions.clone().into_iter().collect_vec(), + region_to_peer, + ) + .await?; + info!( + "Retrying GC for regions {:?} due to outdated file references", + &report.need_retry_regions + ); + final_report.merge(report); + Ok(Some(refreshed_manifest)) + } + } + + /// Handle retryable GC error with backoff and manifest refresh. + async fn handle_gc_retry( + &self, + region_id: RegionId, + retries: &mut usize, + error: error::Error, + region_to_peer: &HashMap, + ) -> Result<(FileRefsManifest, Peer)> { + *retries += 1; + if *retries >= self.config.max_retries_per_region { + error!( + "Failed to GC region {} after {} retries: {}", + region_id, retries, error + ); + return Err(error); + } + + warn!( + "GC failed for region {} (attempt {}/{}): {}. Retrying after backoff...", + region_id, retries, self.config.max_retries_per_region, error + ); + + // Wait for backoff period + sleep(self.config.retry_backoff_duration).await; + + let refreshed_manifest = self + .refresh_file_refs_for(&[region_id], region_to_peer) + .await?; + + // TODO(discord9): Select the best peer based on GC load + // for now gc worker need to be run from datanode that hosts the region + // this limit might be lifted in the future + let peer = self.get_region_peer(region_id, region_to_peer)?; + + Ok((refreshed_manifest, peer)) + } + + /// Refresh file references for related regions, typically used before retrying GC. + async fn refresh_file_refs_for( + &self, + regions: &[RegionId], + region_to_peer: &HashMap, + ) -> Result { + let related_regions = self.find_related_regions(regions).await?; + self.ctx + .get_file_references( + &related_regions, + region_to_peer, + self.config.mailbox_timeout, + ) + .await + .inspect_err(|e| { + error!( + "Failed to refresh file references for regions {:?}: {}", + related_regions, e + ); + }) + } + + /// Get the peer for a given region. + fn get_region_peer( + &self, + region_id: RegionId, + region_to_peer: &HashMap, + ) -> Result { + region_to_peer + .get(®ion_id) + .cloned() + .with_context(|| RegionRouteNotFoundSnafu { region_id }) + } +} diff --git a/src/meta-srv/src/gc/mock.rs b/src/meta-srv/src/gc/mock.rs new file mode 100644 index 000000000000..33aa40eb0176 --- /dev/null +++ b/src/meta-srv/src/gc/mock.rs @@ -0,0 +1,380 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod basic; +mod candidate_select; +mod con; +mod config; +mod err_handle; +mod full_list; +mod integration; +mod misc; +mod retry; + +use std::collections::HashMap; +use std::sync::{Arc, Mutex}; +use std::time::{Duration, Instant}; + +use common_meta::datanode::{RegionManifestInfo, RegionStat}; +use common_meta::key::table_route::PhysicalTableRouteValue; +use common_meta::peer::Peer; +use common_meta::rpc::router::{Region, RegionRoute}; +use ordered_float::OrderedFloat; +use snafu::OptionExt; +use store_api::region_engine::RegionRole; +use store_api::storage::{FileRefsManifest, GcReport, RegionId}; +use table::metadata::TableId; +use tokio::sync::mpsc::Sender; + +use crate::error::{Result, UnexpectedSnafu}; +use crate::gc::candidate::GcCandidate; +use crate::gc::ctx::SchedulerCtx; +use crate::gc::options::GcSchedulerOptions; +use crate::gc::scheduler::{Event, GcScheduler}; + +#[allow(clippy::type_complexity)] +#[derive(Debug, Default)] +pub struct MockSchedulerCtx { + pub table_to_region_stats: Arc>>>>, + pub table_routes: Arc>>, + pub file_refs: Arc>>, + pub gc_reports: Arc>>, + pub candidates: Arc>>>>, + pub get_table_to_region_stats_calls: Arc>, + pub get_file_references_calls: Arc>, + pub gc_regions_calls: Arc>, + // Error injection fields for testing + pub get_table_to_region_stats_error: Arc>>, + pub get_table_route_error: Arc>>, + pub get_file_references_error: Arc>>, + pub gc_regions_error: Arc>>, + // Retry testing fields + pub gc_regions_retry_count: Arc>>, + pub gc_regions_error_sequence: Arc>>, + pub gc_regions_success_after_retries: Arc>>, + // Per-region error injection + pub gc_regions_per_region_errors: Arc>>, +} + +impl MockSchedulerCtx { + pub fn with_table_routes( + self, + table_routes: HashMap)>, + ) -> Self { + *self.table_routes.lock().unwrap() = table_routes + .into_iter() + .map(|(k, (phy_id, region2peer))| { + let phy = PhysicalTableRouteValue::new( + region2peer + .into_iter() + .map(|(region_id, peer)| RegionRoute { + region: Region::new_test(region_id), + leader_peer: Some(peer), + ..Default::default() + }) + .collect(), + ); + + (k, (phy_id, phy)) + }) + .collect(); + self + } + + /// Set an error to be returned by `get_table_to_region_stats` + #[allow(dead_code)] + pub fn with_get_table_to_region_stats_error(self, error: crate::error::Error) -> Self { + *self.get_table_to_region_stats_error.lock().unwrap() = Some(error); + self + } + + /// Set an error to be returned by `get_table_route` + pub fn with_get_table_route_error(self, error: crate::error::Error) -> Self { + *self.get_table_route_error.lock().unwrap() = Some(error); + self + } + + /// Set an error to be returned by `get_file_references` + #[allow(dead_code)] + pub fn with_get_file_references_error(self, error: crate::error::Error) -> Self { + *self.get_file_references_error.lock().unwrap() = Some(error); + self + } + + /// Set an error to be returned by `gc_regions` + pub fn with_gc_regions_error(self, error: crate::error::Error) -> Self { + *self.gc_regions_error.lock().unwrap() = Some(error); + self + } + + /// Set a sequence of errors to be returned by `gc_regions` for retry testing + pub fn set_gc_regions_error_sequence(&self, errors: Vec) { + *self.gc_regions_error_sequence.lock().unwrap() = errors; + } + + /// Set success after a specific number of retries for a region + pub fn set_gc_regions_success_after_retries(&self, region_id: RegionId, retries: usize) { + *self.gc_regions_success_after_retries.lock().unwrap() = + HashMap::from([(region_id, retries)]); + } + + /// Get the retry count for a specific region + pub fn get_retry_count(&self, region_id: RegionId) -> usize { + self.gc_regions_retry_count + .lock() + .unwrap() + .get(®ion_id) + .copied() + .unwrap_or(0) + } + + /// Reset all retry tracking + pub fn reset_retry_tracking(&self) { + *self.gc_regions_retry_count.lock().unwrap() = HashMap::new(); + *self.gc_regions_error_sequence.lock().unwrap() = Vec::new(); + *self.gc_regions_success_after_retries.lock().unwrap() = HashMap::new(); + } + + /// Set an error to be returned for a specific region + pub fn set_gc_regions_error_for_region(&self, region_id: RegionId, error: crate::error::Error) { + self.gc_regions_per_region_errors + .lock() + .unwrap() + .insert(region_id, error); + } + + /// Clear per-region errors + #[allow(unused)] + pub fn clear_gc_regions_per_region_errors(&self) { + self.gc_regions_per_region_errors.lock().unwrap().clear(); + } +} + +#[async_trait::async_trait] +impl SchedulerCtx for MockSchedulerCtx { + async fn get_table_to_region_stats(&self) -> Result>> { + *self.get_table_to_region_stats_calls.lock().unwrap() += 1; + + // Check if we should return an injected error + if let Some(error) = self.get_table_to_region_stats_error.lock().unwrap().take() { + return Err(error); + } + + Ok(self + .table_to_region_stats + .lock() + .unwrap() + .clone() + .unwrap_or_default()) + } + + async fn get_table_route( + &self, + table_id: TableId, + ) -> Result<(TableId, PhysicalTableRouteValue)> { + // Check if we should return an injected error + if let Some(error) = self.get_table_route_error.lock().unwrap().take() { + return Err(error); + } + + Ok(self + .table_routes + .lock() + .unwrap() + .get(&table_id) + .cloned() + .unwrap_or_else(|| (table_id, PhysicalTableRouteValue::default()))) + } + + async fn get_file_references( + &self, + _region_ids: &[RegionId], + _region_to_peer: &HashMap, + _timeout: Duration, + ) -> Result { + *self.get_file_references_calls.lock().unwrap() += 1; + + // Check if we should return an injected error + if let Some(error) = self.get_file_references_error.lock().unwrap().take() { + return Err(error); + } + + Ok(self.file_refs.lock().unwrap().clone().unwrap_or_default()) + } + + async fn gc_regions( + &self, + _peer: Peer, + region_id: RegionId, + _file_refs_manifest: &FileRefsManifest, + _full_file_listing: bool, + _timeout: Duration, + ) -> Result { + *self.gc_regions_calls.lock().unwrap() += 1; + + // Track retry count for this region + { + let mut retry_count = self.gc_regions_retry_count.lock().unwrap(); + *retry_count.entry(region_id).or_insert(0) += 1; + } + + // Check per-region error injection first + if let Some(error) = self + .gc_regions_per_region_errors + .lock() + .unwrap() + .remove(®ion_id) + { + return Err(error); + } + + // Check if we should return an injected error + if let Some(error) = self.gc_regions_error.lock().unwrap().take() { + return Err(error); + } + + // Handle error sequence for retry testing + { + let mut error_sequence = self.gc_regions_error_sequence.lock().unwrap(); + if !error_sequence.is_empty() { + let error = error_sequence.remove(0); + return Err(error); + } + } + + // Handle success after specific number of retries + { + let retry_count = self + .gc_regions_retry_count + .lock() + .unwrap() + .get(®ion_id) + .copied() + .unwrap_or(0); + let success_after_retries = self.gc_regions_success_after_retries.lock().unwrap(); + if let Some(&required_retries) = success_after_retries.get(®ion_id) + && retry_count <= required_retries + { + // Return retryable error until we reach the required retry count + return Err(crate::error::RetryLaterSnafu { + reason: format!( + "Mock retryable error for region {} (attempt {}/{})", + region_id, retry_count, required_retries + ), + } + .build()); + } + } + + self.gc_reports + .lock() + .unwrap() + .get(®ion_id) + .cloned() + .with_context(|| UnexpectedSnafu { + violated: format!("No corresponding gc report for {}", region_id), + }) + } +} + +pub struct TestEnv { + pub scheduler: GcScheduler, + pub ctx: Arc, + #[allow(dead_code)] + tx: Sender, +} + +impl TestEnv { + pub fn new() -> Self { + let ctx = Arc::new(MockSchedulerCtx::default()); + let (tx, rx) = GcScheduler::channel(); + let config = GcSchedulerOptions::default(); + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: rx, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + Self { scheduler, ctx, tx } + } + + pub fn with_candidates(self, candidates: HashMap>) -> Self { + *self.ctx.candidates.lock().unwrap() = Some(candidates); + self + } + + #[allow(dead_code)] + pub async fn run_scheduler(mut self) { + self.scheduler.run().await; + } + + #[allow(dead_code)] + pub async fn tick(&self) { + self.tx.send(Event::Tick).await.unwrap(); + } +} + +fn new_candidate(region_id: RegionId, score: f64) -> GcCandidate { + // well pass threshold for gc + let region_stat = mock_region_stat(region_id, RegionRole::Leader, 10_000, 10); + + GcCandidate { + region_id, + score: OrderedFloat(score), + region_stat, + } +} + +// Helper function to create a mock GC candidate +fn mock_candidate(region_id: RegionId) -> GcCandidate { + let region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); + GcCandidate { + region_id, + score: ordered_float::OrderedFloat(1.0), + region_stat, + } +} + +fn mock_region_stat( + id: RegionId, + role: RegionRole, + approximate_bytes: u64, + sst_num: u64, +) -> RegionStat { + RegionStat { + id, + role, + approximate_bytes, + sst_num, + region_manifest: RegionManifestInfo::Mito { + manifest_version: 0, + flushed_entry_id: 0, + file_removal_rate: 0, + }, + rcus: 0, + wcus: 0, + engine: "mito".to_string(), + num_rows: 0, + memtable_size: 0, + manifest_size: 0, + sst_size: 0, + index_size: 0, + data_topic_latest_entry_id: 0, + metadata_topic_latest_entry_id: 0, + written_bytes: 0, + } +} diff --git a/src/meta-srv/src/gc/mock/basic.rs b/src/meta-srv/src/gc/mock/basic.rs new file mode 100644 index 000000000000..4de93a18ffce --- /dev/null +++ b/src/meta-srv/src/gc/mock/basic.rs @@ -0,0 +1,169 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::{Arc, Mutex}; +use std::time::Instant; + +use common_meta::peer::Peer; +use common_telemetry::init_default_ut_logging; +use store_api::region_engine::RegionRole; +use store_api::storage::{FileId, FileRefsManifest, GcReport, RegionId}; + +use crate::gc::mock::{MockSchedulerCtx, TestEnv, mock_region_stat, new_candidate}; +use crate::gc::{GcScheduler, GcSchedulerOptions}; + +#[tokio::test] +async fn test_process_tables_concurrently_empty() { + let env = TestEnv::new(); + let report = env + .scheduler + .process_tables_concurrently(HashMap::new()) + .await; + + assert_eq!(report.processed_tables, 0); + assert_eq!(report.table_reports.len(), 0); + assert!(report.table_reports.is_empty()); +} + +#[tokio::test] +async fn test_process_tables_concurrently_with_candidates() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + let candidates = HashMap::from([(table_id, vec![new_candidate(region_id, 1.0)])]); + + let mut gc_reports = HashMap::new(); + let deleted_files = vec![FileId::random()]; + gc_reports.insert( + region_id, + GcReport { + deleted_files: HashMap::from([(region_id, deleted_files.clone())]), + ..Default::default() + }, + ); + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id, 1)]), + ..Default::default() + }; + let ctx = MockSchedulerCtx { + gc_reports: Arc::new(Mutex::new(gc_reports)), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer)]), + )])); + + let env = TestEnv::new().with_candidates(candidates); + // We need to replace the ctx with the one with gc_reports + let mut scheduler = env.scheduler; + scheduler.ctx = Arc::new(ctx); + + let candidates = env + .ctx + .candidates + .lock() + .unwrap() + .clone() + .unwrap_or_default(); + + let report = scheduler.process_tables_concurrently(candidates).await; + + assert_eq!(report.processed_tables, 1); + assert_eq!(report.table_reports.len(), 1); + assert_eq!( + report.table_reports[0].success_regions[0].deleted_files[®ion_id], + deleted_files + ); + assert!(report.table_reports[0].failed_regions.is_empty()); +} + +#[tokio::test] +async fn test_handle_tick() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + let candidates = HashMap::from([(table_id, vec![new_candidate(region_id, 1.0)])]); + + let mut gc_reports = HashMap::new(); + gc_reports.insert(region_id, GcReport::default()); + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id, 1)]), + ..Default::default() + }; + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(HashMap::from([( + table_id, + vec![mock_region_stat( + region_id, + RegionRole::Leader, + 200_000_000, + 10, + )], + )])))), + gc_reports: Arc::new(Mutex::new(gc_reports)), + candidates: Arc::new(Mutex::new(Some(candidates))), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer)]), + )])), + ); + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config: GcSchedulerOptions::default(), + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let report = scheduler.handle_tick().await.unwrap(); + + // Validate the returned GcJobReport + assert_eq!(report.processed_tables, 1, "Should process 1 table"); + assert_eq!(report.table_reports.len(), 1, "Should have 1 table report"); + + let table_report = &report.table_reports[0]; + assert_eq!(table_report.table_id, table_id, "Table ID should match"); + assert_eq!( + table_report.success_regions.len(), + 1, + "Should have 1 successful region" + ); + assert!( + table_report.failed_regions.is_empty(), + "Should have no failed regions" + ); + + assert_eq!(*ctx.get_table_to_region_stats_calls.lock().unwrap(), 1); + assert_eq!(*ctx.get_file_references_calls.lock().unwrap(), 1); + assert_eq!(*ctx.gc_regions_calls.lock().unwrap(), 1); + + let tracker = scheduler.region_gc_tracker.lock().await; + assert!( + tracker.contains_key(®ion_id), + "Tracker should have one region: {:?}", + tracker + ); +} diff --git a/src/meta-srv/src/gc/mock/candidate_select.rs b/src/meta-srv/src/gc/mock/candidate_select.rs new file mode 100644 index 000000000000..3c49e0aa6cab --- /dev/null +++ b/src/meta-srv/src/gc/mock/candidate_select.rs @@ -0,0 +1,370 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::{Arc, Mutex}; +use std::time::Instant; + +use common_meta::datanode::RegionManifestInfo; +use common_telemetry::init_default_ut_logging; +use store_api::region_engine::RegionRole; +use store_api::storage::RegionId; + +use crate::gc::mock::{MockSchedulerCtx, mock_region_stat}; +use crate::gc::{GcScheduler, GcSchedulerOptions}; + +/// Candidate Selection Tests +#[tokio::test] +async fn test_gc_candidate_filtering_by_role() { + init_default_ut_logging(); + + let table_id = 1; + let leader_region = RegionId::new(table_id, 1); + let follower_region = RegionId::new(table_id, 2); + + let mut leader_stat = mock_region_stat(leader_region, RegionRole::Leader, 200_000_000, 10); // 200MB + + let mut follower_stat = + mock_region_stat(follower_region, RegionRole::Follower, 200_000_000, 10); // 200MB + + // Set up manifest info for scoring + if let RegionManifestInfo::Mito { + file_removed_cnt, .. + } = &mut leader_stat.region_manifest + { + *file_removed_cnt = 5; + } + if let RegionManifestInfo::Mito { + file_removed_cnt, .. + } = &mut follower_stat.region_manifest + { + *file_removed_cnt = 5; + } + + let table_stats = HashMap::from([(table_id, vec![leader_stat.clone(), follower_stat.clone()])]); + + let ctx = Arc::new(MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + ..Default::default() + }); + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config: GcSchedulerOptions::default(), + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let stats = ctx + .table_to_region_stats + .lock() + .unwrap() + .clone() + .unwrap_or_default(); + + let candidates = scheduler.select_gc_candidates(&stats).await.unwrap(); + + // Should only select leader regions + assert_eq!( + candidates.len(), + 1, + "Expected 1 table with candidates, got {}", + candidates.len() + ); + if let Some(table_candidates) = candidates.get(&table_id) { + assert_eq!( + table_candidates.len(), + 1, + "Expected 1 candidate for table {}, got {}", + table_id, + table_candidates.len() + ); + assert_eq!( + table_candidates[0].region_id, leader_region, + "Expected leader region {}, got {}", + leader_region, table_candidates[0].region_id + ); + } else { + panic!("Expected table {} to have candidates", table_id); + } +} + +#[tokio::test] +async fn test_gc_candidate_size_threshold() { + init_default_ut_logging(); + + let table_id = 1; + let small_region = RegionId::new(table_id, 1); + let large_region = RegionId::new(table_id, 2); + + let mut small_stat = mock_region_stat(small_region, RegionRole::Leader, 50_000_000, 5); // 50MB + if let RegionManifestInfo::Mito { + file_removed_cnt, .. + } = &mut small_stat.region_manifest + { + *file_removed_cnt = 3; + } + + let mut large_stat = mock_region_stat(large_region, RegionRole::Leader, 200_000_000, 20); // 200MB + if let RegionManifestInfo::Mito { + file_removed_cnt, .. + } = &mut large_stat.region_manifest + { + *file_removed_cnt = 5; + } + + let table_stats = HashMap::from([(table_id, vec![small_stat, large_stat])]); + + let ctx = Arc::new(MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + ..Default::default() + }); + + let config = GcSchedulerOptions { + min_region_size_threshold: 100 * 1024 * 1024, // 100MB (default) + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let stats = ctx + .table_to_region_stats + .lock() + .unwrap() + .clone() + .unwrap_or_default(); + + let candidates = scheduler.select_gc_candidates(&stats).await.unwrap(); + + // Should only select large region + assert_eq!( + candidates.len(), + 1, + "Expected 1 table with candidates, got {}", + candidates.len() + ); + if let Some(table_candidates) = candidates.get(&table_id) { + assert_eq!( + table_candidates.len(), + 1, + "Expected 1 candidate for table {}, got {}", + table_id, + table_candidates.len() + ); + assert_eq!( + table_candidates[0].region_id, large_region, + "Expected large region {}, got {}", + large_region, table_candidates[0].region_id + ); + } else { + panic!("Expected table {} to have candidates", table_id); + } +} + +#[tokio::test] +async fn test_gc_candidate_scoring() { + init_default_ut_logging(); + + let table_id = 1; + let low_score_region = RegionId::new(table_id, 1); + let high_score_region = RegionId::new(table_id, 2); + + let mut low_stat = mock_region_stat(low_score_region, RegionRole::Leader, 200_000_000, 5); // 200MB + // Set low file removal rate for low_score_region + if let RegionManifestInfo::Mito { + file_removed_cnt, .. + } = &mut low_stat.region_manifest + { + *file_removed_cnt = 2; + } + + let mut high_stat = mock_region_stat(high_score_region, RegionRole::Leader, 200_000_000, 50); // 200MB + // Set high file removal rate for high_score_region + if let RegionManifestInfo::Mito { + file_removed_cnt, .. + } = &mut high_stat.region_manifest + { + *file_removed_cnt = 20; + } + + let table_stats = HashMap::from([(table_id, vec![low_stat, high_stat])]); + + let ctx = Arc::new(MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + ..Default::default() + }); + + let config = GcSchedulerOptions { + sst_count_weight: 1.0, + file_removed_cnt_weight: 0.5, + min_region_size_threshold: 100 * 1024 * 1024, // 100MB (default) + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let stats = ctx + .table_to_region_stats + .lock() + .unwrap() + .clone() + .unwrap_or_default(); + + let candidates = scheduler.select_gc_candidates(&stats).await.unwrap(); + + // Should select both regions but high score region should be first + assert_eq!( + candidates.len(), + 1, + "Expected 1 table with candidates, got {}", + candidates.len() + ); + if let Some(table_candidates) = candidates.get(&table_id) { + assert_eq!( + table_candidates.len(), + 2, + "Expected 2 candidates for table {}, got {}", + table_id, + table_candidates.len() + ); + // Higher score region should come first (sorted by score descending) + assert_eq!( + table_candidates[0].region_id, high_score_region, + "High score region should be first" + ); + assert!( + table_candidates[0].score > table_candidates[1].score, + "High score region should have higher score: {} > {}", + table_candidates[0].score, + table_candidates[1].score + ); + } else { + panic!("Expected table {} to have candidates", table_id); + } +} + +#[tokio::test] +async fn test_gc_candidate_regions_per_table_threshold() { + init_default_ut_logging(); + + let table_id = 1; + // Create 10 regions for the same table + let mut region_stats = Vec::new(); + + for i in 0..10 { + let region_id = RegionId::new(table_id, i + 1); + let mut stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 20); // 200MB + + // Set different file removal rates to create different scores + // Higher region IDs get higher scores (better GC candidates) + if let RegionManifestInfo::Mito { + file_removed_cnt, .. + } = &mut stat.region_manifest + { + *file_removed_cnt = (i as u64 + 1) * 2; // Region 1: 2, Region 2: 4, ..., Region 10: 20 + } + + region_stats.push(stat); + } + + let table_stats = HashMap::from([(table_id, region_stats)]); + + let ctx = Arc::new(MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + ..Default::default() + }); + + // Set regions_per_table_threshold to 3 + let config = GcSchedulerOptions { + regions_per_table_threshold: 3, + min_region_size_threshold: 100 * 1024 * 1024, // 100MB (default) + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let stats = ctx + .table_to_region_stats + .lock() + .unwrap() + .clone() + .unwrap_or_default(); + + let candidates = scheduler.select_gc_candidates(&stats).await.unwrap(); + + // Should have 1 table with candidates + assert_eq!( + candidates.len(), + 1, + "Expected 1 table with candidates, got {}", + candidates.len() + ); + + if let Some(table_candidates) = candidates.get(&table_id) { + // Should only have 3 candidates due to regions_per_table_threshold + assert_eq!( + table_candidates.len(), + 3, + "Expected 3 candidates for table {} due to regions_per_table_threshold, got {}", + table_id, + table_candidates.len() + ); + + // Verify that the top 3 scoring regions are selected + // Regions 8, 9, 10 should have the highest scores (file_removed_cnt: 16, 18, 20) + // They should be returned in descending order by score + let expected_regions = vec![10, 9, 8]; + let actual_regions: Vec = table_candidates + .iter() + .map(|c| c.region_id.region_number()) + .collect(); + + assert_eq!( + actual_regions, expected_regions, + "Expected regions {:?} to be selected, got {:?}", + expected_regions, actual_regions + ); + + // Verify they are sorted by score in descending order + for i in 0..table_candidates.len() - 1 { + assert!( + table_candidates[i].score >= table_candidates[i + 1].score, + "Candidates should be sorted by score descending: {} >= {}", + table_candidates[i].score, + table_candidates[i + 1].score + ); + } + } else { + panic!("Expected table {} to have candidates", table_id); + } +} diff --git a/src/meta-srv/src/gc/mock/con.rs b/src/meta-srv/src/gc/mock/con.rs new file mode 100644 index 000000000000..b222b3f62513 --- /dev/null +++ b/src/meta-srv/src/gc/mock/con.rs @@ -0,0 +1,461 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::{Arc, Mutex}; +use std::time::{Duration, Instant}; + +use common_meta::key::table_route::PhysicalTableRouteValue; +use common_meta::peer::Peer; +use common_meta::rpc::router::{Region, RegionRoute}; +use common_telemetry::{info, init_default_ut_logging}; +use store_api::region_engine::RegionRole; +use store_api::storage::{FileId, FileRefsManifest, GcReport, RegionId}; + +use crate::gc::mock::{MockSchedulerCtx, mock_candidate, mock_region_stat, new_candidate}; +use crate::gc::{GcScheduler, GcSchedulerOptions}; + +/// Concurrent Processing Tests +#[tokio::test] +async fn test_concurrent_table_processing_limits() { + init_default_ut_logging(); + + let mut candidates = HashMap::new(); + let mut gc_reports = HashMap::new(); + + // Create many tables with candidates + for table_id in 1..=10 { + let region_id = RegionId::new(table_id, 1); + candidates.insert(table_id, vec![new_candidate(region_id, 1.0)]); + gc_reports.insert( + region_id, + GcReport { + deleted_files: HashMap::from([(region_id, vec![FileId::random()])]), + ..Default::default() + }, + ); + } + + let ctx = MockSchedulerCtx { + candidates: Arc::new(Mutex::new(Some(candidates))), + file_refs: Arc::new(Mutex::new(Some(FileRefsManifest { + manifest_version: (1..=10).map(|i| (RegionId::new(i, 1), 1)).collect(), + ..Default::default() + }))), + gc_reports: Arc::new(Mutex::new(gc_reports)), + ..Default::default() + } + .with_table_routes( + (1..=10) + .map(|table_id| { + let region_id = RegionId::new(table_id, 1); + (table_id, (table_id, vec![(region_id, Peer::new(1, ""))])) + }) + .collect(), + ); + + let ctx = Arc::new(ctx); + + let config = GcSchedulerOptions { + max_concurrent_tables: 3, // Set a low limit + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let candidates = ctx.candidates.lock().unwrap().clone().unwrap_or_default(); + let report = scheduler.process_tables_concurrently(candidates).await; + + // Should process all tables but respect concurrency limits + assert_eq!(report.processed_tables, 10); +} + +#[tokio::test] +async fn test_mixed_success_failure_tables() { + init_default_ut_logging(); + + let table1 = 1; + let region1 = RegionId::new(table1, 1); + let table2 = 2; + let region2 = RegionId::new(table2, 1); + let peer = Peer::new(1, ""); + + let mut candidates = HashMap::new(); + candidates.insert(table1, vec![new_candidate(region1, 1.0)]); + candidates.insert(table2, vec![new_candidate(region2, 1.0)]); + + // Set up GC reports for success and failure + let mut gc_reports = HashMap::new(); + gc_reports.insert( + region1, + GcReport { + deleted_files: HashMap::from([(region1, vec![])]), + ..Default::default() + }, + ); + // region2 will have no GC report, simulating failure + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region1, 1), (region2, 1)]), + ..Default::default() + }; + + let ctx = Arc::new( + MockSchedulerCtx { + gc_reports: Arc::new(Mutex::new(gc_reports)), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + candidates: Arc::new(Mutex::new(Some(candidates))), + ..Default::default() + } + .with_table_routes(HashMap::from([ + (table1, (table1, vec![(region1, peer.clone())])), + (table2, (table2, vec![(region2, peer)])), + ])), + ); + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config: GcSchedulerOptions::default(), + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let candidates = ctx.candidates.lock().unwrap().clone().unwrap_or_default(); + + let report = scheduler.process_tables_concurrently(candidates).await; + + // Should have one success and one failure + assert_eq!(report.processed_tables, 2); + assert_eq!(report.table_reports.len(), 2); + assert_eq!(report.failed_tables.len(), 0); + assert_eq!( + report + .table_reports + .iter() + .map(|r| r.success_regions.len()) + .sum::(), + 1 + ); + assert_eq!( + report + .table_reports + .iter() + .map(|r| r.failed_regions.len()) + .sum::(), + 1 + ); +} + +// Region Concurrency Tests + +#[tokio::test] +async fn test_region_gc_concurrency_limit() { + init_default_ut_logging(); + + let table_id = 1; + let peer = Peer::new(1, ""); + + // Create multiple regions for the same table + let mut region_stats = Vec::new(); + let mut candidates = Vec::new(); + let mut gc_reports = HashMap::new(); + + for i in 1..=10 { + let region_id = RegionId::new(table_id, i as u32); + let region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + region_stats.push(region_stat); + + candidates.push(mock_candidate(region_id)); + + gc_reports.insert( + region_id, + GcReport { + deleted_files: HashMap::from([( + region_id, + vec![FileId::random(), FileId::random()], + )]), + ..Default::default() + }, + ); + } + + let table_stats = HashMap::from([(table_id, region_stats)]); + + let file_refs = FileRefsManifest { + manifest_version: (1..=10) + .map(|i| (RegionId::new(table_id, i as u32), 1)) + .collect(), + ..Default::default() + }; + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + gc_reports: Arc::new(Mutex::new(gc_reports)), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + ( + table_id, + (1..=10) + .map(|i| (RegionId::new(table_id, i as u32), peer.clone())) + .collect(), + ), + )])), + ); + + // Configure low concurrency limit + let config = GcSchedulerOptions { + region_gc_concurrency: 3, // Only 3 regions can be processed concurrently + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let start_time = Instant::now(); + let report = scheduler + .process_table_gc(table_id, candidates) + .await + .unwrap(); + let duration = start_time.elapsed(); + + // All regions should be processed successfully + assert_eq!(report.success_regions.len(), 10); + assert_eq!(report.failed_regions.len(), 0); + + // Verify that concurrency limit was respected (this is hard to test directly, + // but we can verify that the processing completed successfully) + info!( + "Processed 10 regions with concurrency limit 3 in {:?}", + duration + ); +} + +#[tokio::test] +async fn test_region_gc_concurrency_with_mixed_results() { + init_default_ut_logging(); + + let table_id = 1; + let peer = Peer::new(1, ""); + + // Create multiple regions with mixed success/failure + let mut region_stats = Vec::new(); + let mut candidates = Vec::new(); + let mut gc_reports = HashMap::new(); + + // Create the context first so we can set errors on it + let ctx = Arc::new(MockSchedulerCtx::default()); + + for i in 1..=6 { + let region_id = RegionId::new(table_id, i as u32); + let region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + region_stats.push(region_stat); + + candidates.push(mock_candidate(region_id)); + + if i % 2 == 0 { + // Even regions will succeed + gc_reports.insert( + region_id, + GcReport { + deleted_files: HashMap::from([( + region_id, + vec![FileId::random(), FileId::random()], + )]), + ..Default::default() + }, + ); + } else { + // Odd regions will fail - set non-retryable error for specific region + ctx.set_gc_regions_error_for_region( + region_id, + crate::error::RegionRouteNotFoundSnafu { region_id }.build(), + ); + } + } + + let table_stats = HashMap::from([(table_id, region_stats)]); + + let file_refs = FileRefsManifest { + manifest_version: (1..=6) + .map(|i| (RegionId::new(table_id, i as u32), 1)) + .collect(), + ..Default::default() + }; + + // Update the context with the data + *ctx.table_to_region_stats.lock().unwrap() = Some(table_stats); + *ctx.gc_reports.lock().unwrap() = gc_reports; + *ctx.file_refs.lock().unwrap() = Some(file_refs); + let region_routes = (1..=6) + .map(|i| RegionRoute { + region: Region::new_test(RegionId::new(table_id, i as u32)), + leader_peer: Some(peer.clone()), + ..Default::default() + }) + .collect(); + + *ctx.table_routes.lock().unwrap() = HashMap::from([( + table_id, + (table_id, PhysicalTableRouteValue::new(region_routes)), + )]); + + // Configure concurrency limit + let config = GcSchedulerOptions { + region_gc_concurrency: 2, // Process 2 regions concurrently + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let report = scheduler + .process_table_gc(table_id, candidates) + .await + .unwrap(); + + // Should have 3 successful and 3 failed regions + assert_eq!(report.success_regions.len(), 3); + assert_eq!(report.failed_regions.len(), 3); + + // Verify that successful regions are the even-numbered ones + for report in &report.success_regions { + for region in report.deleted_files.keys() { + let region_num = region.region_number(); + assert_eq!( + region_num % 2, + 0, + "Successful regions should be even-numbered" + ); + } + } + + // Verify that failed regions are the odd-numbered ones + for region_id in report.failed_regions.keys() { + let region_num = region_id.region_number(); + assert_eq!(region_num % 2, 1, "Failed regions should be odd-numbered"); + } +} + +#[tokio::test] +async fn test_region_gc_concurrency_with_retryable_errors() { + init_default_ut_logging(); + + let table_id = 1; + let peer = Peer::new(1, ""); + + // Create multiple regions + let mut region_stats = Vec::new(); + let mut candidates = Vec::new(); + + for i in 1..=5 { + let region_id = RegionId::new(table_id, i as u32); + let region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + region_stats.push(region_stat); + candidates.push(mock_candidate(region_id)); + } + + let table_stats = HashMap::from([(table_id, region_stats)]); + + let file_refs = FileRefsManifest { + manifest_version: (1..=5) + .map(|i| (RegionId::new(table_id, i as u32), 1)) + .collect(), + ..Default::default() + }; + + let gc_report = (1..=5) + .map(|i| (RegionId::new(table_id, i as u32), GcReport::default())) + .collect(); + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + gc_reports: Arc::new(Mutex::new(gc_report)), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + ( + table_id, + (1..=5) + .map(|i| (RegionId::new(table_id, i as u32), peer.clone())) + .collect(), + ), + )])), + ); + + // Configure concurrency limit and retry settings + let config = GcSchedulerOptions { + region_gc_concurrency: 2, // Process 2 regions concurrently + max_retries_per_region: 2, + retry_backoff_duration: Duration::from_millis(50), + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + // Set up retryable errors for all regions (they'll succeed after 1 retry) + for i in 1..=5 { + let region_id = RegionId::new(table_id, i as u32); + ctx.set_gc_regions_success_after_retries(region_id, 1); + } + + let report = scheduler + .process_table_gc(table_id, candidates) + .await + .unwrap(); + + // All regions should eventually succeed after retries + assert_eq!(report.success_regions.len(), 5); + assert_eq!(report.failed_regions.len(), 0); + + // Verify that retries were attempted for all regions + for i in 1..=5 { + let region_id = RegionId::new(table_id, i as u32); + let retry_count = ctx.get_retry_count(region_id); + assert!( + retry_count >= 1, + "Region {} should have at least 1 attempt", + region_id + ); + } +} diff --git a/src/meta-srv/src/gc/mock/config.rs b/src/meta-srv/src/gc/mock/config.rs new file mode 100644 index 000000000000..f5d8fbde1b1c --- /dev/null +++ b/src/meta-srv/src/gc/mock/config.rs @@ -0,0 +1,196 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::{Arc, Mutex}; +use std::time::Instant; + +use common_meta::datanode::RegionManifestInfo; +use common_telemetry::init_default_ut_logging; +use store_api::region_engine::RegionRole; +use store_api::storage::RegionId; + +use crate::gc::mock::{MockSchedulerCtx, mock_region_stat}; +use crate::gc::{GcScheduler, GcSchedulerOptions}; + +/// Configuration Tests +#[tokio::test] +async fn test_different_gc_weights() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + + let mut region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB to pass size threshold + + if let RegionManifestInfo::Mito { + file_removed_cnt, .. + } = &mut region_stat.region_manifest + { + *file_removed_cnt = 5; + } + + let table_stats = HashMap::from([(table_id, vec![region_stat])]); + + let ctx = Arc::new(MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + ..Default::default() + }); + + // Test with different weights + let config1 = GcSchedulerOptions { + sst_count_weight: 2.0, + file_removed_cnt_weight: 0.5, + min_region_size_threshold: 100 * 1024 * 1024, // 100MB (default) + ..Default::default() + }; + + let scheduler1 = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config: config1, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let stats = ctx + .table_to_region_stats + .lock() + .unwrap() + .clone() + .unwrap_or_default(); + + let candidates1 = scheduler1.select_gc_candidates(&stats).await.unwrap(); + + let config2 = GcSchedulerOptions { + sst_count_weight: 0.5, + file_removed_cnt_weight: 2.0, + min_region_size_threshold: 100 * 1024 * 1024, // 100MB (default) + ..Default::default() + }; + + let scheduler2 = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config: config2, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let stats = &ctx + .table_to_region_stats + .lock() + .unwrap() + .clone() + .unwrap_or_default(); + let candidates2 = scheduler2.select_gc_candidates(stats).await.unwrap(); + + // Both should select the region but with different scores + assert_eq!( + candidates1.len(), + 1, + "Expected 1 table with candidates for config1, got {}", + candidates1.len() + ); + assert_eq!( + candidates2.len(), + 1, + "Expected 1 table with candidates for config2, got {}", + candidates2.len() + ); + + // Verify the region is actually selected + assert!( + candidates1.contains_key(&table_id), + "Config1 should contain table_id {}", + table_id + ); + assert!( + candidates2.contains_key(&table_id), + "Config2 should contain table_id {}", + table_id + ); +} + +#[tokio::test] +async fn test_regions_per_table_threshold() { + init_default_ut_logging(); + + let table_id = 1; + let mut region_stats = Vec::new(); + + // Create many regions + for i in 1..=10 { + let region_id = RegionId::new(table_id, i as u32); + let mut stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + + if let RegionManifestInfo::Mito { + file_removed_cnt, .. + } = &mut stat.region_manifest + { + *file_removed_cnt = 5; + } + + region_stats.push(stat); + } + + let table_stats = HashMap::from([(table_id, region_stats)]); + + let ctx = Arc::new(MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + ..Default::default() + }); + + let config = GcSchedulerOptions { + regions_per_table_threshold: 3, // Limit to 3 regions per table + min_region_size_threshold: 100 * 1024 * 1024, // 100MB (default) + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let stats = ctx + .table_to_region_stats + .lock() + .unwrap() + .clone() + .unwrap_or_default(); + + let candidates = scheduler.select_gc_candidates(&stats).await.unwrap(); + + assert_eq!( + candidates.len(), + 1, + "Expected 1 table with candidates, got {}", + candidates.len() + ); + if let Some(table_candidates) = candidates.get(&table_id) { + // Should be limited to 3 regions + assert_eq!( + table_candidates.len(), + 3, + "Expected 3 candidates for table {}, got {}", + table_id, + table_candidates.len() + ); + } else { + panic!("Expected table {} to have candidates", table_id); + } +} diff --git a/src/meta-srv/src/gc/mock/err_handle.rs b/src/meta-srv/src/gc/mock/err_handle.rs new file mode 100644 index 000000000000..547abe2cd0b7 --- /dev/null +++ b/src/meta-srv/src/gc/mock/err_handle.rs @@ -0,0 +1,241 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::{HashMap, HashSet}; +use std::sync::{Arc, Mutex}; +use std::time::Instant; + +use common_meta::datanode::RegionManifestInfo; +use common_meta::peer::Peer; +use common_telemetry::init_default_ut_logging; +use store_api::region_engine::RegionRole; +use store_api::storage::{FileId, FileRefsManifest, GcReport, RegionId}; + +use crate::gc::mock::{MockSchedulerCtx, mock_region_stat}; +use crate::gc::{GcScheduler, GcSchedulerOptions}; + +/// Error Handling Tests +#[tokio::test] +async fn test_gc_regions_failure_handling() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + + // Create region stat with proper size and file_removal_rate to ensure it gets selected as candidate + let mut region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + if let RegionManifestInfo::Mito { + file_removal_rate, .. + } = &mut region_stat.region_manifest + { + *file_removal_rate = 5; + } + + let table_stats = HashMap::from([(table_id, vec![region_stat])]); + + // Create a context that will return an error for gc_regions + let mut gc_reports = HashMap::new(); + gc_reports.insert(region_id, GcReport::default()); + + // Inject an error for gc_regions method + let gc_error = crate::error::UnexpectedSnafu { + violated: "Simulated GC failure for testing".to_string(), + } + .build(); + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id, 1)]), + file_refs: HashMap::from([(region_id, HashSet::from([FileId::random()]))]), + }; + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + gc_reports: Arc::new(Mutex::new(gc_reports)), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer)]), + )])) + .with_gc_regions_error(gc_error), + ); + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config: GcSchedulerOptions::default(), + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + // This should handle the failure gracefully + let report = scheduler.handle_tick().await.unwrap(); + + // Validate the report shows the failure handling + assert_eq!( + report.processed_tables, 1, + "Should process 1 table despite failure" + ); + assert_eq!(report.table_reports.len(), 1, "Should have 1 table report"); + + let table_report = &report.table_reports[0]; + assert_eq!(table_report.table_id, table_id, "Table ID should match"); + assert_eq!( + table_report.success_regions.len(), + 0, + "Should have 0 successful regions due to GC failure" + ); + assert_eq!( + table_report.failed_regions.len(), + 1, + "Should have 1 failed region" + ); + + // Verify that calls were made despite potential failures + assert_eq!( + *ctx.get_table_to_region_stats_calls.lock().unwrap(), + 1, + "Expected 1 call to get_table_to_region_stats" + ); + assert_eq!( + *ctx.get_file_references_calls.lock().unwrap(), + 1, + "Expected 1 call to get_file_references" + ); + assert_eq!( + *ctx.gc_regions_calls.lock().unwrap(), + 1, + "Expected 1 call to gc_regions" + ); +} + +#[tokio::test] +async fn test_get_file_references_failure() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + + // Create region stat with proper size and file_removal_rate to ensure it gets selected as candidate + let mut region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + if let RegionManifestInfo::Mito { + file_removal_rate, .. + } = &mut region_stat.region_manifest + { + *file_removal_rate = 5; + } + + let table_stats = HashMap::from([(table_id, vec![region_stat])]); + + // Create context with empty file refs (simulating failure) + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + file_refs: Arc::new(Mutex::new(Some(FileRefsManifest::default()))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer)]), + )])), + ); + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config: GcSchedulerOptions::default(), + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let report = scheduler.handle_tick().await.unwrap(); + + // Validate the report shows the expected results + assert_eq!(report.processed_tables, 1, "Should process 1 table"); + assert_eq!(report.table_reports.len(), 0, "Should have 0 table report"); + assert_eq!(report.failed_tables.len(), 1, "Should have 1 failed table"); + + // Should still attempt to get file references + assert_eq!( + *ctx.get_file_references_calls.lock().unwrap(), + 1, + "Expected 1 call to get_file_references" + ); +} + +#[tokio::test] +async fn test_get_table_route_failure() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + + // Create region stat with proper size and file_removal_rate to ensure it gets selected as candidate + let mut region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + if let RegionManifestInfo::Mito { + file_removal_rate, .. + } = &mut region_stat.region_manifest + { + *file_removal_rate = 5; + } + + let table_stats = HashMap::from([(table_id, vec![region_stat])]); + + // Inject an error for get_table_route method to simulate failure + let route_error = crate::error::UnexpectedSnafu { + violated: "Simulated table route failure for testing".to_string(), + } + .build(); + + // Create context with table route error injection + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + ..Default::default() + } + .with_get_table_route_error(route_error), + ); + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config: GcSchedulerOptions::default(), + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + // Get candidates first + let stats = &ctx + .table_to_region_stats + .lock() + .unwrap() + .clone() + .unwrap_or_default(); + let candidates = scheduler.select_gc_candidates(stats).await.unwrap(); + + // This should handle table route failure gracefully + let report = scheduler.process_tables_concurrently(candidates).await; + + // Should process the table but fail due to route error + assert_eq!(report.processed_tables, 1, "Expected 1 processed table"); + assert_eq!( + report.table_reports.len(), + 0, + "Expected 0 successful tables due to route error" + ); +} diff --git a/src/meta-srv/src/gc/mock/full_list.rs b/src/meta-srv/src/gc/mock/full_list.rs new file mode 100644 index 000000000000..b5df49b8f972 --- /dev/null +++ b/src/meta-srv/src/gc/mock/full_list.rs @@ -0,0 +1,273 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::{Arc, Mutex}; +use std::time::{Duration, Instant}; + +use common_meta::peer::Peer; +use common_telemetry::init_default_ut_logging; +use store_api::region_engine::RegionRole; +use store_api::storage::{FileId, FileRefsManifest, GcReport, RegionId}; + +use crate::gc::mock::{MockSchedulerCtx, mock_candidate, mock_region_stat}; +use crate::gc::{GcScheduler, GcSchedulerOptions}; + +// Full File Listing Tests + +#[tokio::test] +async fn test_full_file_listing_first_time_gc() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + + let region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + let table_stats = HashMap::from([(table_id, vec![region_stat])]); + + let gc_report = GcReport { + deleted_files: HashMap::from([(region_id, vec![FileId::random(), FileId::random()])]), + ..Default::default() + }; + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id, 1)]), + ..Default::default() + }; + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + gc_reports: Arc::new(Mutex::new(HashMap::from([(region_id, gc_report)]))), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer)]), + )])), + ); + + // Configure short full file listing interval for testing + let config = GcSchedulerOptions { + full_file_listing_interval: Duration::from_secs(3600), // 1 hour + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + // First GC - should use full listing since region has never been GC'd + let report = scheduler + .process_table_gc(table_id, vec![mock_candidate(region_id)]) + .await + .unwrap(); + + assert_eq!(report.success_regions.len(), 1); + + // Verify that full listing was used by checking the tracker + let tracker = scheduler.region_gc_tracker.lock().await; + let gc_info = tracker + .get(®ion_id) + .expect("Region should be in tracker"); + assert!( + gc_info.last_full_listing_time.is_some(), + "First GC should use full listing" + ); +} + +#[tokio::test] +async fn test_full_file_listing_interval_enforcement() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + + let region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + let table_stats = HashMap::from([(table_id, vec![region_stat])]); + + let gc_report = GcReport { + deleted_files: HashMap::from([(region_id, vec![FileId::random(), FileId::random()])]), + ..Default::default() + }; + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id, 1)]), + ..Default::default() + }; + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + gc_reports: Arc::new(Mutex::new(HashMap::from([(region_id, gc_report)]))), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer)]), + )])), + ); + + // Configure very short full file listing interval for testing + let config = GcSchedulerOptions { + full_file_listing_interval: Duration::from_millis(100), // 100ms + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + // First GC - should use full listing + let report1 = scheduler + .process_table_gc(table_id, vec![mock_candidate(region_id)]) + .await + .unwrap(); + assert_eq!(report1.success_regions.len(), 1); + + // Get the first full listing time + let first_full_listing_time = { + let tracker = scheduler.region_gc_tracker.lock().await; + let gc_info = tracker + .get(®ion_id) + .expect("Region should be in tracker"); + gc_info + .last_full_listing_time + .expect("Should have full listing time") + }; + + // Wait for interval to pass + tokio::time::sleep(Duration::from_millis(150)).await; + + // Second GC - should use full listing again since interval has passed + let report2 = scheduler + .process_table_gc(table_id, vec![mock_candidate(region_id)]) + .await + .unwrap(); + assert_eq!(report2.success_regions.len(), 1); + + // Verify that full listing was used again + let tracker = scheduler.region_gc_tracker.lock().await; + let gc_info = tracker + .get(®ion_id) + .expect("Region should be in tracker"); + let second_full_listing_time = gc_info + .last_full_listing_time + .expect("Should have full listing time"); + + assert!( + second_full_listing_time > first_full_listing_time, + "Second GC should update full listing time" + ); +} + +#[tokio::test] +async fn test_full_file_listing_no_interval_passed() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + + let region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + let table_stats = HashMap::from([(table_id, vec![region_stat])]); + + let gc_report = GcReport { + deleted_files: HashMap::from([(region_id, vec![FileId::random(), FileId::random()])]), + ..Default::default() + }; + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id, 1)]), + ..Default::default() + }; + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + gc_reports: Arc::new(Mutex::new(HashMap::from([(region_id, gc_report)]))), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer)]), + )])), + ); + + // Configure long full file listing interval + let config = GcSchedulerOptions { + full_file_listing_interval: Duration::from_secs(3600), // 1 hour + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + // First GC - should use full listing + let report1 = scheduler + .process_table_gc(table_id, vec![mock_candidate(region_id)]) + .await + .unwrap(); + assert_eq!(report1.success_regions.len(), 1); + + // Get the first full listing time + let first_full_listing_time = { + let tracker = scheduler.region_gc_tracker.lock().await; + let gc_info = tracker + .get(®ion_id) + .expect("Region should be in tracker"); + gc_info + .last_full_listing_time + .expect("Should have full listing time") + }; + + // Second GC immediately - should NOT use full listing since interval hasn't passed + let report2 = scheduler + .process_table_gc(table_id, vec![mock_candidate(region_id)]) + .await + .unwrap(); + assert_eq!(report2.success_regions.len(), 1); + + // Verify that full listing time was NOT updated + let tracker = scheduler.region_gc_tracker.lock().await; + let gc_info = tracker + .get(®ion_id) + .expect("Region should be in tracker"); + let second_full_listing_time = gc_info + .last_full_listing_time + .expect("Should have full listing time"); + + assert_eq!( + second_full_listing_time, first_full_listing_time, + "Second GC should not update full listing time when interval hasn't passed" + ); +} diff --git a/src/meta-srv/src/gc/mock/integration.rs b/src/meta-srv/src/gc/mock/integration.rs new file mode 100644 index 000000000000..0d29f3b1d417 --- /dev/null +++ b/src/meta-srv/src/gc/mock/integration.rs @@ -0,0 +1,225 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::{Arc, Mutex}; +use std::time::{Duration, Instant}; + +use common_meta::datanode::RegionManifestInfo; +use common_meta::peer::Peer; +use common_telemetry::init_default_ut_logging; +use store_api::region_engine::RegionRole; +use store_api::storage::{FileId, FileRefsManifest, GcReport, RegionId}; + +use crate::gc::mock::{MockSchedulerCtx, mock_region_stat}; +use crate::gc::{GcScheduler, GcSchedulerOptions}; + +// Integration Flow Tests + +#[tokio::test] +async fn test_full_gc_workflow() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + + let mut region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + + if let RegionManifestInfo::Mito { + file_removal_rate, .. + } = &mut region_stat.region_manifest + { + *file_removal_rate = 5; + } + + let table_stats = HashMap::from([(table_id, vec![region_stat])]); + + let mut gc_reports = HashMap::new(); + gc_reports.insert( + region_id, + GcReport { + deleted_files: HashMap::from([(region_id, vec![FileId::random(), FileId::random()])]), + ..Default::default() + }, + ); + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id, 1)]), + ..Default::default() + }; + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + gc_reports: Arc::new(Mutex::new(gc_reports)), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer)]), + )])), + ); + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config: GcSchedulerOptions::default(), + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + // Run the full workflow + let report = scheduler.handle_tick().await.unwrap(); + + // Validate the returned GcJobReport + assert_eq!(report.processed_tables, 1, "Should process 1 table"); + assert_eq!(report.table_reports.len(), 1, "Should have 1 table report"); + + let table_report = &report.table_reports[0]; + assert_eq!(table_report.table_id, table_id, "Table ID should match"); + assert_eq!( + table_report.success_regions.len(), + 1, + "Should have 1 successful region" + ); + assert!( + table_report.failed_regions.is_empty(), + "Should have no failed regions" + ); + + // Verify all steps were executed + assert_eq!( + *ctx.get_table_to_region_stats_calls.lock().unwrap(), + 1, + "Expected 1 call to get_table_to_region_stats" + ); + assert_eq!( + *ctx.get_file_references_calls.lock().unwrap(), + 1, + "Expected 1 call to get_file_references" + ); + assert_eq!( + *ctx.gc_regions_calls.lock().unwrap(), + 1, + "Expected 1 call to gc_regions" + ); +} + +#[tokio::test] +async fn test_tracker_cleanup() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + + // Create region stat with proper file_removal_rate to ensure it gets selected as candidate + let mut region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + if let RegionManifestInfo::Mito { + file_removal_rate, .. + } = &mut region_stat.region_manifest + { + *file_removal_rate = 5; + } + + let table_stats = HashMap::from([(table_id, vec![region_stat])]); + + let mut gc_reports = HashMap::new(); + gc_reports.insert(region_id, GcReport::default()); + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id, 1)]), + ..Default::default() + }; + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + gc_reports: Arc::new(Mutex::new(gc_reports)), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer)]), + )])), + ); + + let old_region_gc_tracker = { + let mut tracker = HashMap::new(); + tracker.insert( + region_id, + crate::gc::tracker::RegionGcInfo { + last_full_listing_time: Some(Instant::now() - Duration::from_secs(7200)), // 2 hours ago + last_gc_time: Instant::now() - Duration::from_secs(7200), // 2 hours ago + }, + ); + // also insert a different table that should also be cleaned up + tracker.insert( + RegionId::new(2, 1), + crate::gc::tracker::RegionGcInfo { + last_full_listing_time: Some(Instant::now() - Duration::from_secs(7200)), // 2 hours ago + last_gc_time: Instant::now() - Duration::from_secs(7200), // 2 hours ago + }, + ); + tracker + }; + + // Use a custom config with shorter cleanup interval to trigger cleanup + let config = GcSchedulerOptions { + // 30 minutes + tracker_cleanup_interval: Duration::from_secs(1800), + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(old_region_gc_tracker)), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new( + Instant::now() - Duration::from_secs(3600), // Old cleanup time (1 hour ago) + )), + }; + + let report = scheduler.handle_tick().await.unwrap(); + + // Validate the returned GcJobReport + assert_eq!(report.processed_tables, 1, "Should process 1 table"); + assert_eq!(report.table_reports.len(), 1, "Should have 1 table report"); + + let table_report = &report.table_reports[0]; + assert_eq!(table_report.table_id, table_id, "Table ID should match"); + assert_eq!( + table_report.success_regions.len(), + 1, + "Should have 1 successful region" + ); + assert!( + table_report.failed_regions.is_empty(), + "Should have no failed regions" + ); + + // Verify tracker was updated + let tracker = scheduler.region_gc_tracker.lock().await; + assert!( + tracker.contains_key(®ion_id), + "Tracker should contain region {}", + region_id + ); + // only one entry + assert_eq!(tracker.len(), 1, "Tracker should only have 1 entry"); +} diff --git a/src/meta-srv/src/gc/mock/misc.rs b/src/meta-srv/src/gc/mock/misc.rs new file mode 100644 index 000000000000..9c92f0b6ab47 --- /dev/null +++ b/src/meta-srv/src/gc/mock/misc.rs @@ -0,0 +1,135 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::{Arc, Mutex}; +use std::time::Instant; + +use common_meta::peer::Peer; +use common_telemetry::init_default_ut_logging; +use store_api::storage::{FileRefsManifest, GcReport, RegionId}; + +use crate::gc::mock::{MockSchedulerCtx, new_candidate}; +use crate::gc::{GcScheduler, GcSchedulerOptions}; + +/// Edge Case Tests + +#[tokio::test] +async fn test_empty_file_refs_manifest() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + let candidates = HashMap::from([(table_id, vec![new_candidate(region_id, 1.0)])]); + + // Empty file refs manifest + let file_refs = FileRefsManifest::default(); + + let ctx = Arc::new( + MockSchedulerCtx { + file_refs: Arc::new(Mutex::new(Some(file_refs))), + candidates: Arc::new(Mutex::new(Some(candidates))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer)]), + )])), + ); + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config: GcSchedulerOptions::default(), + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let candidates = ctx.candidates.lock().unwrap().clone().unwrap_or_default(); + + let report = scheduler.process_tables_concurrently(candidates).await; + + assert_eq!(report.processed_tables, 1); + // Should handle empty file refs gracefully +} + +#[tokio::test] +async fn test_multiple_regions_per_table() { + init_default_ut_logging(); + + let table_id = 1; + let region1 = RegionId::new(table_id, 1); + let region2 = RegionId::new(table_id, 2); + let region3 = RegionId::new(table_id, 3); + let peer = Peer::new(1, ""); + + let candidates = HashMap::from([( + table_id, + vec![ + new_candidate(region1, 1.0), + new_candidate(region2, 2.0), + new_candidate(region3, 3.0), + ], + )]); + + let mut gc_reports = HashMap::new(); + gc_reports.insert(region1, GcReport::default()); + gc_reports.insert(region2, GcReport::default()); + gc_reports.insert(region3, GcReport::default()); + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region1, 1), (region2, 1), (region3, 1)]), + ..Default::default() + }; + + let ctx = Arc::new( + MockSchedulerCtx { + gc_reports: Arc::new(Mutex::new(gc_reports)), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + candidates: Arc::new(Mutex::new(Some(candidates))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + ( + table_id, + vec![ + (region1, peer.clone()), + (region2, peer.clone()), + (region3, peer), + ], + ), + )])), + ); + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config: GcSchedulerOptions::default(), + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let candidates = ctx.candidates.lock().unwrap().clone().unwrap_or_default(); + + let report = scheduler.process_tables_concurrently(candidates).await; + + assert_eq!(report.processed_tables, 1); + assert_eq!(report.table_reports.len(), 1); + + // Should have processed all 3 regions + let table_report = &report.table_reports[0]; + assert_eq!(table_report.success_regions.len(), 3); +} diff --git a/src/meta-srv/src/gc/mock/retry.rs b/src/meta-srv/src/gc/mock/retry.rs new file mode 100644 index 000000000000..c06a2af5c166 --- /dev/null +++ b/src/meta-srv/src/gc/mock/retry.rs @@ -0,0 +1,240 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::{Arc, Mutex}; +use std::time::{Duration, Instant}; + +use common_meta::peer::Peer; +use common_telemetry::init_default_ut_logging; +use store_api::region_engine::RegionRole; +use store_api::storage::{FileId, FileRefsManifest, GcReport, RegionId}; + +use crate::gc::mock::{MockSchedulerCtx, mock_candidate, mock_region_stat}; +use crate::gc::{GcScheduler, GcSchedulerOptions}; + +/// Retry Logic Tests +#[tokio::test] +async fn test_retry_logic_with_retryable_errors() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + + let region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + let table_stats = HashMap::from([(table_id, vec![region_stat])]); + + let gc_report = GcReport { + deleted_files: HashMap::from([(region_id, vec![FileId::random(), FileId::random()])]), + ..Default::default() + }; + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id, 1)]), + ..Default::default() + }; + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + gc_reports: Arc::new(Mutex::new(HashMap::from([(region_id, gc_report)]))), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer)]), + )])), + ); + + // Configure retry settings + let config = GcSchedulerOptions { + max_retries_per_region: 3, + retry_backoff_duration: Duration::from_millis(100), // Short backoff for testing + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + // Test 1: Success after 2 retryable errors + ctx.reset_retry_tracking(); + ctx.set_gc_regions_success_after_retries(region_id, 2); + + let start_time = Instant::now(); + let report = scheduler + .process_table_gc(table_id, vec![mock_candidate(region_id)]) + .await + .unwrap(); + let duration = start_time.elapsed(); + + assert_eq!(report.success_regions.len(), 1); + assert_eq!(report.failed_regions.len(), 0); + assert_eq!(ctx.get_retry_count(region_id), 3); // 2 retries + 1 success + + // Verify backoff was applied (should take at least 200ms for 2 retries with 100ms backoff) + assert!( + duration >= Duration::from_millis(200), + "Expected backoff duration not met: {:?}", + duration + ); + + // Test 2: Exceed max retries + ctx.reset_retry_tracking(); + ctx.set_gc_regions_success_after_retries(region_id, 5); // More than max_retries_per_region (3) + + let report = scheduler + .process_table_gc(table_id, vec![mock_candidate(region_id)]) + .await + .unwrap(); + + assert_eq!(report.success_regions.len(), 0); + assert_eq!(report.failed_regions.len(), 1); + assert_eq!(ctx.get_retry_count(region_id), 3); // Should stop at max_retries_per_region +} + +#[tokio::test] +async fn test_retry_logic_with_error_sequence() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + + let region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + let table_stats = HashMap::from([(table_id, vec![region_stat])]); + + let gc_report = GcReport { + deleted_files: HashMap::from([(region_id, vec![FileId::random(), FileId::random()])]), + ..Default::default() + }; + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id, 1)]), + ..Default::default() + }; + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + gc_reports: Arc::new(Mutex::new(HashMap::from([(region_id, gc_report)]))), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer)]), + )])), + ); + + let config = GcSchedulerOptions { + max_retries_per_region: 5, + retry_backoff_duration: Duration::from_millis(50), + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + // Test with sequence: retryable error -> retryable error -> success + let retryable_error1 = crate::error::RetryLaterSnafu { + reason: "Test retryable error 1".to_string(), + } + .build(); + + let retryable_error2 = crate::error::MailboxTimeoutSnafu { id: 123u64 }.build(); + + ctx.reset_retry_tracking(); + ctx.set_gc_regions_error_sequence(vec![retryable_error1, retryable_error2]); + + let report = scheduler + .process_table_gc(table_id, vec![mock_candidate(region_id)]) + .await + .unwrap(); + + assert_eq!(report.success_regions.len(), 1); + assert_eq!(report.failed_regions.len(), 0); + assert_eq!(ctx.get_retry_count(region_id), 3); // 2 errors + 1 success +} + +#[tokio::test] +async fn test_retry_logic_non_retryable_error() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + + let region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB + let table_stats = HashMap::from([(table_id, vec![region_stat])]); + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id, 1)]), + ..Default::default() + }; + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer)]), + )])), + ); + + let config = GcSchedulerOptions { + max_retries_per_region: 3, + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + // Test with non-retryable error (should fail immediately) + let non_retryable_error = crate::error::RegionRouteNotFoundSnafu { region_id }.build(); + + ctx.reset_retry_tracking(); + ctx.gc_regions_error + .lock() + .unwrap() + .replace(non_retryable_error); + + let report = scheduler + .process_table_gc(table_id, vec![mock_candidate(region_id)]) + .await + .unwrap(); + + assert_eq!(report.success_regions.len(), 0); + assert_eq!(report.failed_regions.len(), 1); + assert_eq!(ctx.get_retry_count(region_id), 1); // Only 1 attempt, no retries +} diff --git a/src/meta-srv/src/gc/options.rs b/src/meta-srv/src/gc/options.rs new file mode 100644 index 000000000000..a6e2e8398fb0 --- /dev/null +++ b/src/meta-srv/src/gc/options.rs @@ -0,0 +1,168 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::time::Duration; + +use serde::{Deserialize, Serialize}; + +use crate::error::{self, Result}; + +/// The interval of the gc ticker. +#[allow(unused)] +pub(crate) const TICKER_INTERVAL: Duration = Duration::from_secs(60 * 5); + +/// Configuration for GC operations. +#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] +#[serde(default)] +pub struct GcSchedulerOptions { + /// Whether GC is enabled. Default to false. + /// If set to false, no GC will be performed, and potentially some + /// files from datanodes will never be deleted. + pub enable: bool, + /// Maximum number of tables to process concurrently. + pub max_concurrent_tables: usize, + /// Maximum number of retries per region when GC fails. + pub max_retries_per_region: usize, + /// Concurrency for region GC within a table. + pub region_gc_concurrency: usize, + /// Backoff duration between retries. + pub retry_backoff_duration: Duration, + /// Minimum region size threshold for GC (in bytes). + pub min_region_size_threshold: u64, + /// Weight for SST file count in GC scoring. + pub sst_count_weight: f64, + /// Weight for file removal rate in GC scoring. + pub file_removed_cnt_weight: f64, + /// Cooldown period between GC operations on the same region. + pub gc_cooldown_period: Duration, + /// Maximum number of regions to select for GC per table. + pub regions_per_table_threshold: usize, + /// Timeout duration for mailbox communication with datanodes. + pub mailbox_timeout: Duration, + /// Interval for performing full file listing during GC to find orphan files. + /// Full file listing is expensive but necessary to clean up orphan files. + /// Set to a larger value (e.g., 24 hours) to balance performance and cleanup. + /// Every Nth GC cycle will use full file listing, where N = full_file_listing_interval / TICKER_INTERVAL. + pub full_file_listing_interval: Duration, + /// Interval for cleaning up stale region entries from the GC tracker. + /// This removes entries for regions that no longer exist (e.g., after table drops). + /// Set to a larger value (e.g., 6 hours) since this is just for memory cleanup. + pub tracker_cleanup_interval: Duration, +} + +impl Default for GcSchedulerOptions { + fn default() -> Self { + Self { + enable: false, + max_concurrent_tables: 10, + max_retries_per_region: 3, + retry_backoff_duration: Duration::from_secs(5), + region_gc_concurrency: 16, + min_region_size_threshold: 100 * 1024 * 1024, // 100MB + sst_count_weight: 1.0, + file_removed_cnt_weight: 0.5, + gc_cooldown_period: Duration::from_secs(60 * 5), // 5 minutes + regions_per_table_threshold: 20, // Select top 20 regions per table + mailbox_timeout: Duration::from_secs(60), // 60 seconds + // Perform full file listing every 24 hours to find orphan files + full_file_listing_interval: Duration::from_secs(60 * 60 * 24), + // Clean up stale tracker entries every 6 hours + tracker_cleanup_interval: Duration::from_secs(60 * 60 * 6), + } + } +} + +impl GcSchedulerOptions { + /// Validates the configuration options. + pub fn validate(&self) -> Result<()> { + if self.max_concurrent_tables == 0 { + return error::InvalidArgumentsSnafu { + err_msg: "max_concurrent_tables must be greater than 0", + } + .fail(); + } + + if self.max_retries_per_region == 0 { + return error::InvalidArgumentsSnafu { + err_msg: "max_retries_per_region must be greater than 0", + } + .fail(); + } + + if self.region_gc_concurrency == 0 { + return error::InvalidArgumentsSnafu { + err_msg: "region_gc_concurrency must be greater than 0", + } + .fail(); + } + + if self.retry_backoff_duration.is_zero() { + return error::InvalidArgumentsSnafu { + err_msg: "retry_backoff_duration must be greater than 0", + } + .fail(); + } + + if self.sst_count_weight < 0.0 { + return error::InvalidArgumentsSnafu { + err_msg: "sst_count_weight must be non-negative", + } + .fail(); + } + + if self.file_removed_cnt_weight < 0.0 { + return error::InvalidArgumentsSnafu { + err_msg: "file_removal_rate_weight must be non-negative", + } + .fail(); + } + + if self.gc_cooldown_period.is_zero() { + return error::InvalidArgumentsSnafu { + err_msg: "gc_cooldown_period must be greater than 0", + } + .fail(); + } + + if self.regions_per_table_threshold == 0 { + return error::InvalidArgumentsSnafu { + err_msg: "regions_per_table_threshold must be greater than 0", + } + .fail(); + } + + if self.mailbox_timeout.is_zero() { + return error::InvalidArgumentsSnafu { + err_msg: "mailbox_timeout must be greater than 0", + } + .fail(); + } + + if self.full_file_listing_interval.is_zero() { + return error::InvalidArgumentsSnafu { + err_msg: "full_file_listing_interval must be greater than 0", + } + .fail(); + } + + if self.tracker_cleanup_interval.is_zero() { + return error::InvalidArgumentsSnafu { + err_msg: "tracker_cleanup_interval must be greater than 0", + } + .fail(); + } + + Ok(()) + } +} diff --git a/src/meta-srv/src/gc/scheduler.rs b/src/meta-srv/src/gc/scheduler.rs new file mode 100644 index 000000000000..80aba861f100 --- /dev/null +++ b/src/meta-srv/src/gc/scheduler.rs @@ -0,0 +1,197 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::sync::Arc; +use std::time::Instant; + +use common_meta::key::TableMetadataManagerRef; +use common_procedure::ProcedureManagerRef; +use common_telemetry::{error, info}; +use futures::stream::StreamExt; +use store_api::storage::{GcReport, RegionId}; +use table::metadata::TableId; +use tokio::sync::mpsc::{Receiver, Sender}; + +use crate::cluster::MetaPeerClientRef; +use crate::define_ticker; +use crate::error::Result; +use crate::gc::candidate::GcCandidate; +use crate::gc::ctx::{DefaultGcSchedulerCtx, SchedulerCtx}; +use crate::gc::options::{GcSchedulerOptions, TICKER_INTERVAL}; +use crate::gc::tracker::RegionGcTracker; +use crate::service::mailbox::MailboxRef; + +/// Report for a table garbage collection. +#[derive(Debug, Default)] +pub struct TableGcReport { + #[allow(unused)] + pub table_id: TableId, + /// Successful GC reports for each region. + pub success_regions: Vec, + /// Failed GC reports for each region. + pub failed_regions: HashMap, +} + +/// Report for a GC job. +#[derive(Debug, Default)] +pub struct GcJobReport { + pub processed_tables: usize, + pub table_reports: Vec, + pub failed_tables: HashMap, +} + +/// [`Event`] represents various types of events that can be processed by the gc ticker. +/// +/// Variants: +/// - `Tick`: This event is used to trigger gc periodically. +pub(crate) enum Event { + Tick, +} + +#[allow(unused)] +pub(crate) type GcTickerRef = Arc; + +define_ticker!( + /// [GcTicker] is used to trigger gc periodically. + GcTicker, + event_type = Event, + event_value = Event::Tick +); + +/// [`GcScheduler`] is used to periodically trigger garbage collection on datanodes. +pub struct GcScheduler { + pub(crate) ctx: Arc, + /// The receiver of events. + pub(crate) receiver: Receiver, + /// GC configuration. + pub(crate) config: GcSchedulerOptions, + /// Tracks the last GC time for regions. + pub(crate) region_gc_tracker: Arc>, + /// Last time the tracker was cleaned up. + pub(crate) last_tracker_cleanup: Arc>, +} + +impl GcScheduler { + /// Creates a new [`GcScheduler`] with custom configuration. + pub(crate) fn new_with_config( + table_metadata_manager: TableMetadataManagerRef, + procedure_manager: ProcedureManagerRef, + meta_peer_client: MetaPeerClientRef, + mailbox: MailboxRef, + server_addr: String, + config: GcSchedulerOptions, + ) -> Result<(Self, GcTicker)> { + // Validate configuration before creating the scheduler + config.validate()?; + + let (tx, rx) = Self::channel(); + let gc_ticker = GcTicker::new(TICKER_INTERVAL, tx); + let gc_trigger = Self { + ctx: Arc::new(DefaultGcSchedulerCtx { + table_metadata_manager, + procedure_manager, + meta_peer_client, + mailbox, + server_addr, + }), + receiver: rx, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + Ok((gc_trigger, gc_ticker)) + } + + pub(crate) fn channel() -> (Sender, Receiver) { + tokio::sync::mpsc::channel(8) + } + + /// Starts the gc trigger. + pub fn try_start(mut self) -> Result<()> { + common_runtime::spawn_global(async move { self.run().await }); + info!("GC trigger started"); + Ok(()) + } + + pub(crate) async fn run(&mut self) { + while let Some(event) = self.receiver.recv().await { + match event { + Event::Tick => { + info!("Received gc tick"); + if let Err(e) = self.handle_tick().await { + error!("Failed to handle gc tick: {}", e); + } + } + } + } + } + + pub(crate) async fn handle_tick(&self) -> Result { + info!("Start to trigger gc"); + let report = self.trigger_gc().await?; + + // Periodically clean up stale tracker entries + self.cleanup_tracker_if_needed().await?; + + info!("Finished gc trigger"); + + Ok(report) + } + + /// Process multiple tables concurrently with limited parallelism. + /// + /// TODO(discord9): acquire lock for prevent region migration during gc. + pub(crate) async fn process_tables_concurrently( + &self, + per_table_candidates: HashMap>, + ) -> GcJobReport { + let mut report = GcJobReport::default(); + + // Create a stream of table GC tasks with limited concurrency + let results: Vec<_> = futures::stream::iter( + per_table_candidates + .into_iter() + .filter(|(_, candidates)| !candidates.is_empty()), + ) + .map(|(table_id, candidates)| { + let scheduler = self; + async move { + ( + table_id, + scheduler.process_table_gc(table_id, candidates).await, + ) + } + }) + .buffer_unordered(self.config.max_concurrent_tables) + .collect() + .await; + + // Process all table GC results + for (table_id, result) in results { + report.processed_tables += 1; + match result { + Ok(table) => { + report.table_reports.push(table); + } + Err(e) => { + error!("Failed to process table GC: {:#?}", e); + report.failed_tables.insert(table_id, format!("{:#?}", e)); + } + } + } + + report + } +} diff --git a/src/meta-srv/src/gc/tracker.rs b/src/meta-srv/src/gc/tracker.rs new file mode 100644 index 000000000000..7f128290602f --- /dev/null +++ b/src/meta-srv/src/gc/tracker.rs @@ -0,0 +1,106 @@ +// Copyright 2023 Greptime Team +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::{HashMap, HashSet}; +use std::time::Instant; + +use common_telemetry::info; +use store_api::storage::RegionId; + +use crate::error::Result; +use crate::gc::scheduler::GcScheduler; + +/// Tracks GC timing information for a region. +#[derive(Debug, Clone)] +pub(crate) struct RegionGcInfo { + /// Last time a regular GC was performed on this region. + pub(crate) last_gc_time: Instant, + /// Last time a full file listing GC was performed on this region. + pub(crate) last_full_listing_time: Option, +} + +impl RegionGcInfo { + pub(crate) fn new(last_gc_time: Instant) -> Self { + Self { + last_gc_time, + last_full_listing_time: None, + } + } +} + +/// Tracks the last GC time for regions to implement cooldown. +pub(crate) type RegionGcTracker = HashMap; + +impl GcScheduler { + /// Clean up stale entries from the region GC tracker if enough time has passed. + /// This removes entries for regions that no longer exist in the current table routes. + pub(crate) async fn cleanup_tracker_if_needed(&self) -> Result<()> { + let mut last_cleanup = self.last_tracker_cleanup.lock().await; + let now = Instant::now(); + + // Check if enough time has passed since last cleanup + if now.duration_since(*last_cleanup) < self.config.tracker_cleanup_interval { + return Ok(()); + } + + info!("Starting region GC tracker cleanup"); + let cleanup_start = Instant::now(); + + // Get all current region IDs from table routes + let table_to_region_stats = self.ctx.get_table_to_region_stats().await?; + let mut current_regions = HashSet::new(); + for region_stats in table_to_region_stats.values() { + for region_stat in region_stats { + current_regions.insert(region_stat.id); + } + } + + // Remove stale entries from tracker + let mut tracker = self.region_gc_tracker.lock().await; + let initial_count = tracker.len(); + tracker.retain(|region_id, _| current_regions.contains(region_id)); + let removed_count = initial_count - tracker.len(); + + *last_cleanup = now; + + info!( + "Completed region GC tracker cleanup: removed {} stale entries out of {} total (retained {}). Duration: {:?}", + removed_count, + initial_count, + tracker.len(), + cleanup_start.elapsed() + ); + + Ok(()) + } + + /// Determine if full file listing should be used for a region based on the last full listing time. + pub(crate) async fn should_use_full_listing(&self, region_id: RegionId) -> bool { + let gc_tracker = self.region_gc_tracker.lock().await; + let now = Instant::now(); + + if let Some(gc_info) = gc_tracker.get(®ion_id) { + if let Some(last_full_listing) = gc_info.last_full_listing_time { + let elapsed = now.duration_since(last_full_listing); + elapsed >= self.config.full_file_listing_interval + } else { + // Never did full listing for this region, do it now + true + } + } else { + // First time GC for this region, do full listing + true + } + } +} diff --git a/src/meta-srv/src/metasrv.rs b/src/meta-srv/src/metasrv.rs index 39fbf66ccbb7..c454bc1ca59d 100644 --- a/src/meta-srv/src/metasrv.rs +++ b/src/meta-srv/src/metasrv.rs @@ -67,6 +67,7 @@ use crate::error::{ StartTelemetryTaskSnafu, StopProcedureManagerSnafu, }; use crate::failure_detector::PhiAccrualFailureDetectorOptions; +use crate::gc::{GcSchedulerOptions, GcTickerRef}; use crate::handler::{HeartbeatHandlerGroupBuilder, HeartbeatHandlerGroupRef}; use crate::procedure::ProcedureManagerListenerAdapter; use crate::procedure::region_migration::manager::RegionMigrationManagerRef; @@ -207,6 +208,8 @@ pub struct MetasrvOptions { pub event_recorder: EventRecorderOptions, /// The stats persistence options. pub stats_persistence: StatsPersistenceOptions, + /// The GC scheduler options. + pub gc: GcSchedulerOptions, } impl fmt::Debug for MetasrvOptions { @@ -303,6 +306,7 @@ impl Default for MetasrvOptions { node_max_idle_time: Duration::from_secs(24 * 60 * 60), event_recorder: EventRecorderOptions::default(), stats_persistence: StatsPersistenceOptions::default(), + gc: GcSchedulerOptions::default(), } } } @@ -524,6 +528,7 @@ pub struct Metasrv { table_id_sequence: SequenceRef, reconciliation_manager: ReconciliationManagerRef, resource_stat: ResourceStatRef, + gc_ticker: Option, plugins: Plugins, } @@ -584,6 +589,9 @@ impl Metasrv { if let Some(region_flush_trigger) = &self.region_flush_ticker { leadership_change_notifier.add_listener(region_flush_trigger.clone() as _); } + if let Some(gc_ticker) = &self.gc_ticker { + leadership_change_notifier.add_listener(gc_ticker.clone() as _); + } if let Some(customizer) = self.plugins.get::() { customizer.customize(&mut leadership_change_notifier); } diff --git a/src/meta-srv/src/metasrv/builder.rs b/src/meta-srv/src/metasrv/builder.rs index 5a33dc9c4f9b..04b5bd02c6c0 100644 --- a/src/meta-srv/src/metasrv/builder.rs +++ b/src/meta-srv/src/metasrv/builder.rs @@ -56,6 +56,7 @@ use crate::cache_invalidator::MetasrvCacheInvalidator; use crate::cluster::MetaPeerClientRef; use crate::error::{self, BuildWalOptionsAllocatorSnafu, Result}; use crate::events::EventHandlerImpl; +use crate::gc::GcScheduler; use crate::greptimedb_telemetry::get_greptimedb_telemetry_task; use crate::handler::failure_handler::RegionFailureHandler; use crate::handler::flow_state_handler::FlowStateHandler; @@ -458,6 +459,22 @@ impl MetasrvBuilder { None }; + let gc_ticker = if options.gc.enable { + let (gc_scheduler, gc_ticker) = GcScheduler::new_with_config( + table_metadata_manager.clone(), + procedure_manager.clone(), + meta_peer_client.clone(), + mailbox.clone(), + options.grpc.server_addr.clone(), + options.gc.clone(), + )?; + gc_scheduler.try_start()?; + + Some(Arc::new(gc_ticker)) + } else { + None + }; + let customized_region_lease_renewer = plugins .as_ref() .and_then(|plugins| plugins.get::()); @@ -562,6 +579,7 @@ impl MetasrvBuilder { reconciliation_manager, topic_stats_registry, resource_stat: Arc::new(resource_stat), + gc_ticker, }) } } diff --git a/src/mito2/src/gc.rs b/src/mito2/src/gc.rs index b9285c183b4f..7c33771d94ba 100644 --- a/src/mito2/src/gc.rs +++ b/src/mito2/src/gc.rs @@ -322,6 +322,8 @@ impl LocalGcWorker { let region_id = manifest.metadata.region_id; let current_files = &manifest.files; + let in_manifest_file_cnt = current_files.len(); + let recently_removed_files = self.get_removed_files_expel_times(&manifest).await?; if recently_removed_files.is_empty() { @@ -344,6 +346,8 @@ impl LocalGcWorker { .chain(tmp_ref_files.clone().into_iter()) .collect(); + let in_used_file_cnt = in_used.len(); + let unused_files = self .list_to_be_deleted_files(region_id, &in_used, recently_removed_files, concurrency) .await?; diff --git a/src/mito2/src/manifest/action.rs b/src/mito2/src/manifest/action.rs index dedb228e2542..76058d4ba0c0 100644 --- a/src/mito2/src/manifest/action.rs +++ b/src/mito2/src/manifest/action.rs @@ -18,6 +18,7 @@ use std::collections::{HashMap, HashSet}; use std::time::Duration; use chrono::Utc; +use common_telemetry::warn; use serde::{Deserialize, Serialize}; use snafu::{OptionExt, ResultExt}; use store_api::ManifestVersion; diff --git a/src/mito2/src/manifest/manager.rs b/src/mito2/src/manifest/manager.rs index ab6c9dd26d40..f3354fe97f78 100644 --- a/src/mito2/src/manifest/manager.rs +++ b/src/mito2/src/manifest/manager.rs @@ -215,8 +215,6 @@ impl RegionManifestManager { }) } - /// Opens an existing manifest. - /// /// Returns `Ok(None)` if no such manifest. pub async fn open( options: RegionManifestOptions, diff --git a/src/mito2/src/sst/file_purger.rs b/src/mito2/src/sst/file_purger.rs index 81a004ff15ee..0b31f9b9a79e 100644 --- a/src/mito2/src/sst/file_purger.rs +++ b/src/mito2/src/sst/file_purger.rs @@ -162,7 +162,7 @@ impl FilePurger for ObjectStoreFilePurger { // if not on local file system, instead inform the global file purger to remove the file reference. // notice that no matter whether the file is deleted or not, we need to remove the reference // because the file is no longer in use nonetheless. - self.file_ref_manager.remove_file(&file_meta); + self.file_ref_manager.remove_file(file_meta.file_id()); // TODO(discord9): consider impl a .tombstone file to reduce files needed to list } diff --git a/src/mito2/src/sst/file_ref.rs b/src/mito2/src/sst/file_ref.rs index a281aeb5fa90..5929b51789b2 100644 --- a/src/mito2/src/sst/file_ref.rs +++ b/src/mito2/src/sst/file_ref.rs @@ -22,7 +22,7 @@ use store_api::storage::{FileRef, FileRefsManifest, RegionId}; use crate::error::Result; use crate::metrics::GC_REF_FILE_CNT; use crate::region::MitoRegionRef; -use crate::sst::file::FileMeta; +use crate::sst::file::{FileMeta, RegionFileId}; /// File references for a region. /// It contains all files referenced by the region. @@ -269,7 +269,7 @@ mod tests { expected_region_ref_manifest ); - file_ref_mgr.remove_file(&file_meta); + file_ref_mgr.remove_file(file_meta.file_id()); assert_eq!( file_ref_mgr @@ -285,7 +285,7 @@ mod tests { expected_region_ref_manifest ); - file_ref_mgr.remove_file(&file_meta); + file_ref_mgr.remove_file(file_meta.file_id()); assert!( file_ref_mgr From 21ab763a84c4fb59e549b83d0060951e31e3dbbb Mon Sep 17 00:00:00 2001 From: discord9 Date: Thu, 13 Nov 2025 11:50:50 +0800 Subject: [PATCH 02/10] chore Signed-off-by: discord9 --- src/meta-srv/src/gc/mock/integration.rs | 10 +++++----- src/mito2/src/manifest/action.rs | 1 - 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/meta-srv/src/gc/mock/integration.rs b/src/meta-srv/src/gc/mock/integration.rs index 0d29f3b1d417..f5d29e348567 100644 --- a/src/meta-srv/src/gc/mock/integration.rs +++ b/src/meta-srv/src/gc/mock/integration.rs @@ -38,10 +38,10 @@ async fn test_full_gc_workflow() { let mut region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB if let RegionManifestInfo::Mito { - file_removal_rate, .. + file_removed_cnt, .. } = &mut region_stat.region_manifest { - *file_removal_rate = 5; + *file_removed_cnt = 5; } let table_stats = HashMap::from([(table_id, vec![region_stat])]); @@ -126,13 +126,13 @@ async fn test_tracker_cleanup() { let region_id = RegionId::new(table_id, 1); let peer = Peer::new(1, ""); - // Create region stat with proper file_removal_rate to ensure it gets selected as candidate + // Create region stat with proper file_removed_cnt to ensure it gets selected as candidate let mut region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB if let RegionManifestInfo::Mito { - file_removal_rate, .. + file_removed_cnt, .. } = &mut region_stat.region_manifest { - *file_removal_rate = 5; + *file_removed_cnt = 5; } let table_stats = HashMap::from([(table_id, vec![region_stat])]); diff --git a/src/mito2/src/manifest/action.rs b/src/mito2/src/manifest/action.rs index 76058d4ba0c0..dedb228e2542 100644 --- a/src/mito2/src/manifest/action.rs +++ b/src/mito2/src/manifest/action.rs @@ -18,7 +18,6 @@ use std::collections::{HashMap, HashSet}; use std::time::Duration; use chrono::Utc; -use common_telemetry::warn; use serde::{Deserialize, Serialize}; use snafu::{OptionExt, ResultExt}; use store_api::ManifestVersion; From 3d0851316a5e3ff20f73151b4573fff75ea8d6dc Mon Sep 17 00:00:00 2001 From: discord9 Date: Thu, 13 Nov 2025 12:33:36 +0800 Subject: [PATCH 03/10] chore Signed-off-by: discord9 --- src/meta-srv/src/gc/mock.rs | 2 +- src/meta-srv/src/gc/mock/err_handle.rs | 18 +++++++++--------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/meta-srv/src/gc/mock.rs b/src/meta-srv/src/gc/mock.rs index 33aa40eb0176..3c1f761d3a8f 100644 --- a/src/meta-srv/src/gc/mock.rs +++ b/src/meta-srv/src/gc/mock.rs @@ -363,7 +363,7 @@ fn mock_region_stat( region_manifest: RegionManifestInfo::Mito { manifest_version: 0, flushed_entry_id: 0, - file_removal_rate: 0, + file_removed_cnt: 0, }, rcus: 0, wcus: 0, diff --git a/src/meta-srv/src/gc/mock/err_handle.rs b/src/meta-srv/src/gc/mock/err_handle.rs index 547abe2cd0b7..a4e1003ec258 100644 --- a/src/meta-srv/src/gc/mock/err_handle.rs +++ b/src/meta-srv/src/gc/mock/err_handle.rs @@ -34,13 +34,13 @@ async fn test_gc_regions_failure_handling() { let region_id = RegionId::new(table_id, 1); let peer = Peer::new(1, ""); - // Create region stat with proper size and file_removal_rate to ensure it gets selected as candidate + // Create region stat with proper size and file_removed_cnt to ensure it gets selected as candidate let mut region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB if let RegionManifestInfo::Mito { - file_removal_rate, .. + file_removed_cnt, .. } = &mut region_stat.region_manifest { - *file_removal_rate = 5; + *file_removed_cnt = 5; } let table_stats = HashMap::from([(table_id, vec![region_stat])]); @@ -131,13 +131,13 @@ async fn test_get_file_references_failure() { let region_id = RegionId::new(table_id, 1); let peer = Peer::new(1, ""); - // Create region stat with proper size and file_removal_rate to ensure it gets selected as candidate + // Create region stat with proper size and file_removed_cnt to ensure it gets selected as candidate let mut region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB if let RegionManifestInfo::Mito { - file_removal_rate, .. + file_removed_cnt, .. } = &mut region_stat.region_manifest { - *file_removal_rate = 5; + *file_removed_cnt = 5; } let table_stats = HashMap::from([(table_id, vec![region_stat])]); @@ -185,13 +185,13 @@ async fn test_get_table_route_failure() { let table_id = 1; let region_id = RegionId::new(table_id, 1); - // Create region stat with proper size and file_removal_rate to ensure it gets selected as candidate + // Create region stat with proper size and file_removed_cnt to ensure it gets selected as candidate let mut region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); // 200MB if let RegionManifestInfo::Mito { - file_removal_rate, .. + file_removed_cnt, .. } = &mut region_stat.region_manifest { - *file_removal_rate = 5; + *file_removed_cnt = 5; } let table_stats = HashMap::from([(table_id, vec![region_stat])]); From cb6bd14f68ea2cf6d274a421c8992e195748d83e Mon Sep 17 00:00:00 2001 From: discord9 Date: Mon, 17 Nov 2025 11:16:53 +0800 Subject: [PATCH 04/10] wip: refactoring test Signed-off-by: discord9 --- src/meta-srv/src/gc/handler.rs | 466 ++++++++++++++++++++++++- src/meta-srv/src/gc/mock.rs | 63 ++-- src/meta-srv/src/gc/mock/basic.rs | 61 ++-- src/meta-srv/src/gc/mock/con.rs | 94 +++-- src/meta-srv/src/gc/mock/err_handle.rs | 71 ++-- src/meta-srv/src/gc/mock/full_list.rs | 35 +- src/meta-srv/src/gc/mock/misc.rs | 37 +- src/meta-srv/src/gc/mock/retry.rs | 372 +++++++++++++++++++- src/meta-srv/src/gc/scheduler.rs | 74 ++-- src/meta-srv/src/gc/tracker.rs | 22 ++ 10 files changed, 1094 insertions(+), 201 deletions(-) diff --git a/src/meta-srv/src/gc/handler.rs b/src/meta-srv/src/gc/handler.rs index 39f4df44097b..5358cb340e22 100644 --- a/src/meta-srv/src/gc/handler.rs +++ b/src/meta-srv/src/gc/handler.rs @@ -53,14 +53,26 @@ impl GcScheduler { return Ok(Default::default()); } - // Step 3: Process tables concurrently with limited parallelism - let report = self.process_tables_concurrently(per_table_candidates).await; + // Step 3: Aggregate candidates by datanode + let datanode_to_candidates = self + .aggregate_candidates_by_datanode(per_table_candidates) + .await?; + + if datanode_to_candidates.is_empty() { + info!("No valid datanode candidates found, skipping GC cycle"); + return Ok(Default::default()); + } + + // Step 4: Process datanodes concurrently with limited parallelism + let report = self + .process_datanodes_concurrently(datanode_to_candidates) + .await; let duration = start_time.elapsed(); info!( - "Finished GC cycle. Processed {} tables ({} successful). Duration: {:?}", - report.processed_tables, - report.table_reports.len(), + "Finished GC cycle. Processed {} datanodes ({} successful). Duration: {:?}", + report.per_datanode_reports.len(), // Reuse field for datanode count + report.failed_datanodes.len(), duration ); debug!("Detailed GC Job Report: {report:#?}"); @@ -257,13 +269,15 @@ impl GcScheduler { // Determine if we should use full file listing for this region let should_use_full_listing = self.should_use_full_listing(region_id).await; + let mut current_retry_round = 0; - loop { + while current_retry_round <= self.config.max_retries_per_region { + current_retry_round += 1; match self .ctx .gc_regions( peer.clone(), - region_id, + &[region_id], ¤t_manifest, should_use_full_listing, self.config.mailbox_timeout, @@ -299,6 +313,7 @@ impl GcScheduler { } } } + unreachable!() } /// Handle successful GC report, checking if retry is needed for outdated regions. @@ -406,4 +421,441 @@ impl GcScheduler { .cloned() .with_context(|| RegionRouteNotFoundSnafu { region_id }) } + /// Aggregate GC candidates by their corresponding datanode peer. + pub(crate) async fn aggregate_candidates_by_datanode( + &self, + per_table_candidates: HashMap>, + ) -> Result>> { + let mut datanode_to_candidates: HashMap> = HashMap::new(); + + for (table_id, candidates) in per_table_candidates { + if candidates.is_empty() { + continue; + } + + // Get table route information to map regions to peers + let (phy_table_id, table_peer) = self.ctx.get_table_route(table_id).await?; + + if phy_table_id != table_id { + // Skip logical tables + continue; + } + + let region_to_peer = table_peer + .region_routes + .iter() + .filter_map(|r| { + r.leader_peer + .as_ref() + .map(|peer| (r.region.id, peer.clone())) + }) + .collect::>(); + + for candidate in candidates { + if let Some(peer) = region_to_peer.get(&candidate.region_id) { + datanode_to_candidates + .entry(peer.clone()) + .or_default() + .push((table_id, candidate)); + } else { + warn!( + "Skipping region {} for table {}: no leader peer found", + candidate.region_id, table_id + ); + } + } + } + + info!( + "Aggregated GC candidates for {} datanodes", + datanode_to_candidates.len() + ); + Ok(datanode_to_candidates) + } + + /// Process multiple datanodes concurrently with limited parallelism. + pub(crate) async fn process_datanodes_concurrently( + &self, + datanode_to_candidates: HashMap>, + ) -> GcJobReport { + let mut report = GcJobReport::default(); + + // Create a stream of datanode GC tasks with limited concurrency + let results: Vec<_> = futures::stream::iter( + datanode_to_candidates + .into_iter() + .filter(|(_, candidates)| !candidates.is_empty()), + ) + .map(|(peer, candidates)| { + let scheduler = self; + let peer_clone = peer.clone(); + async move { + ( + peer, + scheduler.process_datanode_gc(peer_clone, candidates).await, + ) + } + }) + .buffer_unordered(self.config.max_concurrent_tables) // Reuse table concurrency limit for datanodes + .collect() + .await; + + // Process all datanode GC results and collect regions that need retry from table reports + for (peer, result) in results { + match result { + Ok(dn_report) => { + report.per_datanode_reports.insert(peer.id, dn_report); + } + Err(e) => { + error!("Failed to process datanode GC for peer {}: {:#?}", peer, e); + // Note: We don't have a direct way to map peer to table_id here, + // so we just log the error. The table_reports will contain individual region failures. + report.failed_datanodes.entry(peer.id).or_default().push(e); + } + } + } + + // Collect all regions that need retry from the table reports + let all_need_retry_regions: Vec = report + .per_datanode_reports + .iter() + .flat_map(|(_, report)| report.need_retry_regions.iter().copied()) + .collect(); + + // Handle regions that need retry due to migration + // These regions should be rediscovered and retried in the current GC cycle + if !all_need_retry_regions.is_empty() { + info!( + "Found {} regions that need retry due to migration or outdated file references", + all_need_retry_regions.len() + ); + + // Retry these regions by rediscovering their current datanodes + match self + .retry_regions_with_rediscovery(&all_need_retry_regions) + .await + { + Ok(retry_report) => report.merge(retry_report), + Err(e) => { + error!("Failed to retry regions: {}", e); + } + } + } + + report + } + + /// Process GC for a single datanode with all its candidate regions. + /// Returns the table reports for this datanode. + pub(crate) async fn process_datanode_gc( + &self, + peer: Peer, + candidates: Vec<(TableId, GcCandidate)>, + ) -> Result { + info!( + "Starting GC for datanode {} with {} candidate regions", + peer, + candidates.len() + ); + + if candidates.is_empty() { + return Ok(Default::default()); + } + + // Extract all region IDs and map them to their peers + let region_to_peer: HashMap = candidates + .iter() + .map(|(_, candidate)| (candidate.region_id, peer.clone())) + .collect(); + + let all_region_ids: Vec = candidates.iter().map(|(_, c)| c.region_id).collect(); + + // Step 1: Get file references for all regions on this datanode + let file_refs_manifest = self + .ctx + .get_file_references( + &all_region_ids, + ®ion_to_peer, + self.config.mailbox_timeout, + ) + .await?; + + // Step 2: Create a single GcRegionProcedure for all regions on this datanode + + let gc_report = { + // Partition regions into full listing and fast listing in a single pass + let mut need_full_list_regions = Vec::new(); + let mut fast_list_regions = Vec::new(); + + for region_id in &all_region_ids { + if self.should_use_full_listing(*region_id).await { + need_full_list_regions.push(*region_id); + } else { + fast_list_regions.push(*region_id); + } + } + + let mut combined_report = GcReport::default(); + + // First process regions that need full listing + if !need_full_list_regions.is_empty() { + match self + .ctx + .gc_regions( + peer.clone(), + &need_full_list_regions, + &file_refs_manifest, + true, + self.config.mailbox_timeout, + ) + .await + { + Ok(report) => combined_report.merge(report), + Err(e) => { + error!( + "Failed to GC regions {:?} on datanode {}: {}", + need_full_list_regions, peer, e + ); + + // Add to need_retry_regions since it failed + combined_report + .need_retry_regions + .extend(need_full_list_regions.clone()); + } + } + } + + if !fast_list_regions.is_empty() { + match self + .ctx + .gc_regions( + peer.clone(), + &fast_list_regions, + &file_refs_manifest, + true, + self.config.mailbox_timeout, + ) + .await + { + Ok(report) => combined_report.merge(report), + Err(e) => { + error!( + "Failed to GC regions {:?} on datanode {}: {}", + fast_list_regions, peer, e + ); + + // Add to need_retry_regions since it failed + combined_report + .need_retry_regions + .extend(fast_list_regions.clone().into_iter()); + } + } + } + + combined_report + }; + + // Step 3: Process the combined GC report and update table reports + for region_id in &all_region_ids { + // Update GC tracker for successful regions + let mut gc_tracker = self.region_gc_tracker.lock().await; + let now = Instant::now(); + let gc_info = gc_tracker + .entry(*region_id) + .or_insert_with(|| RegionGcInfo::new(now)); + gc_info.last_gc_time = now; + // TODO: Set last_full_listing_time if full listing was used + } + + info!( + "Completed GC for datanode {}: {} regions processed", + peer, + all_region_ids.len() + ); + + Ok(gc_report) + } + + /// Retry regions that need retry by rediscovering their current datanodes. + /// This handles cases where regions have migrated to different datanodes. + async fn retry_regions_with_rediscovery( + &self, + retry_regions: &[RegionId], + ) -> Result { + info!( + "Rediscovering datanodes for {} regions that need retry", + retry_regions.len() + ); + + let mut current_retry_regions: Vec = retry_regions.to_vec(); + let mut final_report = GcJobReport::default(); + let mut retry_round = 0; + + // Continue retrying until all regions succeed or reach max retry limit + while !current_retry_regions.is_empty() && retry_round < self.config.max_retries_per_region + { + retry_round += 1; + + info!( + "Starting retry round {}/{} for {} regions", + retry_round, + self.config.max_retries_per_region, + current_retry_regions.len() + ); + + // Step 1: Rediscover current datanodes for retry regions + let mut region_to_peer = HashMap::new(); + let mut peer_to_regions: HashMap> = HashMap::new(); + + for ®ion_id in ¤t_retry_regions { + let table_id = region_id.table_id(); + + match self.ctx.get_table_route(table_id).await { + Ok((_phy_table_id, table_route)) => { + // Find the region in the table route + let mut found = false; + for region_route in &table_route.region_routes { + if region_route.region.id == region_id { + if let Some(leader_peer) = ®ion_route.leader_peer { + region_to_peer.insert(region_id, leader_peer.clone()); + peer_to_regions + .entry(leader_peer.clone()) + .or_default() + .push(region_id); + found = true; + break; + } + } + } + + if !found { + warn!( + "Failed to find region {} in table route or no leader peer found in retry round {}", + region_id, retry_round + ); + } + } + Err(e) => { + warn!( + "Failed to get table route for region {} (table {}) in retry round {}: {}", + region_id, table_id, retry_round, e + ); + } + } + } + + if peer_to_regions.is_empty() { + warn!( + "No valid datanodes found for retry regions in round {}", + retry_round + ); + break; + } + + info!( + "Rediscovered {} datanodes for retry regions in round {}", + peer_to_regions.len(), + retry_round + ); + + // Step 2: Process retry regions by calling gc_regions on rediscovered datanodes + let mut round_report = GcJobReport::default(); + + for (peer, regions) in peer_to_regions { + info!( + "Retrying GC for {} regions on datanode {} in round {}", + regions.len(), + peer, + retry_round + ); + + // Get fresh file references for these regions + let file_refs_manifest = match self + .ctx + .get_file_references(®ions, ®ion_to_peer, self.config.mailbox_timeout) + .await + { + Ok(manifest) => manifest, + Err(e) => { + error!( + "Failed to get file references for retry regions on datanode {} in round {}: {}", + peer, retry_round, e + ); + continue; + } + }; + + // Process all regions on this datanode + let mut successful_regions = 0; + + // FIXME: batch and send to peers + for region_id in ®ions { + let should_full_listing = self.should_use_full_listing(*region_id).await; + match self + .ctx + .gc_regions( + peer.clone(), + &[*region_id], + &file_refs_manifest, + should_full_listing, // Don't use full listing for retry + self.config.mailbox_timeout, + ) + .await + { + Ok(report) => { + successful_regions += report.deleted_files.len(); + final_report + .per_datanode_reports + .entry(peer.id) + .or_default() + .merge(report.clone()); + + // Update GC tracker for successful retry + self.update_full_listing_time(*region_id, should_full_listing) + .await; + } + Err(e) => { + error!( + "Failed to retry GC for region {} on datanode {} in round {}: {}", + region_id, peer, retry_round, e + ); + final_report + .per_datanode_reports + .entry(peer.id) + .or_default() + .need_retry_regions + .insert(*region_id); + } + } + } + + info!( + "Completed retry GC for datanode {} in round {}: {} regions processed, {} successful", + peer, + retry_round, + regions.len(), + successful_regions + ); + } + + if !current_retry_regions.is_empty() && retry_round < self.config.max_retries_per_region + { + // Calculate exponential backoff: base_duration * 2^(retry_round - 1) + let backoff_multiplier = 2_u32.pow(retry_round.saturating_sub(1) as u32); + let backoff_duration = self.config.retry_backoff_duration * backoff_multiplier; + + info!( + "{} regions still need retry after round {}, waiting {} seconds before next round (exponential backoff)", + current_retry_regions.len(), + retry_round, + backoff_duration.as_secs() + ); + + // Wait for backoff period before next retry round + sleep(backoff_duration).await; + } + } + + Ok(final_report) + } } diff --git a/src/meta-srv/src/gc/mock.rs b/src/meta-srv/src/gc/mock.rs index 3c1f761d3a8f..c6bbbe300edf 100644 --- a/src/meta-srv/src/gc/mock.rs +++ b/src/meta-srv/src/gc/mock.rs @@ -100,9 +100,8 @@ impl MockSchedulerCtx { } /// Set an error to be returned by `get_table_route` - pub fn with_get_table_route_error(self, error: crate::error::Error) -> Self { + pub fn set_table_route_error(&self, error: crate::error::Error) { *self.get_table_route_error.lock().unwrap() = Some(error); - self } /// Set an error to be returned by `get_file_references` @@ -216,27 +215,29 @@ impl SchedulerCtx for MockSchedulerCtx { async fn gc_regions( &self, _peer: Peer, - region_id: RegionId, + region_ids: &[RegionId], _file_refs_manifest: &FileRefsManifest, _full_file_listing: bool, _timeout: Duration, ) -> Result { *self.gc_regions_calls.lock().unwrap() += 1; - // Track retry count for this region - { + // Track retry count for all regions (use the first region as representative) + if let Some(&first_region_id) = region_ids.first() { let mut retry_count = self.gc_regions_retry_count.lock().unwrap(); - *retry_count.entry(region_id).or_insert(0) += 1; + *retry_count.entry(first_region_id).or_insert(0) += 1; } - // Check per-region error injection first - if let Some(error) = self - .gc_regions_per_region_errors - .lock() - .unwrap() - .remove(®ion_id) - { - return Err(error); + // Check per-region error injection first (for any region) + for ®ion_id in region_ids { + if let Some(error) = self + .gc_regions_per_region_errors + .lock() + .unwrap() + .remove(®ion_id) + { + return Err(error); + } } // Check if we should return an injected error @@ -253,38 +254,46 @@ impl SchedulerCtx for MockSchedulerCtx { } } - // Handle success after specific number of retries - { + // Handle success after specific number of retries (use first region as representative) + if let Some(&first_region_id) = region_ids.first() { let retry_count = self .gc_regions_retry_count .lock() .unwrap() - .get(®ion_id) + .get(&first_region_id) .copied() .unwrap_or(0); let success_after_retries = self.gc_regions_success_after_retries.lock().unwrap(); - if let Some(&required_retries) = success_after_retries.get(®ion_id) + if let Some(&required_retries) = success_after_retries.get(&first_region_id) && retry_count <= required_retries { // Return retryable error until we reach the required retry count return Err(crate::error::RetryLaterSnafu { reason: format!( "Mock retryable error for region {} (attempt {}/{})", - region_id, retry_count, required_retries + first_region_id, retry_count, required_retries ), } .build()); } } - self.gc_reports - .lock() - .unwrap() - .get(®ion_id) - .cloned() - .with_context(|| UnexpectedSnafu { - violated: format!("No corresponding gc report for {}", region_id), - }) + // Collect and merge reports for all requested regions + let mut combined_report = GcReport::default(); + let gc_reports = self.gc_reports.lock().unwrap(); + + for ®ion_id in region_ids { + if let Some(report) = gc_reports.get(®ion_id) { + combined_report.merge(report.clone()); + } else { + return Err(UnexpectedSnafu { + violated: format!("No corresponding gc report for {}", region_id), + } + .build()); + } + } + + Ok(combined_report) } } diff --git a/src/meta-srv/src/gc/mock/basic.rs b/src/meta-srv/src/gc/mock/basic.rs index 4de93a18ffce..128803feb3b7 100644 --- a/src/meta-srv/src/gc/mock/basic.rs +++ b/src/meta-srv/src/gc/mock/basic.rs @@ -25,20 +25,19 @@ use crate::gc::mock::{MockSchedulerCtx, TestEnv, mock_region_stat, new_candidate use crate::gc::{GcScheduler, GcSchedulerOptions}; #[tokio::test] -async fn test_process_tables_concurrently_empty() { +async fn test_process_datanodes_concurrently_empty() { let env = TestEnv::new(); let report = env .scheduler - .process_tables_concurrently(HashMap::new()) + .process_datanodes_concurrently(HashMap::new()) .await; - assert_eq!(report.processed_tables, 0); - assert_eq!(report.table_reports.len(), 0); - assert!(report.table_reports.is_empty()); + assert_eq!(report.per_datanode_reports.len(), 0); + assert_eq!(report.failed_datanodes.len(), 0); } #[tokio::test] -async fn test_process_tables_concurrently_with_candidates() { +async fn test_process_datanodes_concurrently_with_candidates() { init_default_ut_logging(); let table_id = 1; @@ -66,31 +65,31 @@ async fn test_process_tables_concurrently_with_candidates() { } .with_table_routes(HashMap::from([( table_id, - (table_id, vec![(region_id, peer)]), + (table_id, vec![(region_id, peer.clone())]), )])); - let env = TestEnv::new().with_candidates(candidates); + let env = TestEnv::new(); // We need to replace the ctx with the one with gc_reports let mut scheduler = env.scheduler; scheduler.ctx = Arc::new(ctx); - let candidates = env - .ctx - .candidates - .lock() - .unwrap() - .clone() - .unwrap_or_default(); + let candidates = HashMap::from([(1, vec![new_candidate(region_id, 1.0)])]); - let report = scheduler.process_tables_concurrently(candidates).await; + // Convert table-based candidates to datanode-based candidates + let datanode_to_candidates = HashMap::from([( + peer, + candidates + .into_iter() + .flat_map(|(table_id, candidates)| candidates.into_iter().map(move |c| (table_id, c))) + .collect(), + )]); - assert_eq!(report.processed_tables, 1); - assert_eq!(report.table_reports.len(), 1); - assert_eq!( - report.table_reports[0].success_regions[0].deleted_files[®ion_id], - deleted_files - ); - assert!(report.table_reports[0].failed_regions.is_empty()); + let report = scheduler + .process_datanodes_concurrently(datanode_to_candidates) + .await; + + assert_eq!(report.per_datanode_reports.len(), 1); + assert_eq!(report.failed_datanodes.len(), 0); } #[tokio::test] @@ -141,19 +140,15 @@ async fn test_handle_tick() { let report = scheduler.handle_tick().await.unwrap(); // Validate the returned GcJobReport - assert_eq!(report.processed_tables, 1, "Should process 1 table"); - assert_eq!(report.table_reports.len(), 1, "Should have 1 table report"); - - let table_report = &report.table_reports[0]; - assert_eq!(table_report.table_id, table_id, "Table ID should match"); assert_eq!( - table_report.success_regions.len(), + report.per_datanode_reports.len(), 1, - "Should have 1 successful region" + "Should process 1 datanode" ); - assert!( - table_report.failed_regions.is_empty(), - "Should have no failed regions" + assert_eq!( + report.failed_datanodes.len(), + 0, + "Should have 0 failed datanodes" ); assert_eq!(*ctx.get_table_to_region_stats_calls.lock().unwrap(), 1); diff --git a/src/meta-srv/src/gc/mock/con.rs b/src/meta-srv/src/gc/mock/con.rs index b222b3f62513..04609e844738 100644 --- a/src/meta-srv/src/gc/mock/con.rs +++ b/src/meta-srv/src/gc/mock/con.rs @@ -81,10 +81,24 @@ async fn test_concurrent_table_processing_limits() { }; let candidates = ctx.candidates.lock().unwrap().clone().unwrap_or_default(); - let report = scheduler.process_tables_concurrently(candidates).await; - // Should process all tables but respect concurrency limits - assert_eq!(report.processed_tables, 10); + // Convert table-based candidates to datanode-based candidates + let peer = Peer::new(1, ""); + let datanode_to_candidates = HashMap::from([( + peer, + candidates + .into_iter() + .flat_map(|(table_id, candidates)| candidates.into_iter().map(move |c| (table_id, c))) + .collect(), + )]); + + let report = scheduler + .process_datanodes_concurrently(datanode_to_candidates) + .await; + + // Should process all datanodes + assert_eq!(report.per_datanode_reports.len(), 1); + assert_eq!(report.failed_datanodes.len(), 0); } #[tokio::test] @@ -140,28 +154,35 @@ async fn test_mixed_success_failure_tables() { let candidates = ctx.candidates.lock().unwrap().clone().unwrap_or_default(); - let report = scheduler.process_tables_concurrently(candidates).await; + // Convert table-based candidates to datanode-based candidates + let peer = Peer::new(1, ""); + let datanode_to_candidates = HashMap::from([( + peer, + candidates + .into_iter() + .flat_map(|(table_id, candidates)| candidates.into_iter().map(move |c| (table_id, c))) + .collect(), + )]); - // Should have one success and one failure - assert_eq!(report.processed_tables, 2); - assert_eq!(report.table_reports.len(), 2); - assert_eq!(report.failed_tables.len(), 0); - assert_eq!( - report - .table_reports - .iter() - .map(|r| r.success_regions.len()) - .sum::(), - 1 - ); + let report = scheduler + .process_datanodes_concurrently(datanode_to_candidates) + .await; + + // Should have one datanode with mixed results + assert_eq!(report.per_datanode_reports.len(), 1); + // also check one failed region assert_eq!( report - .table_reports + .per_datanode_reports .iter() - .map(|r| r.failed_regions.len()) - .sum::(), + .next() + .unwrap() + .1 + .need_retry_regions + .len(), 1 ); + assert_eq!(report.failed_datanodes.len(), 0); } // Region Concurrency Tests @@ -239,15 +260,20 @@ async fn test_region_gc_concurrency_limit() { }; let start_time = Instant::now(); - let report = scheduler - .process_table_gc(table_id, candidates) + let reports = scheduler + .process_datanode_gc( + peer, + candidates.into_iter().map(|c| (table_id, c)).collect(), + ) .await .unwrap(); let duration = start_time.elapsed(); // All regions should be processed successfully - assert_eq!(report.success_regions.len(), 10); - assert_eq!(report.failed_regions.len(), 0); + assert_eq!(reports.len(), 1); + let table_report = &reports[0]; + assert_eq!(table_report.success_regions.len(), 10); + assert_eq!(table_report.failed_regions.len(), 0); // Verify that concurrency limit was respected (this is hard to test directly, // but we can verify that the processing completed successfully) @@ -340,18 +366,23 @@ async fn test_region_gc_concurrency_with_mixed_results() { last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), }; - let report = scheduler - .process_table_gc(table_id, candidates) + let reports = scheduler + .process_datanode_gc( + peer, + candidates.into_iter().map(|c| (table_id, c)).collect(), + ) .await .unwrap(); // Should have 3 successful and 3 failed regions + assert_eq!(reports.len(), 1); + let report = &reports[0]; assert_eq!(report.success_regions.len(), 3); assert_eq!(report.failed_regions.len(), 3); // Verify that successful regions are the even-numbered ones - for report in &report.success_regions { - for region in report.deleted_files.keys() { + for success_report in &report.success_regions { + for region in success_report.deleted_files.keys() { let region_num = region.region_number(); assert_eq!( region_num % 2, @@ -439,12 +470,17 @@ async fn test_region_gc_concurrency_with_retryable_errors() { ctx.set_gc_regions_success_after_retries(region_id, 1); } - let report = scheduler - .process_table_gc(table_id, candidates) + let reports = scheduler + .process_datanode_gc( + peer, + candidates.into_iter().map(|c| (table_id, c)).collect(), + ) .await .unwrap(); // All regions should eventually succeed after retries + assert_eq!(reports.len(), 1); + let report = &reports[0]; assert_eq!(report.success_regions.len(), 5); assert_eq!(report.failed_regions.len(), 0); diff --git a/src/meta-srv/src/gc/mock/err_handle.rs b/src/meta-srv/src/gc/mock/err_handle.rs index a4e1003ec258..7e558a61b7d4 100644 --- a/src/meta-srv/src/gc/mock/err_handle.rs +++ b/src/meta-srv/src/gc/mock/err_handle.rs @@ -87,22 +87,14 @@ async fn test_gc_regions_failure_handling() { // Validate the report shows the failure handling assert_eq!( - report.processed_tables, 1, - "Should process 1 table despite failure" - ); - assert_eq!(report.table_reports.len(), 1, "Should have 1 table report"); - - let table_report = &report.table_reports[0]; - assert_eq!(table_report.table_id, table_id, "Table ID should match"); - assert_eq!( - table_report.success_regions.len(), - 0, - "Should have 0 successful regions due to GC failure" + report.per_datanode_reports.len(), + 1, + "Should process 1 datanode despite failure" ); assert_eq!( - table_report.failed_regions.len(), + report.failed_datanodes.len(), 1, - "Should have 1 failed region" + "Should have 1 failed datanode" ); // Verify that calls were made despite potential failures @@ -166,9 +158,17 @@ async fn test_get_file_references_failure() { let report = scheduler.handle_tick().await.unwrap(); // Validate the report shows the expected results - assert_eq!(report.processed_tables, 1, "Should process 1 table"); - assert_eq!(report.table_reports.len(), 0, "Should have 0 table report"); - assert_eq!(report.failed_tables.len(), 1, "Should have 1 failed table"); + // In the new implementation, even if get_file_references fails, we still create a datanode report + assert_eq!( + report.per_datanode_reports.len(), + 1, + "Should process 1 datanode" + ); + assert_eq!( + report.failed_datanodes.len(), + 1, + "Should have 1 failed datanode" + ); // Should still attempt to get file references assert_eq!( @@ -203,13 +203,11 @@ async fn test_get_table_route_failure() { .build(); // Create context with table route error injection - let ctx = Arc::new( - MockSchedulerCtx { - table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), - ..Default::default() - } - .with_get_table_route_error(route_error), - ); + let ctx = Arc::new(MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + ..Default::default() + }); + ctx.set_table_route_error(route_error); let scheduler = GcScheduler { ctx: ctx.clone(), @@ -228,14 +226,29 @@ async fn test_get_table_route_failure() { .unwrap_or_default(); let candidates = scheduler.select_gc_candidates(stats).await.unwrap(); + // Convert table-based candidates to datanode-based candidates + let datanode_to_candidates = HashMap::from([( + Peer::new(1, ""), + candidates + .into_iter() + .flat_map(|(table_id, candidates)| candidates.into_iter().map(move |c| (table_id, c))) + .collect(), + )]); + // This should handle table route failure gracefully - let report = scheduler.process_tables_concurrently(candidates).await; + let report = scheduler + .process_datanodes_concurrently(datanode_to_candidates) + .await; - // Should process the table but fail due to route error - assert_eq!(report.processed_tables, 1, "Expected 1 processed table"); + // Should process the datanode but fail due to route error assert_eq!( - report.table_reports.len(), - 0, - "Expected 0 successful tables due to route error" + report.per_datanode_reports.len(), + 1, + "Expected 1 datanode report" + ); + assert_eq!( + report.failed_datanodes.len(), + 1, + "Expected 1 failed datanode due to route error" ); } diff --git a/src/meta-srv/src/gc/mock/full_list.rs b/src/meta-srv/src/gc/mock/full_list.rs index b5df49b8f972..1943fd1dd078 100644 --- a/src/meta-srv/src/gc/mock/full_list.rs +++ b/src/meta-srv/src/gc/mock/full_list.rs @@ -56,7 +56,7 @@ async fn test_full_file_listing_first_time_gc() { } .with_table_routes(HashMap::from([( table_id, - (table_id, vec![(region_id, peer)]), + (table_id, vec![(region_id, peer.clone())]), )])), ); @@ -75,12 +75,12 @@ async fn test_full_file_listing_first_time_gc() { }; // First GC - should use full listing since region has never been GC'd - let report = scheduler - .process_table_gc(table_id, vec![mock_candidate(region_id)]) + let reports = scheduler + .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) .await .unwrap(); - assert_eq!(report.success_regions.len(), 1); + assert_eq!(reports.deleted_files.len(), 1); // Verify that full listing was used by checking the tracker let tracker = scheduler.region_gc_tracker.lock().await; @@ -123,7 +123,7 @@ async fn test_full_file_listing_interval_enforcement() { } .with_table_routes(HashMap::from([( table_id, - (table_id, vec![(region_id, peer)]), + (table_id, vec![(region_id, peer.clone())]), )])), ); @@ -142,11 +142,11 @@ async fn test_full_file_listing_interval_enforcement() { }; // First GC - should use full listing - let report1 = scheduler - .process_table_gc(table_id, vec![mock_candidate(region_id)]) + let reports1 = scheduler + .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) .await .unwrap(); - assert_eq!(report1.success_regions.len(), 1); + assert_eq!(reports1.deleted_files.len(), 1); // Get the first full listing time let first_full_listing_time = { @@ -163,11 +163,10 @@ async fn test_full_file_listing_interval_enforcement() { tokio::time::sleep(Duration::from_millis(150)).await; // Second GC - should use full listing again since interval has passed - let report2 = scheduler - .process_table_gc(table_id, vec![mock_candidate(region_id)]) + let _reports2 = scheduler + .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) .await .unwrap(); - assert_eq!(report2.success_regions.len(), 1); // Verify that full listing was used again let tracker = scheduler.region_gc_tracker.lock().await; @@ -214,7 +213,7 @@ async fn test_full_file_listing_no_interval_passed() { } .with_table_routes(HashMap::from([( table_id, - (table_id, vec![(region_id, peer)]), + (table_id, vec![(region_id, peer.clone())]), )])), ); @@ -233,11 +232,11 @@ async fn test_full_file_listing_no_interval_passed() { }; // First GC - should use full listing - let report1 = scheduler - .process_table_gc(table_id, vec![mock_candidate(region_id)]) + let reports1 = scheduler + .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) .await .unwrap(); - assert_eq!(report1.success_regions.len(), 1); + assert_eq!(reports1.deleted_files.len(), 1); // Get the first full listing time let first_full_listing_time = { @@ -251,11 +250,11 @@ async fn test_full_file_listing_no_interval_passed() { }; // Second GC immediately - should NOT use full listing since interval hasn't passed - let report2 = scheduler - .process_table_gc(table_id, vec![mock_candidate(region_id)]) + let reports2 = scheduler + .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) .await .unwrap(); - assert_eq!(report2.success_regions.len(), 1); + assert_eq!(reports2.deleted_files.len(), 1); // Verify that full listing time was NOT updated let tracker = scheduler.region_gc_tracker.lock().await; diff --git a/src/meta-srv/src/gc/mock/misc.rs b/src/meta-srv/src/gc/mock/misc.rs index 9c92f0b6ab47..1ecea700f87e 100644 --- a/src/meta-srv/src/gc/mock/misc.rs +++ b/src/meta-srv/src/gc/mock/misc.rs @@ -59,9 +59,22 @@ async fn test_empty_file_refs_manifest() { let candidates = ctx.candidates.lock().unwrap().clone().unwrap_or_default(); - let report = scheduler.process_tables_concurrently(candidates).await; + // Convert table-based candidates to datanode-based candidates + let peer = Peer::new(1, ""); + let datanode_to_candidates = HashMap::from([( + peer, + candidates + .into_iter() + .flat_map(|(table_id, candidates)| candidates.into_iter().map(move |c| (table_id, c))) + .collect(), + )]); + + let report = scheduler + .process_datanodes_concurrently(datanode_to_candidates) + .await; - assert_eq!(report.processed_tables, 1); + assert_eq!(report.per_datanode_reports.len(), 1); + assert_eq!(report.failed_datanodes.len(), 0); // Should handle empty file refs gracefully } @@ -124,12 +137,20 @@ async fn test_multiple_regions_per_table() { let candidates = ctx.candidates.lock().unwrap().clone().unwrap_or_default(); - let report = scheduler.process_tables_concurrently(candidates).await; + // Convert table-based candidates to datanode-based candidates + let peer = Peer::new(1, ""); + let datanode_to_candidates = HashMap::from([( + peer, + candidates + .into_iter() + .flat_map(|(table_id, candidates)| candidates.into_iter().map(move |c| (table_id, c))) + .collect(), + )]); - assert_eq!(report.processed_tables, 1); - assert_eq!(report.table_reports.len(), 1); + let report = scheduler + .process_datanodes_concurrently(datanode_to_candidates) + .await; - // Should have processed all 3 regions - let table_report = &report.table_reports[0]; - assert_eq!(table_report.success_regions.len(), 3); + assert_eq!(report.per_datanode_reports.len(), 1); + assert_eq!(report.failed_datanodes.len(), 0); } diff --git a/src/meta-srv/src/gc/mock/retry.rs b/src/meta-srv/src/gc/mock/retry.rs index c06a2af5c166..0e220c838036 100644 --- a/src/meta-srv/src/gc/mock/retry.rs +++ b/src/meta-srv/src/gc/mock/retry.rs @@ -55,7 +55,7 @@ async fn test_retry_logic_with_retryable_errors() { } .with_table_routes(HashMap::from([( table_id, - (table_id, vec![(region_id, peer)]), + (table_id, vec![(region_id, peer.clone())]), )])), ); @@ -79,12 +79,14 @@ async fn test_retry_logic_with_retryable_errors() { ctx.set_gc_regions_success_after_retries(region_id, 2); let start_time = Instant::now(); - let report = scheduler - .process_table_gc(table_id, vec![mock_candidate(region_id)]) + let reports = scheduler + .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) .await .unwrap(); let duration = start_time.elapsed(); + assert_eq!(reports.len(), 1); + let report = &reports[0]; assert_eq!(report.success_regions.len(), 1); assert_eq!(report.failed_regions.len(), 0); assert_eq!(ctx.get_retry_count(region_id), 3); // 2 retries + 1 success @@ -100,11 +102,13 @@ async fn test_retry_logic_with_retryable_errors() { ctx.reset_retry_tracking(); ctx.set_gc_regions_success_after_retries(region_id, 5); // More than max_retries_per_region (3) - let report = scheduler - .process_table_gc(table_id, vec![mock_candidate(region_id)]) + let reports = scheduler + .process_datanode_gc(peer, vec![(table_id, mock_candidate(region_id))]) .await .unwrap(); + assert_eq!(reports.len(), 1); + let report = &reports[0]; assert_eq!(report.success_regions.len(), 0); assert_eq!(report.failed_regions.len(), 1); assert_eq!(ctx.get_retry_count(region_id), 3); // Should stop at max_retries_per_region @@ -140,7 +144,7 @@ async fn test_retry_logic_with_error_sequence() { } .with_table_routes(HashMap::from([( table_id, - (table_id, vec![(region_id, peer)]), + (table_id, vec![(region_id, peer.clone())]), )])), ); @@ -169,11 +173,13 @@ async fn test_retry_logic_with_error_sequence() { ctx.reset_retry_tracking(); ctx.set_gc_regions_error_sequence(vec![retryable_error1, retryable_error2]); - let report = scheduler - .process_table_gc(table_id, vec![mock_candidate(region_id)]) + let reports = scheduler + .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) .await .unwrap(); + assert_eq!(reports.len(), 1); + let report = &reports[0]; assert_eq!(report.success_regions.len(), 1); assert_eq!(report.failed_regions.len(), 0); assert_eq!(ctx.get_retry_count(region_id), 3); // 2 errors + 1 success @@ -203,7 +209,7 @@ async fn test_retry_logic_non_retryable_error() { } .with_table_routes(HashMap::from([( table_id, - (table_id, vec![(region_id, peer)]), + (table_id, vec![(region_id, peer.clone())]), )])), ); @@ -229,12 +235,356 @@ async fn test_retry_logic_non_retryable_error() { .unwrap() .replace(non_retryable_error); - let report = scheduler - .process_table_gc(table_id, vec![mock_candidate(region_id)]) + let reports = scheduler + .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) .await .unwrap(); + assert_eq!(reports.len(), 1); + let report = &reports[0]; assert_eq!(report.success_regions.len(), 0); assert_eq!(report.failed_regions.len(), 1); assert_eq!(ctx.get_retry_count(region_id), 1); // Only 1 attempt, no retries } + +/// Test need_retry_regions functionality with multi-round retry +#[tokio::test] +async fn test_need_retry_regions_multi_round() { + init_default_ut_logging(); + + let table_id = 1; + let region_id1 = RegionId::new(table_id, 1); + let region_id2 = RegionId::new(table_id, 2); + let peer1 = Peer::new(1, ""); + let peer2 = Peer::new(2, ""); + + let region_stat1 = mock_region_stat(region_id1, RegionRole::Leader, 200_000_000, 10); + let region_stat2 = mock_region_stat(region_id2, RegionRole::Leader, 150_000_000, 8); + let table_stats = HashMap::from([(table_id, vec![region_stat1, region_stat2])]); + + // Create GC reports with need_retry_regions + let gc_report1 = GcReport { + deleted_files: HashMap::from([(region_id1, vec![FileId::random()])]), + need_retry_regions: std::collections::HashSet::from([region_id1]), // This region needs retry + }; + + let gc_report2 = GcReport { + deleted_files: HashMap::from([(region_id2, vec![FileId::random()])]), + need_retry_regions: std::collections::HashSet::new(), // This region succeeds + }; + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id1, 1), (region_id2, 1)]), + ..Default::default() + }; + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + gc_reports: Arc::new(Mutex::new(HashMap::from([ + (region_id1, gc_report1), + (region_id2, gc_report2), + ]))), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + ( + table_id, + vec![(region_id1, peer1.clone()), (region_id2, peer2.clone())], + ), + )])), + ); + + let config = GcSchedulerOptions { + max_retries_per_region: 3, + retry_backoff_duration: Duration::from_millis(100), + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + // Set up success after 1 retry for region1 (which needs retry) + ctx.reset_retry_tracking(); + ctx.set_gc_regions_success_after_retries(region_id1, 1); + + let start_time = Instant::now(); + // Use peer1 since both regions belong to the same table and we're testing single datanode scenario + let reports = scheduler + .process_datanode_gc(peer1.clone(), vec![(table_id, mock_candidate(region_id1))]) + .await + .unwrap(); + let reports2 = scheduler + .process_datanode_gc(peer2.clone(), vec![(table_id, mock_candidate(region_id2))]) + .await + .unwrap(); + let duration = start_time.elapsed(); + + // Verify results + assert_eq!(reports.len(), 1); + let report = &reports[0]; + assert_eq!(report.success_regions.len(), 2); // Both regions should succeed eventually + assert_eq!(report.failed_regions.len(), 0); + + // Verify retry count for region1 (should be 2: 1 initial failure + 1 retry success) + assert_eq!(ctx.get_retry_count(region_id1), 2); + // region2 should only be processed once + assert_eq!(ctx.get_retry_count(region_id2), 1); + + // Verify exponential backoff was applied (should take at least 100ms for 1 retry round) + assert!( + duration >= Duration::from_millis(100), + "Expected backoff duration not met: {:?}", + duration + ); +} + +/// Test need_retry_regions with exponential backoff across multiple rounds +#[tokio::test] +async fn test_need_retry_regions_exponential_backoff() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + + let region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); + let table_stats = HashMap::from([(table_id, vec![region_stat])]); + + // Create GC report that will need multiple retry rounds + let gc_report = GcReport { + deleted_files: HashMap::from([(region_id, vec![FileId::random()])]), + need_retry_regions: std::collections::HashSet::from([region_id]), + }; + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id, 1)]), + ..Default::default() + }; + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + gc_reports: Arc::new(Mutex::new(HashMap::from([(region_id, gc_report)]))), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer.clone())]), + )])), + ); + + let config = GcSchedulerOptions { + max_retries_per_region: 4, + retry_backoff_duration: Duration::from_millis(50), // Short base duration for testing + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + // Set up success after 3 retries (so 4 total rounds: initial + 3 retries) + ctx.reset_retry_tracking(); + ctx.set_gc_regions_success_after_retries(region_id, 3); + + let start_time = Instant::now(); + let reports = scheduler + .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) + .await + .unwrap(); + let duration = start_time.elapsed(); + + // Verify results + assert_eq!(reports.len(), 1); + let report = &reports[0]; + assert_eq!(report.success_regions.len(), 1); + assert_eq!(report.failed_regions.len(), 0); + assert_eq!(ctx.get_retry_count(region_id), 4); // 1 initial + 3 retries + + // Verify exponential backoff was applied + // Expected backoff: 50ms + 100ms + 200ms = 350ms minimum + assert!( + duration >= Duration::from_millis(350), + "Expected exponential backoff duration not met: {:?}", + duration + ); +} + +/// Test need_retry_regions with table route rediscovery failure +#[tokio::test] +async fn test_need_retry_regions_table_route_failure() { + init_default_ut_logging(); + + let table_id = 1; + let region_id = RegionId::new(table_id, 1); + let peer = Peer::new(1, ""); + + let region_stat = mock_region_stat(region_id, RegionRole::Leader, 200_000_000, 10); + let table_stats = HashMap::from([(table_id, vec![region_stat])]); + + // Create GC report with need_retry_regions + let gc_report = GcReport { + deleted_files: HashMap::from([(region_id, vec![FileId::random()])]), + need_retry_regions: std::collections::HashSet::from([region_id]), + }; + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id, 1)]), + ..Default::default() + }; + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + gc_reports: Arc::new(Mutex::new(HashMap::from([(region_id, gc_report)]))), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([( + table_id, + (table_id, vec![(region_id, peer.clone())]), + )])), + ); + + // Inject table route error to simulate rediscovery failure + let table_route_error = crate::error::RegionRouteNotFoundSnafu { region_id }.build(); + ctx.set_table_route_error(table_route_error); + + let config = GcSchedulerOptions { + max_retries_per_region: 3, + retry_backoff_duration: Duration::from_millis(50), + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + let reports = scheduler + .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) + .await + .unwrap(); + + // The region should fail since table route rediscovery failed + assert_eq!(reports.len(), 1); + let report = &reports[0]; + assert_eq!(report.success_regions.len(), 0); + assert_eq!(report.failed_regions.len(), 1); +} + +/// Test need_retry_regions with mixed success/failure scenarios +#[tokio::test] +async fn test_need_retry_regions_mixed_scenarios() { + init_default_ut_logging(); + + let table_id = 1; + let region_id1 = RegionId::new(table_id, 1); // Will succeed after retry + let region_id2 = RegionId::new(table_id, 2); // Will fail after max retries + let region_id3 = RegionId::new(table_id, 3); // Will succeed immediately + let peer = Peer::new(1, ""); + + let region_stat1 = mock_region_stat(region_id1, RegionRole::Leader, 200_000_000, 10); + let region_stat2 = mock_region_stat(region_id2, RegionRole::Leader, 150_000_000, 8); + let region_stat3 = mock_region_stat(region_id3, RegionRole::Leader, 100_000_000, 5); + let table_stats = HashMap::from([(table_id, vec![region_stat1, region_stat2, region_stat3])]); + + // Create GC reports + let gc_report1 = GcReport { + deleted_files: HashMap::from([(region_id1, vec![FileId::random()])]), + need_retry_regions: std::collections::HashSet::from([region_id1]), + }; + + let gc_report2 = GcReport { + deleted_files: HashMap::from([(region_id2, vec![FileId::random()])]), + need_retry_regions: std::collections::HashSet::from([region_id2]), + }; + + let gc_report3 = GcReport { + deleted_files: HashMap::from([(region_id3, vec![FileId::random()])]), + need_retry_regions: std::collections::HashSet::new(), + }; + + let file_refs = FileRefsManifest { + manifest_version: HashMap::from([(region_id1, 1), (region_id2, 1), (region_id3, 1)]), + ..Default::default() + }; + + let routes = vec![ + (region_id1, peer.clone()), + (region_id2, peer.clone()), + (region_id3, peer), + ]; + + let ctx = Arc::new( + MockSchedulerCtx { + table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), + gc_reports: Arc::new(Mutex::new(HashMap::from([ + (region_id1, gc_report1), + (region_id2, gc_report2), + (region_id3, gc_report3), + ]))), + file_refs: Arc::new(Mutex::new(Some(file_refs))), + ..Default::default() + } + .with_table_routes(HashMap::from([(table_id, (table_id, routes.clone()))])), + ); + + let config = GcSchedulerOptions { + max_retries_per_region: 2, // Low limit to test failure case + retry_backoff_duration: Duration::from_millis(50), + ..Default::default() + }; + + let scheduler = GcScheduler { + ctx: ctx.clone(), + receiver: GcScheduler::channel().1, + config, + region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), + last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), + }; + + // Set up different retry scenarios + ctx.reset_retry_tracking(); + ctx.set_gc_regions_success_after_retries(region_id1, 1); // Succeed after 1 retry + ctx.set_gc_regions_success_after_retries(region_id2, 5); // Needs more retries than limit (2) + // region3 succeeds immediately (default behavior) + + // Process each region separately with its corresponding peer + let mut all_reports = Vec::new(); + for (region_id, peer) in routes { + let reports = scheduler + .process_datanode_gc(peer, vec![(table_id, mock_candidate(region_id))]) + .await + .unwrap(); + all_reports.extend(reports); + } + + // Combine results from all regions + let combined_report = all_reports.into_iter().next().unwrap_or_default(); + + // Verify mixed results + assert_eq!(combined_report.success_regions.len(), 2); // region1 and region3 succeed + assert_eq!(combined_report.failed_regions.len(), 1); // region2 fails due to retry limit + + // Verify retry counts + assert_eq!(ctx.get_retry_count(region_id1), 2); // 1 initial + 1 retry + assert_eq!(ctx.get_retry_count(region_id2), 2); // Stops at max_retries_per_region + assert_eq!(ctx.get_retry_count(region_id3), 1); // Succeeds immediately +} diff --git a/src/meta-srv/src/gc/scheduler.rs b/src/meta-srv/src/gc/scheduler.rs index 80aba861f100..35b5fdf7569b 100644 --- a/src/meta-srv/src/gc/scheduler.rs +++ b/src/meta-srv/src/gc/scheduler.rs @@ -47,9 +47,40 @@ pub struct TableGcReport { /// Report for a GC job. #[derive(Debug, Default)] pub struct GcJobReport { - pub processed_tables: usize, - pub table_reports: Vec, - pub failed_tables: HashMap, + pub per_datanode_reports: HashMap, + pub failed_datanodes: HashMap>, +} +impl GcJobReport { + pub fn merge(&mut self, mut other: GcJobReport) { + // merge per_datanode_reports&failed_datanodes + let all_dn_ids = self + .per_datanode_reports + .keys() + .cloned() + .chain(other.per_datanode_reports.keys().cloned()) + .collect::>(); + for dn_id in all_dn_ids { + let mut self_report = self.per_datanode_reports.entry(dn_id).or_default(); + self_report.merge( + other + .per_datanode_reports + .remove(&dn_id) + .unwrap_or_default(), + ); + } + let all_failed_dn_ids = self + .failed_datanodes + .keys() + .cloned() + .chain(other.failed_datanodes.keys().cloned()) + .collect::>(); + for dn_id in all_failed_dn_ids { + let entry = self.failed_datanodes.entry(dn_id).or_default(); + if let Some(other_errors) = other.failed_datanodes.remove(&dn_id) { + entry.extend(other_errors); + } + } + } } /// [`Event`] represents various types of events that can be processed by the gc ticker. @@ -157,41 +188,6 @@ impl GcScheduler { &self, per_table_candidates: HashMap>, ) -> GcJobReport { - let mut report = GcJobReport::default(); - - // Create a stream of table GC tasks with limited concurrency - let results: Vec<_> = futures::stream::iter( - per_table_candidates - .into_iter() - .filter(|(_, candidates)| !candidates.is_empty()), - ) - .map(|(table_id, candidates)| { - let scheduler = self; - async move { - ( - table_id, - scheduler.process_table_gc(table_id, candidates).await, - ) - } - }) - .buffer_unordered(self.config.max_concurrent_tables) - .collect() - .await; - - // Process all table GC results - for (table_id, result) in results { - report.processed_tables += 1; - match result { - Ok(table) => { - report.table_reports.push(table); - } - Err(e) => { - error!("Failed to process table GC: {:#?}", e); - report.failed_tables.insert(table_id, format!("{:#?}", e)); - } - } - } - - report + unimplemented!("TODO: remove this unused") } } diff --git a/src/meta-srv/src/gc/tracker.rs b/src/meta-srv/src/gc/tracker.rs index 7f128290602f..373cab98c4b6 100644 --- a/src/meta-srv/src/gc/tracker.rs +++ b/src/meta-srv/src/gc/tracker.rs @@ -103,4 +103,26 @@ impl GcScheduler { true } } + + pub(crate) async fn update_full_listing_time( + &self, + region_id: RegionId, + did_full_listing: bool, + ) { + let mut gc_tracker = self.region_gc_tracker.lock().await; + let now = Instant::now(); + + gc_tracker + .entry(region_id) + .and_modify(|info| { + if did_full_listing { + info.last_full_listing_time = Some(now); + } + info.last_gc_time = now; + }) + .or_insert_with(|| RegionGcInfo { + last_gc_time: now, + last_full_listing_time: if did_full_listing { Some(now) } else { None }, + }); + } } From fc8b01bde8d6370f565c085f7a3422b74a18c95a Mon Sep 17 00:00:00 2001 From: discord9 Date: Mon, 17 Nov 2025 18:24:49 +0800 Subject: [PATCH 05/10] fix: also get from follower peer Signed-off-by: discord9 --- src/meta-srv/src/gc/ctx.rs | 1 + src/meta-srv/src/gc/handler.rs | 733 +++++++++++++-------------------- src/meta-srv/src/gc/mock.rs | 80 ++-- 3 files changed, 324 insertions(+), 490 deletions(-) diff --git a/src/meta-srv/src/gc/ctx.rs b/src/meta-srv/src/gc/ctx.rs index cfeff1cc1059..4a160ef1b516 100644 --- a/src/meta-srv/src/gc/ctx.rs +++ b/src/meta-srv/src/gc/ctx.rs @@ -38,6 +38,7 @@ use crate::gc::Region2Peers; use crate::cluster::MetaPeerClientRef; use crate::error; use crate::error::{Result, TableMetadataManagerSnafu}; +use crate::gc::handler::Region2Peers; use crate::gc::procedure::GcRegionProcedure; use crate::handler::HeartbeatMailbox; use crate::service::mailbox::{Channel, MailboxRef}; diff --git a/src/meta-srv/src/gc/handler.rs b/src/meta-srv/src/gc/handler.rs index 5358cb340e22..ae11c231e201 100644 --- a/src/meta-srv/src/gc/handler.rs +++ b/src/meta-srv/src/gc/handler.rs @@ -12,25 +12,29 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::sync::Arc; use std::sync::atomic::{AtomicUsize, Ordering}; use std::time::Instant; +use common_meta::key::table_route::PhysicalTableRouteValue; use common_meta::peer::Peer; use common_telemetry::{debug, error, info, warn}; use futures::StreamExt; -use itertools::Itertools; use snafu::OptionExt as _; use store_api::storage::{FileRefsManifest, GcReport, RegionId}; use table::metadata::TableId; use tokio::time::sleep; -use crate::error::{self, RegionRouteNotFoundSnafu, Result, UnexpectedSnafu}; +use crate::error::{RegionRouteNotFoundSnafu, Result}; use crate::gc::candidate::GcCandidate; -use crate::gc::scheduler::{GcJobReport, GcScheduler, TableGcReport}; +use crate::gc::scheduler::{GcJobReport, GcScheduler}; use crate::gc::tracker::RegionGcInfo; +pub(crate) type Region2Peers = HashMap)>; + +pub(crate) type Peer2Regions = HashMap>; + impl GcScheduler { /// Iterate through all region stats, find region that might need gc, and send gc instruction to /// the corresponding datanode with improved parallel processing and retry logic. @@ -65,7 +69,7 @@ impl GcScheduler { // Step 4: Process datanodes concurrently with limited parallelism let report = self - .process_datanodes_concurrently(datanode_to_candidates) + .process_datanodes_with_retry(datanode_to_candidates) .await; let duration = start_time.elapsed(); @@ -80,164 +84,9 @@ impl GcScheduler { Ok(report) } - /// Process GC for a single table with all its candidate regions. - pub(crate) async fn process_table_gc( - &self, - table_id: TableId, - candidates: Vec, - ) -> Result { - info!( - "Starting GC for table {} with {} candidate regions", - table_id, - candidates.len() - ); - - let mut report = TableGcReport { - table_id, - ..Default::default() - }; - - // Step 1: Get table route information - // if is logic table, can simply pass. - let (phy_table_id, table_peer) = self.ctx.get_table_route(table_id).await?; - - if phy_table_id != table_id { - return Ok(report); - } - - let region_to_peer = table_peer - .region_routes - .iter() - .filter_map(|r| { - r.leader_peer - .as_ref() - .map(|peer| (r.region.id, peer.clone())) - }) - .collect::>(); - - // Step 2: Determine related regions for file reference fetching - let candidate_region_ids: Vec = candidates.iter().map(|c| c.region_id).collect(); - let related_region_ids = self.find_related_regions(&candidate_region_ids).await?; - - // Step 3: Get file references for related regions - let file_refs_manifest = self - .ctx - .get_file_references( - &related_region_ids, - ®ion_to_peer, - self.config.mailbox_timeout, - ) - .await?; - - // Step 4: Filter out candidates that don't have file references available - let total_candidates = candidates.len(); - let mut valid_candidates = Vec::new(); - for candidate in candidates { - // Check if we have file references for this region - if file_refs_manifest - .manifest_version - .contains_key(&candidate.region_id) - { - // Check if this peer's addr were successfully obtained - if region_to_peer.contains_key(&candidate.region_id) { - valid_candidates.push(candidate); - } else { - UnexpectedSnafu { - violated: format!("Missing peer info for region {}", candidate.region_id), - } - .fail()?; - } - } else { - error!( - "Missing file references entry for region {}", - candidate.region_id - ); - UnexpectedSnafu { - violated: format!( - "Missing file references entry for region {}", - candidate.region_id - ), - } - .fail()?; - } - } - - // Step 5: Process each valid candidate region with retry logic - let valid_candidates_count = valid_candidates.len(); - let successful_regions = Arc::new(AtomicUsize::new(0)); - let failed_regions = Arc::new(AtomicUsize::new(0)); - let reports = Arc::new(tokio::sync::Mutex::new(Vec::new())); - let err_reports = Arc::new(tokio::sync::Mutex::new(Vec::new())); - - let file_refs_manifest = Arc::new(file_refs_manifest); - let region_to_peer = Arc::new(region_to_peer); - - futures::stream::iter(valid_candidates) - .for_each_concurrent(Some(self.config.region_gc_concurrency), |candidate| { - let region_id = candidate.region_id; - let file_refs_manifest = file_refs_manifest.clone(); - let region_to_peer = region_to_peer.clone(); - let successful_regions = successful_regions.clone(); - let failed_regions = failed_regions.clone(); - let region_gc_tracker = self.region_gc_tracker.clone(); - let reports = reports.clone(); - let err_reports = err_reports.clone(); - - async move { - match self - .process_region_gc_with_retry( - candidate, - &file_refs_manifest, - ®ion_to_peer, - ) - .await - { - Ok((report, used_full_listing)) => { - reports.lock().await.push(report); - successful_regions.fetch_add(1, Ordering::Relaxed); - // Update GC tracker - let mut gc_tracker = region_gc_tracker.lock().await; - let now = Instant::now(); - let gc_info = gc_tracker - .entry(region_id) - .or_insert_with(|| RegionGcInfo::new(now)); - gc_info.last_gc_time = now; - if used_full_listing { - gc_info.last_full_listing_time = Some(now); - } - } - Err(e) => { - failed_regions.fetch_add(1, Ordering::Relaxed); - error!("Failed to GC region {} after all retries: {}", region_id, e); - // TODO: collect errors into table gc report - err_reports - .lock() - .await - .push((region_id, format!("{:#?}", e))); - } - } - } - }) - .await; - - let successful_regions = successful_regions.load(Ordering::Relaxed); - - info!( - "Completed GC for table {}: {}/{} regions successful ({} skipped due to missing file references)", - table_id, - successful_regions, - valid_candidates_count, - total_candidates - valid_candidates_count - ); - - report.success_regions = reports.lock().await.drain(..).collect(); - report.failed_regions = err_reports.lock().await.drain(..).collect(); - Ok(report) - } - /// Find related regions that might share files with the candidate regions. /// Currently returns the same regions since repartition is not implemented yet. - /// TODO(discord9): When repartition is implemented, this should also find parent/child regions + /// TODO(discord9): When repartition is implemented, this should also find src/dst regions /// that might share files with the candidate regions. pub(crate) async fn find_related_regions( &self, @@ -246,170 +95,6 @@ impl GcScheduler { Ok(candidate_region_ids.to_vec()) } - /// Process GC for a single region with retry logic. - /// Returns the GC report and a boolean indicating whether full file listing was used. - pub(crate) async fn process_region_gc_with_retry( - &self, - candidate: GcCandidate, - file_refs_manifest: &FileRefsManifest, - // TODO(discord9): maybe also refresh region_to_peer mapping if needed? - region_to_peer: &HashMap, - ) -> Result<(GcReport, bool)> { - let region_id = candidate.region_id; - - // TODO(discord9): Select the best peer based on GC load - // for now gc worker need to be run from datanode that hosts the region - // this limit might be lifted in the future - let mut peer = self.get_region_peer(region_id, region_to_peer)?; - - let mut retries = 0; - let mut current_manifest = file_refs_manifest.clone(); - // Final report for recording all deleted files - let mut final_report = GcReport::default(); - - // Determine if we should use full file listing for this region - let should_use_full_listing = self.should_use_full_listing(region_id).await; - let mut current_retry_round = 0; - - while current_retry_round <= self.config.max_retries_per_region { - current_retry_round += 1; - match self - .ctx - .gc_regions( - peer.clone(), - &[region_id], - ¤t_manifest, - should_use_full_listing, - self.config.mailbox_timeout, - ) - .await - { - Ok(report) => { - match self - .handle_gc_success(region_id, report, &mut final_report, region_to_peer) - .await? - { - None => return Ok((final_report, should_use_full_listing)), - Some(refreshed_manifest) => { - current_manifest = refreshed_manifest; - } - } - } - // Retryable errors: refresh file references and retry with backoff - Err(e) if e.is_retryable() => { - // TODO(discord9): might do it on table level instead - let (refreshed_manifest, refreshed_peer) = self - .handle_gc_retry(region_id, &mut retries, e, region_to_peer) - .await?; - current_manifest = refreshed_manifest; - peer = refreshed_peer; - } - Err(e) => { - error!( - "Non-retryable error during GC for region {}: {}", - region_id, e - ); - return Err(e); - } - } - } - unreachable!() - } - - /// Handle successful GC report, checking if retry is needed for outdated regions. - async fn handle_gc_success( - &self, - region_id: RegionId, - report: GcReport, - final_report: &mut GcReport, - region_to_peer: &HashMap, - ) -> Result> { - if report.need_retry_regions.is_empty() { - final_report.merge(report); - debug!( - "Successfully completed GC for region {} with report: {final_report:?}", - region_id - ); - // note that need_retry_regions should be empty here - // since no more outdated regions - final_report.need_retry_regions.clear(); - Ok(None) - } else { - // retry outdated regions if needed - let refreshed_manifest = self - .refresh_file_refs_for( - &report.need_retry_regions.clone().into_iter().collect_vec(), - region_to_peer, - ) - .await?; - info!( - "Retrying GC for regions {:?} due to outdated file references", - &report.need_retry_regions - ); - final_report.merge(report); - Ok(Some(refreshed_manifest)) - } - } - - /// Handle retryable GC error with backoff and manifest refresh. - async fn handle_gc_retry( - &self, - region_id: RegionId, - retries: &mut usize, - error: error::Error, - region_to_peer: &HashMap, - ) -> Result<(FileRefsManifest, Peer)> { - *retries += 1; - if *retries >= self.config.max_retries_per_region { - error!( - "Failed to GC region {} after {} retries: {}", - region_id, retries, error - ); - return Err(error); - } - - warn!( - "GC failed for region {} (attempt {}/{}): {}. Retrying after backoff...", - region_id, retries, self.config.max_retries_per_region, error - ); - - // Wait for backoff period - sleep(self.config.retry_backoff_duration).await; - - let refreshed_manifest = self - .refresh_file_refs_for(&[region_id], region_to_peer) - .await?; - - // TODO(discord9): Select the best peer based on GC load - // for now gc worker need to be run from datanode that hosts the region - // this limit might be lifted in the future - let peer = self.get_region_peer(region_id, region_to_peer)?; - - Ok((refreshed_manifest, peer)) - } - - /// Refresh file references for related regions, typically used before retrying GC. - async fn refresh_file_refs_for( - &self, - regions: &[RegionId], - region_to_peer: &HashMap, - ) -> Result { - let related_regions = self.find_related_regions(regions).await?; - self.ctx - .get_file_references( - &related_regions, - region_to_peer, - self.config.mailbox_timeout, - ) - .await - .inspect_err(|e| { - error!( - "Failed to refresh file references for regions {:?}: {}", - related_regions, e - ); - }) - } - /// Get the peer for a given region. fn get_region_peer( &self, @@ -474,7 +159,7 @@ impl GcScheduler { } /// Process multiple datanodes concurrently with limited parallelism. - pub(crate) async fn process_datanodes_concurrently( + pub(crate) async fn process_datanodes_with_retry( &self, datanode_to_candidates: HashMap>, ) -> GcJobReport { @@ -562,14 +247,12 @@ impl GcScheduler { return Ok(Default::default()); } - // Extract all region IDs and map them to their peers - let region_to_peer: HashMap = candidates - .iter() - .map(|(_, candidate)| (candidate.region_id, peer.clone())) - .collect(); - let all_region_ids: Vec = candidates.iter().map(|(_, c)| c.region_id).collect(); + let (region_to_peer, _) = self + .discover_datanodes_for_regions(&all_region_ids, 0) + .await?; + // Step 1: Get file references for all regions on this datanode let file_refs_manifest = self .ctx @@ -582,7 +265,7 @@ impl GcScheduler { // Step 2: Create a single GcRegionProcedure for all regions on this datanode - let gc_report = { + let (gc_report, fullly_listed_regions) = { // Partition regions into full listing and fast listing in a single pass let mut need_full_list_regions = Vec::new(); let mut fast_list_regions = Vec::new(); @@ -651,8 +334,12 @@ impl GcScheduler { } } } + let fullly_listed_regions = need_full_list_regions + .into_iter() + .filter(|r| !combined_report.need_retry_regions.contains(r)) + .collect::>(); - combined_report + (combined_report, fullly_listed_regions) }; // Step 3: Process the combined GC report and update table reports @@ -664,7 +351,10 @@ impl GcScheduler { .entry(*region_id) .or_insert_with(|| RegionGcInfo::new(now)); gc_info.last_gc_time = now; - // TODO: Set last_full_listing_time if full listing was used + // notice need to set last_full_listing_time if full listing was used + if fullly_listed_regions.contains(region_id) { + gc_info.last_full_listing_time = Some(now); + } } info!( @@ -703,46 +393,10 @@ impl GcScheduler { current_retry_regions.len() ); - // Step 1: Rediscover current datanodes for retry regions - let mut region_to_peer = HashMap::new(); - let mut peer_to_regions: HashMap> = HashMap::new(); - - for ®ion_id in ¤t_retry_regions { - let table_id = region_id.table_id(); - - match self.ctx.get_table_route(table_id).await { - Ok((_phy_table_id, table_route)) => { - // Find the region in the table route - let mut found = false; - for region_route in &table_route.region_routes { - if region_route.region.id == region_id { - if let Some(leader_peer) = ®ion_route.leader_peer { - region_to_peer.insert(region_id, leader_peer.clone()); - peer_to_regions - .entry(leader_peer.clone()) - .or_default() - .push(region_id); - found = true; - break; - } - } - } - - if !found { - warn!( - "Failed to find region {} in table route or no leader peer found in retry round {}", - region_id, retry_round - ); - } - } - Err(e) => { - warn!( - "Failed to get table route for region {} (table {}) in retry round {}: {}", - region_id, table_id, retry_round, e - ); - } - } - } + // Step 1: Rediscover current datanodes for retry regions (batched for efficiency) + let (region_to_peer, peer_to_regions) = self + .discover_datanodes_for_regions(¤t_retry_regions, retry_round) + .await?; if peer_to_regions.is_empty() { warn!( @@ -758,76 +412,219 @@ impl GcScheduler { retry_round ); - // Step 2: Process retry regions by calling gc_regions on rediscovered datanodes - let mut round_report = GcJobReport::default(); + // Step 2: Process retry regions by calling gc_regions on rediscovered datanodes (batched) + let retry_result = self + .process_retry_regions_by_peers(region_to_peer, peer_to_regions, retry_round) + .await; - for (peer, regions) in peer_to_regions { - info!( - "Retrying GC for {} regions on datanode {} in round {}", - regions.len(), - peer, - retry_round - ); + // Merge the retry results into the final report + final_report.merge(retry_result); - // Get fresh file references for these regions - let file_refs_manifest = match self - .ctx - .get_file_references(®ions, ®ion_to_peer, self.config.mailbox_timeout) - .await - { - Ok(manifest) => manifest, - Err(e) => { - error!( - "Failed to get file references for retry regions on datanode {} in round {}: {}", - peer, retry_round, e + // Update the list of regions that still need retry + current_retry_regions = final_report + .per_datanode_reports + .values() + .flat_map(|r| r.need_retry_regions.iter().copied()) + .collect(); + + // Apply exponential backoff if there are still regions to retry + if !current_retry_regions.is_empty() && retry_round < self.config.max_retries_per_region + { + self.apply_retry_backoff(current_retry_regions.len(), retry_round) + .await; + } + } + + info!( + "Completed retry processing: {} rounds, {} regions processed", + retry_round, + retry_regions.len() - current_retry_regions.len() + ); + + Ok(final_report) + } + + /// Discover datanodes for the given regions by fetching table routes in batches. + /// Returns mappings from region to peer(leader, Vec) and peer to regions. + async fn discover_datanodes_for_regions( + &self, + regions: &[RegionId], + retry_round: usize, + ) -> Result<(Region2Peers, Peer2Regions)> { + let mut region_to_peer = HashMap::new(); + let mut peer_to_regions = HashMap::new(); + + // Group regions by table ID for batch processing + let mut table_to_regions: HashMap> = HashMap::new(); + for ®ion_id in regions { + let table_id = region_id.table_id(); + table_to_regions + .entry(table_id) + .or_default() + .push(region_id); + } + + // Process each table's regions together for efficiency + for (table_id, table_regions) in table_to_regions { + match self.ctx.get_table_route(table_id).await { + Ok((_phy_table_id, table_route)) => { + self.process_table_regions_for_rediscovery( + &table_route, + &table_regions, + &mut region_to_peer, + &mut peer_to_regions, + retry_round, + ); + } + Err(e) => { + warn!( + "Failed to get table route for table {} in retry round {}: {}", + table_id, retry_round, e + ); + // Continue with other tables instead of failing completely + continue; + } + } + } + + Ok((region_to_peer, peer_to_regions)) + } + + /// Process regions for a single table to find their current leader peers. + fn process_table_regions_for_rediscovery( + &self, + table_route: &PhysicalTableRouteValue, + table_regions: &[RegionId], + region_to_peer: &mut Region2Peers, + peer_to_regions: &mut Peer2Regions, + retry_round: usize, + ) { + for ®ion_id in table_regions { + let mut found = false; + + // Find the region in the table route + for region_route in &table_route.region_routes { + if region_route.region.id == region_id { + if let Some(leader_peer) = ®ion_route.leader_peer { + region_to_peer.insert( + region_id, + (leader_peer.clone(), region_route.follower_peers.clone()), ); - continue; - } - }; - - // Process all regions on this datanode - let mut successful_regions = 0; - - // FIXME: batch and send to peers - for region_id in ®ions { - let should_full_listing = self.should_use_full_listing(*region_id).await; - match self - .ctx - .gc_regions( - peer.clone(), - &[*region_id], - &file_refs_manifest, - should_full_listing, // Don't use full listing for retry - self.config.mailbox_timeout, - ) - .await - { - Ok(report) => { - successful_regions += report.deleted_files.len(); - final_report - .per_datanode_reports - .entry(peer.id) - .or_default() - .merge(report.clone()); - - // Update GC tracker for successful retry - self.update_full_listing_time(*region_id, should_full_listing) - .await; - } - Err(e) => { - error!( - "Failed to retry GC for region {} on datanode {} in round {}: {}", - region_id, peer, retry_round, e - ); - final_report - .per_datanode_reports - .entry(peer.id) - .or_default() - .need_retry_regions - .insert(*region_id); - } + peer_to_regions + .entry(leader_peer.clone()) + .or_default() + .insert(region_id); + found = true; + break; } } + } + + if !found { + warn!( + "Failed to find region {} in table route or no leader peer found in retry round {}", + region_id, retry_round + ); + } + } + } + + /// Process retry regions by sending batched GC requests to their respective datanodes. + async fn process_retry_regions_by_peers( + &self, + region_to_peer: Region2Peers, + peer_to_regions: Peer2Regions, + retry_round: usize, + ) -> GcJobReport { + let mut round_report = GcJobReport::default(); + + for (peer, regions) in peer_to_regions { + info!( + "Retrying GC for {} regions on datanode {} in round {}", + regions.len(), + peer, + retry_round + ); + + let all_related_regions = self + .find_related_regions(®ions.iter().cloned().collect::>()) + .await + .unwrap_or_default(); + + // Get fresh file references for these regions + let file_refs_manifest = match self + .ctx + .get_file_references( + &all_related_regions, + ®ion_to_peer, + self.config.mailbox_timeout, + ) + .await + { + Ok(manifest) => manifest, + Err(e) => { + error!( + "Failed to get file references for retry regions on datanode {} in round {}: {}", + peer, retry_round, e + ); + continue; + } + }; + + // Process all regions on this datanode in a single batch + let peer_report = self + .process_regions_batch_on_peer( + peer.clone(), + regions, + file_refs_manifest, + retry_round, + ) + .await; + + // Merge the peer's report into the round report + if let Some(report) = peer_report { + round_report + .per_datanode_reports + .entry(peer.id) + .or_default() + .merge(report); + } + } + + round_report + } + + /// Process a batch of regions on a single peer. + async fn process_regions_batch_on_peer( + &self, + peer: Peer, + regions: HashSet, + file_refs_manifest: FileRefsManifest, + retry_round: usize, + ) -> Option { + let regions = regions.into_iter().collect::>(); + // Determine if any region in the batch needs full listing + let should_full_listing = self.should_use_full_listing_for_batch(®ions).await; + + match self + .ctx + .gc_regions( + peer.clone(), + ®ions, + &file_refs_manifest, + should_full_listing, + self.config.mailbox_timeout, + ) + .await + { + Ok(report) => { + let successful_regions = report.deleted_files.len(); + + // Update GC tracker for all successful regions + for ®ion_id in ®ions { + self.update_full_listing_time(region_id, should_full_listing) + .await; + } info!( "Completed retry GC for datanode {} in round {}: {} regions processed, {} successful", @@ -836,26 +633,54 @@ impl GcScheduler { regions.len(), successful_regions ); - } - if !current_retry_regions.is_empty() && retry_round < self.config.max_retries_per_region - { - // Calculate exponential backoff: base_duration * 2^(retry_round - 1) - let backoff_multiplier = 2_u32.pow(retry_round.saturating_sub(1) as u32); - let backoff_duration = self.config.retry_backoff_duration * backoff_multiplier; - - info!( - "{} regions still need retry after round {}, waiting {} seconds before next round (exponential backoff)", - current_retry_regions.len(), + Some(report) + } + Err(e) => { + error!( + "Failed to retry GC for {} regions on datanode {} in round {}: {}", + regions.len(), + peer, retry_round, - backoff_duration.as_secs() + e ); - // Wait for backoff period before next retry round - sleep(backoff_duration).await; + // Mark all regions in this batch as needing retry + let mut failed_report = GcReport::default(); + for region_id in regions { + failed_report.need_retry_regions.insert(region_id); + } + + Some(failed_report) } } + } - Ok(final_report) + /// Determine if full file listing should be used for any region in the batch. + async fn should_use_full_listing_for_batch(&self, regions: &[RegionId]) -> bool { + // Use full listing if any region in the batch needs it + for ®ion_id in regions { + if self.should_use_full_listing(region_id).await { + return true; + } + } + false + } + + /// Apply exponential backoff before the next retry round. + async fn apply_retry_backoff(&self, remaining_regions: usize, retry_round: usize) { + // Calculate exponential backoff: base_duration * 2^(retry_round - 1) + let backoff_multiplier = 2_u32.pow(retry_round.saturating_sub(1) as u32); + let backoff_duration = self.config.retry_backoff_duration * backoff_multiplier; + + info!( + "{} regions still need retry after round {}, waiting {} seconds before next round (exponential backoff)", + remaining_regions, + retry_round, + backoff_duration.as_secs() + ); + + // Wait for backoff period before next retry round + sleep(backoff_duration).await; } } diff --git a/src/meta-srv/src/gc/mock.rs b/src/meta-srv/src/gc/mock.rs index c6bbbe300edf..2b0b38563ba7 100644 --- a/src/meta-srv/src/gc/mock.rs +++ b/src/meta-srv/src/gc/mock.rs @@ -31,15 +31,15 @@ use common_meta::key::table_route::PhysicalTableRouteValue; use common_meta::peer::Peer; use common_meta::rpc::router::{Region, RegionRoute}; use ordered_float::OrderedFloat; -use snafu::OptionExt; use store_api::region_engine::RegionRole; use store_api::storage::{FileRefsManifest, GcReport, RegionId}; use table::metadata::TableId; use tokio::sync::mpsc::Sender; -use crate::error::{Result, UnexpectedSnafu}; +use crate::error::Result; use crate::gc::candidate::GcCandidate; use crate::gc::ctx::SchedulerCtx; +use crate::gc::handler::Region2Peers; use crate::gc::options::GcSchedulerOptions; use crate::gc::scheduler::{Event, GcScheduler}; @@ -199,7 +199,7 @@ impl SchedulerCtx for MockSchedulerCtx { async fn get_file_references( &self, _region_ids: &[RegionId], - _region_to_peer: &HashMap, + _region_to_peer: &Region2Peers, _timeout: Duration, ) -> Result { *self.get_file_references_calls.lock().unwrap() += 1; @@ -222,12 +222,6 @@ impl SchedulerCtx for MockSchedulerCtx { ) -> Result { *self.gc_regions_calls.lock().unwrap() += 1; - // Track retry count for all regions (use the first region as representative) - if let Some(&first_region_id) = region_ids.first() { - let mut retry_count = self.gc_regions_retry_count.lock().unwrap(); - *retry_count.entry(first_region_id).or_insert(0) += 1; - } - // Check per-region error injection first (for any region) for ®ion_id in region_ids { if let Some(error) = self @@ -254,46 +248,60 @@ impl SchedulerCtx for MockSchedulerCtx { } } - // Handle success after specific number of retries (use first region as representative) - if let Some(&first_region_id) = region_ids.first() { + // Build the final report by processing each region individually + let mut final_report = GcReport::default(); + let gc_reports = self.gc_reports.lock().unwrap(); + let success_after_retries = self.gc_regions_success_after_retries.lock().unwrap(); + + for ®ion_id in region_ids { + // Get current retry count for this region let retry_count = self .gc_regions_retry_count .lock() .unwrap() - .get(&first_region_id) + .get(®ion_id) .copied() .unwrap_or(0); - let success_after_retries = self.gc_regions_success_after_retries.lock().unwrap(); - if let Some(&required_retries) = success_after_retries.get(&first_region_id) - && retry_count <= required_retries - { - // Return retryable error until we reach the required retry count - return Err(crate::error::RetryLaterSnafu { - reason: format!( - "Mock retryable error for region {} (attempt {}/{})", - first_region_id, retry_count, required_retries - ), + + // Check if this region should succeed or need retry + if let Some(&required_retries) = success_after_retries.get(®ion_id) { + if retry_count < required_retries { + // This region needs more retries - add to need_retry_regions + final_report.need_retry_regions.insert(region_id); + // Track the retry attempt + let mut retry_count_map = self.gc_regions_retry_count.lock().unwrap(); + *retry_count_map.entry(region_id).or_insert(0) += 1; + } else { + // This region has completed all required retries - succeed + if let Some(report) = gc_reports.get(®ion_id) { + final_report.merge(report.clone()); + } + // Track the success attempt + let mut retry_count_map = self.gc_regions_retry_count.lock().unwrap(); + *retry_count_map.entry(region_id).or_insert(0) += 1; } - .build()); + } else { + // No retry requirement - succeed immediately + if let Some(report) = gc_reports.get(®ion_id) { + final_report.merge(report.clone()); + } + // Track the success attempt + let mut retry_count_map = self.gc_regions_retry_count.lock().unwrap(); + *retry_count_map.entry(region_id).or_insert(0) += 1; } } - // Collect and merge reports for all requested regions - let mut combined_report = GcReport::default(); - let gc_reports = self.gc_reports.lock().unwrap(); - - for ®ion_id in region_ids { - if let Some(report) = gc_reports.get(®ion_id) { - combined_report.merge(report.clone()); - } else { - return Err(UnexpectedSnafu { - violated: format!("No corresponding gc report for {}", region_id), - } - .build()); + // Handle error sequence for retry testing (this should override the retry logic) + { + let mut error_sequence = self.gc_regions_error_sequence.lock().unwrap(); + if !error_sequence.is_empty() { + let error = error_sequence.remove(0); + return Err(error); } } - Ok(combined_report) + // Return the report with need_retry_regions populated - let the caller handle retry logic + Ok(final_report) } } From 660f520042bf729e005278f6dc161e9e3dd42e42 Mon Sep 17 00:00:00 2001 From: discord9 Date: Tue, 18 Nov 2025 14:55:57 +0800 Subject: [PATCH 06/10] test: update mock test Signed-off-by: discord9 --- src/meta-srv/src/gc/handler.rs | 15 +- src/meta-srv/src/gc/mock.rs | 84 ++++++++-- src/meta-srv/src/gc/mock/basic.rs | 4 +- src/meta-srv/src/gc/mock/con.rs | 133 ++++++++------- src/meta-srv/src/gc/mock/err_handle.rs | 84 +++++++--- src/meta-srv/src/gc/mock/integration.rs | 67 +++++--- src/meta-srv/src/gc/mock/misc.rs | 4 +- src/meta-srv/src/gc/mock/retry.rs | 214 ++++++++++++++---------- src/meta-srv/src/gc/scheduler.rs | 17 +- 9 files changed, 393 insertions(+), 229 deletions(-) diff --git a/src/meta-srv/src/gc/handler.rs b/src/meta-srv/src/gc/handler.rs index ae11c231e201..6606822a58ff 100644 --- a/src/meta-srv/src/gc/handler.rs +++ b/src/meta-srv/src/gc/handler.rs @@ -384,6 +384,12 @@ impl GcScheduler { // Continue retrying until all regions succeed or reach max retry limit while !current_retry_regions.is_empty() && retry_round < self.config.max_retries_per_region { + // Apply exponential backoff if there are still regions to retry + if !current_retry_regions.is_empty() && retry_round < self.config.max_retries_per_region + { + self.apply_retry_backoff(current_retry_regions.len(), retry_round) + .await; + } retry_round += 1; info!( @@ -426,13 +432,6 @@ impl GcScheduler { .values() .flat_map(|r| r.need_retry_regions.iter().copied()) .collect(); - - // Apply exponential backoff if there are still regions to retry - if !current_retry_regions.is_empty() && retry_round < self.config.max_retries_per_region - { - self.apply_retry_backoff(current_retry_regions.len(), retry_round) - .await; - } } info!( @@ -677,7 +676,7 @@ impl GcScheduler { "{} regions still need retry after round {}, waiting {} seconds before next round (exponential backoff)", remaining_regions, retry_round, - backoff_duration.as_secs() + backoff_duration.as_secs_f32() ); // Wait for backoff period before next retry round diff --git a/src/meta-srv/src/gc/mock.rs b/src/meta-srv/src/gc/mock.rs index 2b0b38563ba7..ff357635a356 100644 --- a/src/meta-srv/src/gc/mock.rs +++ b/src/meta-srv/src/gc/mock.rs @@ -22,7 +22,7 @@ mod integration; mod misc; mod retry; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::sync::{Arc, Mutex}; use std::time::{Duration, Instant}; @@ -30,19 +30,31 @@ use common_meta::datanode::{RegionManifestInfo, RegionStat}; use common_meta::key::table_route::PhysicalTableRouteValue; use common_meta::peer::Peer; use common_meta::rpc::router::{Region, RegionRoute}; +use common_telemetry::debug; use ordered_float::OrderedFloat; use store_api::region_engine::RegionRole; use store_api::storage::{FileRefsManifest, GcReport, RegionId}; use table::metadata::TableId; use tokio::sync::mpsc::Sender; -use crate::error::Result; +use crate::error::{Result, UnexpectedSnafu}; use crate::gc::candidate::GcCandidate; use crate::gc::ctx::SchedulerCtx; use crate::gc::handler::Region2Peers; use crate::gc::options::GcSchedulerOptions; use crate::gc::scheduler::{Event, GcScheduler}; +pub fn new_empty_report_with(region_ids: impl IntoIterator) -> GcReport { + let mut deleted_files = HashMap::new(); + for region_id in region_ids { + deleted_files.insert(region_id, vec![]); + } + GcReport { + deleted_files, + need_retry_regions: HashSet::new(), + } +} + #[allow(clippy::type_complexity)] #[derive(Debug, Default)] pub struct MockSchedulerCtx { @@ -124,8 +136,10 @@ impl MockSchedulerCtx { /// Set success after a specific number of retries for a region pub fn set_gc_regions_success_after_retries(&self, region_id: RegionId, retries: usize) { - *self.gc_regions_success_after_retries.lock().unwrap() = - HashMap::from([(region_id, retries)]); + self.gc_regions_success_after_retries + .lock() + .unwrap() + .insert(region_id, retries); } /// Get the retry count for a specific region @@ -198,8 +212,8 @@ impl SchedulerCtx for MockSchedulerCtx { async fn get_file_references( &self, - _region_ids: &[RegionId], - _region_to_peer: &Region2Peers, + region_ids: &[RegionId], + region_to_peer: &Region2Peers, _timeout: Duration, ) -> Result { *self.get_file_references_calls.lock().unwrap() += 1; @@ -208,6 +222,17 @@ impl SchedulerCtx for MockSchedulerCtx { if let Some(error) = self.get_file_references_error.lock().unwrap().take() { return Err(error); } + if region_ids + .iter() + .any(|region_id| !region_to_peer.contains_key(region_id)) + { + UnexpectedSnafu { + violated: format!( + "region_to_peer{region_to_peer:?} does not contain all region_ids requested: {:?}", + region_ids + ), + }.fail()?; + } Ok(self.file_refs.lock().unwrap().clone().unwrap_or_default()) } @@ -230,12 +255,26 @@ impl SchedulerCtx for MockSchedulerCtx { .unwrap() .remove(®ion_id) { + *self + .gc_regions_retry_count + .lock() + .unwrap() + .entry(region_id) + .or_insert(0) += 1; return Err(error); } } // Check if we should return an injected error if let Some(error) = self.gc_regions_error.lock().unwrap().take() { + for region_id in region_ids { + *self + .gc_regions_retry_count + .lock() + .unwrap() + .entry(*region_id) + .or_insert(0) += 1; + } return Err(error); } @@ -244,6 +283,14 @@ impl SchedulerCtx for MockSchedulerCtx { let mut error_sequence = self.gc_regions_error_sequence.lock().unwrap(); if !error_sequence.is_empty() { let error = error_sequence.remove(0); + for region_id in region_ids { + *self + .gc_regions_retry_count + .lock() + .unwrap() + .entry(*region_id) + .or_insert(0) += 1; + } return Err(error); } } @@ -266,12 +313,22 @@ impl SchedulerCtx for MockSchedulerCtx { // Check if this region should succeed or need retry if let Some(&required_retries) = success_after_retries.get(®ion_id) { if retry_count < required_retries { + debug!( + "Region {} needs retry (attempt {}/{})", + region_id, + retry_count + 1, + required_retries + ); // This region needs more retries - add to need_retry_regions final_report.need_retry_regions.insert(region_id); // Track the retry attempt let mut retry_count_map = self.gc_regions_retry_count.lock().unwrap(); *retry_count_map.entry(region_id).or_insert(0) += 1; } else { + debug!( + "Region {} has completed retries - succeeding now", + region_id + ); // This region has completed all required retries - succeed if let Some(report) = gc_reports.get(®ion_id) { final_report.merge(report.clone()); @@ -281,13 +338,20 @@ impl SchedulerCtx for MockSchedulerCtx { *retry_count_map.entry(region_id).or_insert(0) += 1; } } else { - // No retry requirement - succeed immediately + // No retry requirement - check if we have a GC report for this region if let Some(report) = gc_reports.get(®ion_id) { + // We have a GC report - succeed immediately final_report.merge(report.clone()); + // Track the success attempt + let mut retry_count_map = self.gc_regions_retry_count.lock().unwrap(); + *retry_count_map.entry(region_id).or_insert(0) += 1; + } else { + // No GC report available - this region should be marked for retry + final_report.need_retry_regions.insert(region_id); + // Track the attempt + let mut retry_count_map = self.gc_regions_retry_count.lock().unwrap(); + *retry_count_map.entry(region_id).or_insert(0) += 1; } - // Track the success attempt - let mut retry_count_map = self.gc_regions_retry_count.lock().unwrap(); - *retry_count_map.entry(region_id).or_insert(0) += 1; } } diff --git a/src/meta-srv/src/gc/mock/basic.rs b/src/meta-srv/src/gc/mock/basic.rs index 128803feb3b7..c5ef7a3a0a35 100644 --- a/src/meta-srv/src/gc/mock/basic.rs +++ b/src/meta-srv/src/gc/mock/basic.rs @@ -29,7 +29,7 @@ async fn test_process_datanodes_concurrently_empty() { let env = TestEnv::new(); let report = env .scheduler - .process_datanodes_concurrently(HashMap::new()) + .process_datanodes_with_retry(HashMap::new()) .await; assert_eq!(report.per_datanode_reports.len(), 0); @@ -85,7 +85,7 @@ async fn test_process_datanodes_concurrently_with_candidates() { )]); let report = scheduler - .process_datanodes_concurrently(datanode_to_candidates) + .process_datanodes_with_retry(datanode_to_candidates) .await; assert_eq!(report.per_datanode_reports.len(), 1); diff --git a/src/meta-srv/src/gc/mock/con.rs b/src/meta-srv/src/gc/mock/con.rs index 04609e844738..d6d35b8f2add 100644 --- a/src/meta-srv/src/gc/mock/con.rs +++ b/src/meta-srv/src/gc/mock/con.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::sync::{Arc, Mutex}; use std::time::{Duration, Instant}; @@ -68,7 +68,8 @@ async fn test_concurrent_table_processing_limits() { let ctx = Arc::new(ctx); let config = GcSchedulerOptions { - max_concurrent_tables: 3, // Set a low limit + max_concurrent_tables: 3, // Set a low limit + retry_backoff_duration: Duration::from_millis(50), // for faster test ..Default::default() }; @@ -93,7 +94,7 @@ async fn test_concurrent_table_processing_limits() { )]); let report = scheduler - .process_datanodes_concurrently(datanode_to_candidates) + .process_datanodes_with_retry(datanode_to_candidates) .await; // Should process all datanodes @@ -165,23 +166,14 @@ async fn test_mixed_success_failure_tables() { )]); let report = scheduler - .process_datanodes_concurrently(datanode_to_candidates) + .process_datanodes_with_retry(datanode_to_candidates) .await; // Should have one datanode with mixed results assert_eq!(report.per_datanode_reports.len(), 1); - // also check one failed region - assert_eq!( - report - .per_datanode_reports - .iter() - .next() - .unwrap() - .1 - .need_retry_regions - .len(), - 1 - ); + // also check one failed region (region2 has no GC report, so it should be in need_retry_regions) + let datanode_report = report.per_datanode_reports.values().next().unwrap(); + assert_eq!(datanode_report.need_retry_regions.len(), 1); assert_eq!(report.failed_datanodes.len(), 0); } @@ -248,6 +240,7 @@ async fn test_region_gc_concurrency_limit() { // Configure low concurrency limit let config = GcSchedulerOptions { region_gc_concurrency: 3, // Only 3 regions can be processed concurrently + retry_backoff_duration: Duration::from_millis(50), // for faster test ..Default::default() }; @@ -260,7 +253,7 @@ async fn test_region_gc_concurrency_limit() { }; let start_time = Instant::now(); - let reports = scheduler + let report = scheduler .process_datanode_gc( peer, candidates.into_iter().map(|c| (table_id, c)).collect(), @@ -270,10 +263,14 @@ async fn test_region_gc_concurrency_limit() { let duration = start_time.elapsed(); // All regions should be processed successfully - assert_eq!(reports.len(), 1); - let table_report = &reports[0]; - assert_eq!(table_report.success_regions.len(), 10); - assert_eq!(table_report.failed_regions.len(), 0); + // Check that all 10 regions have deleted files + assert_eq!(report.deleted_files.len(), 10); + for i in 1..=10 { + let region_id = RegionId::new(table_id, i as u32); + assert!(report.deleted_files.contains_key(®ion_id)); + assert_eq!(report.deleted_files[®ion_id].len(), 2); // Each region has 2 deleted files + } + assert!(report.need_retry_regions.is_empty()); // Verify that concurrency limit was respected (this is hard to test directly, // but we can verify that the processing completed successfully) @@ -355,6 +352,7 @@ async fn test_region_gc_concurrency_with_mixed_results() { // Configure concurrency limit let config = GcSchedulerOptions { region_gc_concurrency: 2, // Process 2 regions concurrently + retry_backoff_duration: Duration::from_millis(50), // for faster test ..Default::default() }; @@ -366,37 +364,43 @@ async fn test_region_gc_concurrency_with_mixed_results() { last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), }; - let reports = scheduler - .process_datanode_gc( - peer, - candidates.into_iter().map(|c| (table_id, c)).collect(), - ) - .await - .unwrap(); + let dn2candidates = HashMap::from([( + peer.clone(), + candidates.into_iter().map(|c| (table_id, c)).collect(), + )]); + + let report = scheduler.process_datanodes_with_retry(dn2candidates).await; + + let report = report.per_datanode_reports.get(&peer.id).unwrap(); // Should have 3 successful and 3 failed regions - assert_eq!(reports.len(), 1); - let report = &reports[0]; - assert_eq!(report.success_regions.len(), 3); - assert_eq!(report.failed_regions.len(), 3); - - // Verify that successful regions are the even-numbered ones - for success_report in &report.success_regions { - for region in success_report.deleted_files.keys() { - let region_num = region.region_number(); - assert_eq!( - region_num % 2, - 0, - "Successful regions should be even-numbered" + // Even regions (2, 4, 6) should succeed, odd regions (1, 3, 5) should fail + let mut successful_regions = 0; + let mut failed_regions = 0; + + for i in 1..=6 { + let region_id = RegionId::new(table_id, i as u32); + if i % 2 == 0 { + // Even regions should succeed + assert!( + report.deleted_files.contains_key(®ion_id), + "Even region {} should succeed", + i ); + successful_regions += 1; + } else { + // Odd regions should fail + assert!( + report.need_retry_regions.contains(®ion_id), + "Odd region {} should fail", + i + ); + failed_regions += 1; } } - // Verify that failed regions are the odd-numbered ones - for region_id in report.failed_regions.keys() { - let region_num = region_id.region_number(); - assert_eq!(region_num % 2, 1, "Failed regions should be odd-numbered"); - } + assert_eq!(successful_regions, 3, "Should have 3 successful regions"); + assert_eq!(failed_regions, 3, "Should have 3 failed regions"); } #[tokio::test] @@ -427,7 +431,14 @@ async fn test_region_gc_concurrency_with_retryable_errors() { }; let gc_report = (1..=5) - .map(|i| (RegionId::new(table_id, i as u32), GcReport::default())) + .map(|i| { + let region_id = RegionId::new(table_id, i as u32); + ( + region_id, + // mock the actual gc report with deleted files when succeeded(even no files to delete) + GcReport::new(HashMap::from([(region_id, vec![])]), HashSet::new()), + ) + }) .collect(); let ctx = Arc::new( @@ -470,19 +481,25 @@ async fn test_region_gc_concurrency_with_retryable_errors() { ctx.set_gc_regions_success_after_retries(region_id, 1); } - let reports = scheduler - .process_datanode_gc( - peer, - candidates.into_iter().map(|c| (table_id, c)).collect(), - ) - .await - .unwrap(); + let dn2candidates = HashMap::from([( + peer.clone(), + candidates.into_iter().map(|c| (table_id, c)).collect(), + )]); + let report = scheduler.process_datanodes_with_retry(dn2candidates).await; + + let report = report.per_datanode_reports.get(&peer.id).unwrap(); // All regions should eventually succeed after retries - assert_eq!(reports.len(), 1); - let report = &reports[0]; - assert_eq!(report.success_regions.len(), 5); - assert_eq!(report.failed_regions.len(), 0); + assert_eq!(report.deleted_files.len(), 5); + for i in 1..=5 { + let region_id = RegionId::new(table_id, i as u32); + assert!( + report.deleted_files.contains_key(®ion_id), + "Region {} should succeed", + i + ); + } + assert!(report.need_retry_regions.is_empty()); // Verify that retries were attempted for all regions for i in 1..=5 { diff --git a/src/meta-srv/src/gc/mock/err_handle.rs b/src/meta-srv/src/gc/mock/err_handle.rs index 7e558a61b7d4..e31892f2bb80 100644 --- a/src/meta-srv/src/gc/mock/err_handle.rs +++ b/src/meta-srv/src/gc/mock/err_handle.rs @@ -14,7 +14,7 @@ use std::collections::{HashMap, HashSet}; use std::sync::{Arc, Mutex}; -use std::time::Instant; +use std::time::{Duration, Instant}; use common_meta::datanode::RegionManifestInfo; use common_meta::peer::Peer; @@ -22,7 +22,7 @@ use common_telemetry::init_default_ut_logging; use store_api::region_engine::RegionRole; use store_api::storage::{FileId, FileRefsManifest, GcReport, RegionId}; -use crate::gc::mock::{MockSchedulerCtx, mock_region_stat}; +use crate::gc::mock::{MockSchedulerCtx, mock_region_stat, new_empty_report_with}; use crate::gc::{GcScheduler, GcSchedulerOptions}; /// Error Handling Tests @@ -93,8 +93,20 @@ async fn test_gc_regions_failure_handling() { ); assert_eq!( report.failed_datanodes.len(), + 0, + "Should have 0 failed datanodes (failure handled via need_retry_regions)" + ); + + // Check that the region is in need_retry_regions due to the failure + let datanode_report = report.per_datanode_reports.values().next().unwrap(); + assert_eq!( + datanode_report.need_retry_regions.len(), 1, - "Should have 1 failed datanode" + "Should have 1 region in need_retry_regions due to failure" + ); + assert!( + datanode_report.need_retry_regions.contains(®ion_id), + "Region should be in need_retry_regions" ); // Verify that calls were made despite potential failures @@ -103,15 +115,13 @@ async fn test_gc_regions_failure_handling() { 1, "Expected 1 call to get_table_to_region_stats" ); - assert_eq!( - *ctx.get_file_references_calls.lock().unwrap(), - 1, - "Expected 1 call to get_file_references" + assert!( + *ctx.get_file_references_calls.lock().unwrap() >= 1, + "Expected at least 1 call to get_file_references" ); - assert_eq!( - *ctx.gc_regions_calls.lock().unwrap(), - 1, - "Expected 1 call to gc_regions" + assert!( + *ctx.gc_regions_calls.lock().unwrap() >= 1, + "Expected at least 1 call to gc_regions" ); } @@ -139,6 +149,10 @@ async fn test_get_file_references_failure() { MockSchedulerCtx { table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), file_refs: Arc::new(Mutex::new(Some(FileRefsManifest::default()))), + gc_reports: Arc::new(Mutex::new(HashMap::from([( + region_id, + new_empty_report_with([region_id]), + )]))), ..Default::default() } .with_table_routes(HashMap::from([( @@ -150,7 +164,10 @@ async fn test_get_file_references_failure() { let scheduler = GcScheduler { ctx: ctx.clone(), receiver: GcScheduler::channel().1, - config: GcSchedulerOptions::default(), + config: GcSchedulerOptions { + retry_backoff_duration: Duration::from_millis(10), // shorten for test + ..Default::default() + }, region_gc_tracker: Arc::new(tokio::sync::Mutex::new(HashMap::new())), last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), }; @@ -166,15 +183,28 @@ async fn test_get_file_references_failure() { ); assert_eq!( report.failed_datanodes.len(), - 1, - "Should have 1 failed datanode" + 0, + "Should have 0 failed datanodes (failure handled gracefully)" ); - // Should still attempt to get file references - assert_eq!( - *ctx.get_file_references_calls.lock().unwrap(), - 1, - "Expected 1 call to get_file_references" + // The region should be processed but may have empty results due to file refs failure + let datanode_report = report.per_datanode_reports.values().next().unwrap(); + // The current implementation still processes the region even with file refs failure + // and creates an empty entry in deleted_files + assert!( + datanode_report.deleted_files.contains_key(®ion_id), + "Should have region in deleted_files (even if empty)" + ); + assert!( + datanode_report.deleted_files[®ion_id].is_empty(), + "Should have empty deleted files due to file refs failure" + ); + + // Should still attempt to get file references (may be called multiple times due to retry logic) + assert!( + *ctx.get_file_references_calls.lock().unwrap() >= 1, + "Expected at least 1 call to get_file_references, got {}", + *ctx.get_file_references_calls.lock().unwrap() ); } @@ -237,18 +267,22 @@ async fn test_get_table_route_failure() { // This should handle table route failure gracefully let report = scheduler - .process_datanodes_concurrently(datanode_to_candidates) + .process_datanodes_with_retry(datanode_to_candidates) .await; - - // Should process the datanode but fail due to route error + dbg!(&report); + // Should process the datanode but handle route error gracefully assert_eq!( report.per_datanode_reports.len(), - 1, - "Expected 1 datanode report" + 0, + "Expected 0 datanode report" ); assert_eq!( report.failed_datanodes.len(), 1, - "Expected 1 failed datanode due to route error" + "Expected 1 failed datanodes (route error handled gracefully)" + ); + assert!( + report.failed_datanodes.contains_key(&1), + "Failed datanodes should contain the datanode with route error" ); } diff --git a/src/meta-srv/src/gc/mock/integration.rs b/src/meta-srv/src/gc/mock/integration.rs index f5d29e348567..904967727644 100644 --- a/src/meta-srv/src/gc/mock/integration.rs +++ b/src/meta-srv/src/gc/mock/integration.rs @@ -22,7 +22,7 @@ use common_telemetry::init_default_ut_logging; use store_api::region_engine::RegionRole; use store_api::storage::{FileId, FileRefsManifest, GcReport, RegionId}; -use crate::gc::mock::{MockSchedulerCtx, mock_region_stat}; +use crate::gc::mock::{MockSchedulerCtx, mock_region_stat, new_empty_report_with}; use crate::gc::{GcScheduler, GcSchedulerOptions}; // Integration Flow Tests @@ -84,20 +84,34 @@ async fn test_full_gc_workflow() { // Run the full workflow let report = scheduler.handle_tick().await.unwrap(); - // Validate the returned GcJobReport - assert_eq!(report.processed_tables, 1, "Should process 1 table"); - assert_eq!(report.table_reports.len(), 1, "Should have 1 table report"); - - let table_report = &report.table_reports[0]; - assert_eq!(table_report.table_id, table_id, "Table ID should match"); + // Validate the returned GcJobReport - should have 1 datanode report assert_eq!( - table_report.success_regions.len(), + report.per_datanode_reports.len(), 1, - "Should have 1 successful region" + "Should process 1 datanode" + ); + assert_eq!( + report.failed_datanodes.len(), + 0, + "Should have no failed datanodes" + ); + + // Get the datanode report + let datanode_report = report.per_datanode_reports.values().next().unwrap(); + + // Check that the region was processed successfully + assert!( + datanode_report.deleted_files.contains_key(®ion_id), + "Should have deleted files for region" + ); + assert_eq!( + datanode_report.deleted_files[®ion_id].len(), + 2, + "Should have 2 deleted files" ); assert!( - table_report.failed_regions.is_empty(), - "Should have no failed regions" + datanode_report.need_retry_regions.is_empty(), + "Should have no retry regions" ); // Verify all steps were executed @@ -138,7 +152,7 @@ async fn test_tracker_cleanup() { let table_stats = HashMap::from([(table_id, vec![region_stat])]); let mut gc_reports = HashMap::new(); - gc_reports.insert(region_id, GcReport::default()); + gc_reports.insert(region_id, new_empty_report_with([region_id])); let file_refs = FileRefsManifest { manifest_version: HashMap::from([(region_id, 1)]), @@ -197,20 +211,29 @@ async fn test_tracker_cleanup() { let report = scheduler.handle_tick().await.unwrap(); - // Validate the returned GcJobReport - assert_eq!(report.processed_tables, 1, "Should process 1 table"); - assert_eq!(report.table_reports.len(), 1, "Should have 1 table report"); - - let table_report = &report.table_reports[0]; - assert_eq!(table_report.table_id, table_id, "Table ID should match"); + // Validate the returned GcJobReport - should have 1 datanode report assert_eq!( - table_report.success_regions.len(), + report.per_datanode_reports.len(), 1, - "Should have 1 successful region" + "Should process 1 datanode" + ); + assert_eq!( + report.failed_datanodes.len(), + 0, + "Should have no failed datanodes" + ); + + // Get the datanode report + let datanode_report = report.per_datanode_reports.values().next().unwrap(); + + // Check that the region was processed successfully + assert!( + datanode_report.deleted_files.contains_key(®ion_id), + "Should have deleted files for region" ); assert!( - table_report.failed_regions.is_empty(), - "Should have no failed regions" + datanode_report.need_retry_regions.is_empty(), + "Should have no retry regions" ); // Verify tracker was updated diff --git a/src/meta-srv/src/gc/mock/misc.rs b/src/meta-srv/src/gc/mock/misc.rs index 1ecea700f87e..ee02175f32ec 100644 --- a/src/meta-srv/src/gc/mock/misc.rs +++ b/src/meta-srv/src/gc/mock/misc.rs @@ -70,7 +70,7 @@ async fn test_empty_file_refs_manifest() { )]); let report = scheduler - .process_datanodes_concurrently(datanode_to_candidates) + .process_datanodes_with_retry(datanode_to_candidates) .await; assert_eq!(report.per_datanode_reports.len(), 1); @@ -148,7 +148,7 @@ async fn test_multiple_regions_per_table() { )]); let report = scheduler - .process_datanodes_concurrently(datanode_to_candidates) + .process_datanodes_with_retry(datanode_to_candidates) .await; assert_eq!(report.per_datanode_reports.len(), 1); diff --git a/src/meta-srv/src/gc/mock/retry.rs b/src/meta-srv/src/gc/mock/retry.rs index 0e220c838036..09e5439c4348 100644 --- a/src/meta-srv/src/gc/mock/retry.rs +++ b/src/meta-srv/src/gc/mock/retry.rs @@ -21,7 +21,7 @@ use common_telemetry::init_default_ut_logging; use store_api::region_engine::RegionRole; use store_api::storage::{FileId, FileRefsManifest, GcReport, RegionId}; -use crate::gc::mock::{MockSchedulerCtx, mock_candidate, mock_region_stat}; +use crate::gc::mock::{MockSchedulerCtx, mock_candidate, mock_region_stat, new_empty_report_with}; use crate::gc::{GcScheduler, GcSchedulerOptions}; /// Retry Logic Tests @@ -79,17 +79,23 @@ async fn test_retry_logic_with_retryable_errors() { ctx.set_gc_regions_success_after_retries(region_id, 2); let start_time = Instant::now(); - let reports = scheduler - .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) - .await - .unwrap(); + let mut datanode_to_candidates = HashMap::new(); + datanode_to_candidates.insert(peer.clone(), vec![(table_id, mock_candidate(region_id))]); + + let job_report = scheduler + .process_datanodes_with_retry(datanode_to_candidates) + .await; let duration = start_time.elapsed(); - assert_eq!(reports.len(), 1); - let report = &reports[0]; - assert_eq!(report.success_regions.len(), 1); - assert_eq!(report.failed_regions.len(), 0); - assert_eq!(ctx.get_retry_count(region_id), 3); // 2 retries + 1 success + // Extract the datanode report from the job report + let report = job_report.per_datanode_reports.get(&peer.id).unwrap(); + + // Check that the report has deleted files for the region + assert!(report.deleted_files.contains_key(®ion_id)); + assert_eq!(report.deleted_files[®ion_id].len(), 2); // We created 2 files in the mock + // The current implementation retries all errors up to max_retries_per_region + // so the region may still be in need_retry_regions after all retries are exhausted + assert_eq!(ctx.get_retry_count(region_id), 3); // 2 retries + 1 success (handled by mock) // Verify backoff was applied (should take at least 200ms for 2 retries with 100ms backoff) assert!( @@ -102,16 +108,15 @@ async fn test_retry_logic_with_retryable_errors() { ctx.reset_retry_tracking(); ctx.set_gc_regions_success_after_retries(region_id, 5); // More than max_retries_per_region (3) - let reports = scheduler - .process_datanode_gc(peer, vec![(table_id, mock_candidate(region_id))]) - .await - .unwrap(); + let dn2candidates = + HashMap::from([(peer.clone(), vec![(table_id, mock_candidate(region_id))])]); + let report = scheduler.process_datanodes_with_retry(dn2candidates).await; + + let report = report.per_datanode_reports.get(&peer.id).unwrap(); - assert_eq!(reports.len(), 1); - let report = &reports[0]; - assert_eq!(report.success_regions.len(), 0); - assert_eq!(report.failed_regions.len(), 1); - assert_eq!(ctx.get_retry_count(region_id), 3); // Should stop at max_retries_per_region + // Since the region exceeded max retries, it should be in need_retry_regions + assert!(report.need_retry_regions.contains(®ion_id)); + assert_eq!(ctx.get_retry_count(region_id), 4); // 1 initial + 3 retries (mock tracks all attempts) } #[tokio::test] @@ -173,16 +178,22 @@ async fn test_retry_logic_with_error_sequence() { ctx.reset_retry_tracking(); ctx.set_gc_regions_error_sequence(vec![retryable_error1, retryable_error2]); - let reports = scheduler - .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) - .await - .unwrap(); + let mut datanode_to_candidates = HashMap::new(); + datanode_to_candidates.insert(peer.clone(), vec![(table_id, mock_candidate(region_id))]); + + let job_report = scheduler + .process_datanodes_with_retry(datanode_to_candidates) + .await; + + // Extract the datanode report from the job report + let report = job_report.per_datanode_reports.get(&peer.id).unwrap(); - assert_eq!(reports.len(), 1); - let report = &reports[0]; - assert_eq!(report.success_regions.len(), 1); - assert_eq!(report.failed_regions.len(), 0); - assert_eq!(ctx.get_retry_count(region_id), 3); // 2 errors + 1 success + // Check that the report has deleted files for the region + assert!(report.deleted_files.contains_key(®ion_id)); + assert_eq!(report.deleted_files[®ion_id].len(), 2); // We created 2 files in the mock + // The current implementation retries all errors up to max_retries_per_region + // so the region may still be in need_retry_regions after all retries are exhausted + assert_eq!(ctx.get_retry_count(region_id), 3); // 2 errors + 1 success (handled by mock) } #[tokio::test] @@ -205,6 +216,10 @@ async fn test_retry_logic_non_retryable_error() { MockSchedulerCtx { table_to_region_stats: Arc::new(Mutex::new(Some(table_stats))), file_refs: Arc::new(Mutex::new(Some(file_refs))), + gc_reports: Arc::new(Mutex::new(HashMap::from([( + region_id, + new_empty_report_with([region_id]), + )]))), ..Default::default() } .with_table_routes(HashMap::from([( @@ -215,6 +230,7 @@ async fn test_retry_logic_non_retryable_error() { let config = GcSchedulerOptions { max_retries_per_region: 3, + retry_backoff_duration: Duration::from_millis(50), ..Default::default() }; @@ -235,16 +251,19 @@ async fn test_retry_logic_non_retryable_error() { .unwrap() .replace(non_retryable_error); - let reports = scheduler - .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) - .await - .unwrap(); + let mut datanode_to_candidates = HashMap::new(); + datanode_to_candidates.insert(peer.clone(), vec![(table_id, mock_candidate(region_id))]); + + let job_report = scheduler + .process_datanodes_with_retry(datanode_to_candidates) + .await; - assert_eq!(reports.len(), 1); - let report = &reports[0]; - assert_eq!(report.success_regions.len(), 0); - assert_eq!(report.failed_regions.len(), 1); - assert_eq!(ctx.get_retry_count(region_id), 1); // Only 1 attempt, no retries + // Extract the datanode report from the job report + let report = job_report.per_datanode_reports.get(&peer.id).unwrap(); + + // Non-retryable error should still retry(then success) + assert!(report.deleted_files.contains_key(®ion_id)); + assert_eq!(ctx.get_retry_count(region_id), 2); // Should retry up to max_retries_per_region (3 retry + 1 success) } /// Test need_retry_regions functionality with multi-round retry @@ -317,25 +336,35 @@ async fn test_need_retry_regions_multi_round() { let start_time = Instant::now(); // Use peer1 since both regions belong to the same table and we're testing single datanode scenario - let reports = scheduler - .process_datanode_gc(peer1.clone(), vec![(table_id, mock_candidate(region_id1))]) - .await - .unwrap(); - let reports2 = scheduler - .process_datanode_gc(peer2.clone(), vec![(table_id, mock_candidate(region_id2))]) - .await - .unwrap(); + let report1 = scheduler + .process_datanodes_with_retry(HashMap::from([( + peer1.clone(), + vec![(table_id, mock_candidate(region_id1))], + )])) + .await; + let report2 = scheduler + .process_datanodes_with_retry(HashMap::from([( + peer2.clone(), + vec![(table_id, mock_candidate(region_id2))], + )])) + .await; let duration = start_time.elapsed(); - // Verify results - assert_eq!(reports.len(), 1); - let report = &reports[0]; - assert_eq!(report.success_regions.len(), 2); // Both regions should succeed eventually - assert_eq!(report.failed_regions.len(), 0); + let report1 = report1.per_datanode_reports.get(&peer1.id).unwrap(); + let report2 = report2.per_datanode_reports.get(&peer2.id).unwrap(); + + // Verify results - both reports should have their respective regions + assert!(report1.deleted_files.contains_key(®ion_id1)); + assert_eq!(report1.deleted_files[®ion_id1].len(), 1); // We created 1 file in the mock + assert!(report1.need_retry_regions.is_empty()); + + assert!(report2.deleted_files.contains_key(®ion_id2)); + assert_eq!(report2.deleted_files[®ion_id2].len(), 1); // We created 1 file in the mock + assert!(report2.need_retry_regions.is_empty()); // Verify retry count for region1 (should be 2: 1 initial failure + 1 retry success) assert_eq!(ctx.get_retry_count(region_id1), 2); - // region2 should only be processed once + // region2 should only be processed once (no retry needed) assert_eq!(ctx.get_retry_count(region_id2), 1); // Verify exponential backoff was applied (should take at least 100ms for 1 retry round) @@ -361,7 +390,7 @@ async fn test_need_retry_regions_exponential_backoff() { // Create GC report that will need multiple retry rounds let gc_report = GcReport { deleted_files: HashMap::from([(region_id, vec![FileId::random()])]), - need_retry_regions: std::collections::HashSet::from([region_id]), + need_retry_regions: std::collections::HashSet::from([]), }; let file_refs = FileRefsManifest { @@ -401,23 +430,26 @@ async fn test_need_retry_regions_exponential_backoff() { ctx.set_gc_regions_success_after_retries(region_id, 3); let start_time = Instant::now(); - let reports = scheduler - .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) - .await - .unwrap(); + let mut datanode_to_candidates = HashMap::new(); + datanode_to_candidates.insert(peer.clone(), vec![(table_id, mock_candidate(region_id))]); + let job_report = scheduler + .process_datanodes_with_retry(datanode_to_candidates) + .await; let duration = start_time.elapsed(); + // Extract the datanode report from the job report + let report = job_report.per_datanode_reports.get(&peer.id).unwrap(); + + dbg!(&report); // Verify results - assert_eq!(reports.len(), 1); - let report = &reports[0]; - assert_eq!(report.success_regions.len(), 1); - assert_eq!(report.failed_regions.len(), 0); - assert_eq!(ctx.get_retry_count(region_id), 4); // 1 initial + 3 retries + assert!(report.deleted_files.contains_key(®ion_id)); + assert!(report.need_retry_regions.is_empty()); + assert_eq!(ctx.get_retry_count(region_id), 4); // 1 initial + 3 retries (mock tracks all attempts) // Verify exponential backoff was applied - // Expected backoff: 50ms + 100ms + 200ms = 350ms minimum + // Expected backoff: 50ms + 100ms = 150ms minimum assert!( - duration >= Duration::from_millis(350), + duration >= Duration::from_millis(150), "Expected exponential backoff duration not met: {:?}", duration ); @@ -477,16 +509,14 @@ async fn test_need_retry_regions_table_route_failure() { last_tracker_cleanup: Arc::new(tokio::sync::Mutex::new(Instant::now())), }; - let reports = scheduler - .process_datanode_gc(peer.clone(), vec![(table_id, mock_candidate(region_id))]) - .await - .unwrap(); - - // The region should fail since table route rediscovery failed - assert_eq!(reports.len(), 1); - let report = &reports[0]; - assert_eq!(report.success_regions.len(), 0); - assert_eq!(report.failed_regions.len(), 1); + let report = scheduler + .process_datanodes_with_retry(HashMap::from([( + peer.clone(), + vec![(table_id, mock_candidate(region_id))], + )])) + .await; + assert!(report.per_datanode_reports.is_empty()); + assert!(report.failed_datanodes.contains_key(&peer.id)); } /// Test need_retry_regions with mixed success/failure scenarios @@ -566,25 +596,31 @@ async fn test_need_retry_regions_mixed_scenarios() { ctx.set_gc_regions_success_after_retries(region_id2, 5); // Needs more retries than limit (2) // region3 succeeds immediately (default behavior) - // Process each region separately with its corresponding peer - let mut all_reports = Vec::new(); + // Process each region separately with its corresponding peer and check results directly for (region_id, peer) in routes { - let reports = scheduler - .process_datanode_gc(peer, vec![(table_id, mock_candidate(region_id))]) - .await - .unwrap(); - all_reports.extend(reports); + let dn2candiate = + HashMap::from([(peer.clone(), vec![(table_id, mock_candidate(region_id))])]); + let report = scheduler.process_datanodes_with_retry(dn2candiate).await; + + let report = report.per_datanode_reports.get(&peer.id).unwrap(); + + if region_id == region_id1 { + // region1 should succeed after 1 retry + assert!(report.deleted_files.contains_key(®ion_id1)); + assert!(report.need_retry_regions.is_empty()); + } else if region_id == region_id2 { + // region2 should fail due to retry limit + assert!(!report.deleted_files.contains_key(®ion_id2)); + assert!(report.need_retry_regions.contains(®ion_id2)); + } else if region_id == region_id3 { + // region3 should succeed immediately + assert!(report.deleted_files.contains_key(®ion_id3)); + assert!(report.need_retry_regions.is_empty()); + } } - // Combine results from all regions - let combined_report = all_reports.into_iter().next().unwrap_or_default(); - - // Verify mixed results - assert_eq!(combined_report.success_regions.len(), 2); // region1 and region3 succeed - assert_eq!(combined_report.failed_regions.len(), 1); // region2 fails due to retry limit - // Verify retry counts - assert_eq!(ctx.get_retry_count(region_id1), 2); // 1 initial + 1 retry - assert_eq!(ctx.get_retry_count(region_id2), 2); // Stops at max_retries_per_region + assert_eq!(ctx.get_retry_count(region_id1), 2); // Succeeds immediately + assert_eq!(ctx.get_retry_count(region_id2), 3); // Stops at max_retries_per_region assert_eq!(ctx.get_retry_count(region_id3), 1); // Succeeds immediately } diff --git a/src/meta-srv/src/gc/scheduler.rs b/src/meta-srv/src/gc/scheduler.rs index 35b5fdf7569b..097570e7a4ae 100644 --- a/src/meta-srv/src/gc/scheduler.rs +++ b/src/meta-srv/src/gc/scheduler.rs @@ -53,20 +53,9 @@ pub struct GcJobReport { impl GcJobReport { pub fn merge(&mut self, mut other: GcJobReport) { // merge per_datanode_reports&failed_datanodes - let all_dn_ids = self - .per_datanode_reports - .keys() - .cloned() - .chain(other.per_datanode_reports.keys().cloned()) - .collect::>(); - for dn_id in all_dn_ids { + for (dn_id, report) in other.per_datanode_reports { let mut self_report = self.per_datanode_reports.entry(dn_id).or_default(); - self_report.merge( - other - .per_datanode_reports - .remove(&dn_id) - .unwrap_or_default(), - ); + self_report.merge(report); } let all_failed_dn_ids = self .failed_datanodes @@ -80,6 +69,8 @@ impl GcJobReport { entry.extend(other_errors); } } + self.failed_datanodes + .retain(|dn_id, _| !self.per_datanode_reports.contains_key(dn_id)); } } From 49750994cf2df12de0d849042ea9c88d517c931f Mon Sep 17 00:00:00 2001 From: discord9 Date: Tue, 18 Nov 2025 15:41:30 +0800 Subject: [PATCH 07/10] revert some change&clean up Signed-off-by: discord9 --- src/meta-srv/src/gc/handler.rs | 16 +--------------- src/meta-srv/src/gc/mock.rs | 1 + src/meta-srv/src/gc/mock/basic.rs | 2 -- src/meta-srv/src/gc/scheduler.rs | 28 ++-------------------------- src/mito2/src/sst/file_purger.rs | 2 +- src/mito2/src/sst/file_ref.rs | 6 +++--- 6 files changed, 8 insertions(+), 47 deletions(-) diff --git a/src/meta-srv/src/gc/handler.rs b/src/meta-srv/src/gc/handler.rs index 6606822a58ff..16016292161c 100644 --- a/src/meta-srv/src/gc/handler.rs +++ b/src/meta-srv/src/gc/handler.rs @@ -13,20 +13,17 @@ // limitations under the License. use std::collections::{HashMap, HashSet}; -use std::sync::Arc; -use std::sync::atomic::{AtomicUsize, Ordering}; use std::time::Instant; use common_meta::key::table_route::PhysicalTableRouteValue; use common_meta::peer::Peer; use common_telemetry::{debug, error, info, warn}; use futures::StreamExt; -use snafu::OptionExt as _; use store_api::storage::{FileRefsManifest, GcReport, RegionId}; use table::metadata::TableId; use tokio::time::sleep; -use crate::error::{RegionRouteNotFoundSnafu, Result}; +use crate::error::Result; use crate::gc::candidate::GcCandidate; use crate::gc::scheduler::{GcJobReport, GcScheduler}; use crate::gc::tracker::RegionGcInfo; @@ -95,17 +92,6 @@ impl GcScheduler { Ok(candidate_region_ids.to_vec()) } - /// Get the peer for a given region. - fn get_region_peer( - &self, - region_id: RegionId, - region_to_peer: &HashMap, - ) -> Result { - region_to_peer - .get(®ion_id) - .cloned() - .with_context(|| RegionRouteNotFoundSnafu { region_id }) - } /// Aggregate GC candidates by their corresponding datanode peer. pub(crate) async fn aggregate_candidates_by_datanode( &self, diff --git a/src/meta-srv/src/gc/mock.rs b/src/meta-srv/src/gc/mock.rs index ff357635a356..9a762d3bf13f 100644 --- a/src/meta-srv/src/gc/mock.rs +++ b/src/meta-srv/src/gc/mock.rs @@ -376,6 +376,7 @@ pub struct TestEnv { tx: Sender, } +#[allow(unused)] impl TestEnv { pub fn new() -> Self { let ctx = Arc::new(MockSchedulerCtx::default()); diff --git a/src/meta-srv/src/gc/mock/basic.rs b/src/meta-srv/src/gc/mock/basic.rs index c5ef7a3a0a35..c035dd991234 100644 --- a/src/meta-srv/src/gc/mock/basic.rs +++ b/src/meta-srv/src/gc/mock/basic.rs @@ -73,8 +73,6 @@ async fn test_process_datanodes_concurrently_with_candidates() { let mut scheduler = env.scheduler; scheduler.ctx = Arc::new(ctx); - let candidates = HashMap::from([(1, vec![new_candidate(region_id, 1.0)])]); - // Convert table-based candidates to datanode-based candidates let datanode_to_candidates = HashMap::from([( peer, diff --git a/src/meta-srv/src/gc/scheduler.rs b/src/meta-srv/src/gc/scheduler.rs index 097570e7a4ae..264cf24da7bd 100644 --- a/src/meta-srv/src/gc/scheduler.rs +++ b/src/meta-srv/src/gc/scheduler.rs @@ -19,31 +19,17 @@ use std::time::Instant; use common_meta::key::TableMetadataManagerRef; use common_procedure::ProcedureManagerRef; use common_telemetry::{error, info}; -use futures::stream::StreamExt; -use store_api::storage::{GcReport, RegionId}; -use table::metadata::TableId; +use store_api::storage::GcReport; use tokio::sync::mpsc::{Receiver, Sender}; use crate::cluster::MetaPeerClientRef; use crate::define_ticker; use crate::error::Result; -use crate::gc::candidate::GcCandidate; use crate::gc::ctx::{DefaultGcSchedulerCtx, SchedulerCtx}; use crate::gc::options::{GcSchedulerOptions, TICKER_INTERVAL}; use crate::gc::tracker::RegionGcTracker; use crate::service::mailbox::MailboxRef; -/// Report for a table garbage collection. -#[derive(Debug, Default)] -pub struct TableGcReport { - #[allow(unused)] - pub table_id: TableId, - /// Successful GC reports for each region. - pub success_regions: Vec, - /// Failed GC reports for each region. - pub failed_regions: HashMap, -} - /// Report for a GC job. #[derive(Debug, Default)] pub struct GcJobReport { @@ -54,7 +40,7 @@ impl GcJobReport { pub fn merge(&mut self, mut other: GcJobReport) { // merge per_datanode_reports&failed_datanodes for (dn_id, report) in other.per_datanode_reports { - let mut self_report = self.per_datanode_reports.entry(dn_id).or_default(); + let self_report = self.per_datanode_reports.entry(dn_id).or_default(); self_report.merge(report); } let all_failed_dn_ids = self @@ -171,14 +157,4 @@ impl GcScheduler { Ok(report) } - - /// Process multiple tables concurrently with limited parallelism. - /// - /// TODO(discord9): acquire lock for prevent region migration during gc. - pub(crate) async fn process_tables_concurrently( - &self, - per_table_candidates: HashMap>, - ) -> GcJobReport { - unimplemented!("TODO: remove this unused") - } } diff --git a/src/mito2/src/sst/file_purger.rs b/src/mito2/src/sst/file_purger.rs index 0b31f9b9a79e..81a004ff15ee 100644 --- a/src/mito2/src/sst/file_purger.rs +++ b/src/mito2/src/sst/file_purger.rs @@ -162,7 +162,7 @@ impl FilePurger for ObjectStoreFilePurger { // if not on local file system, instead inform the global file purger to remove the file reference. // notice that no matter whether the file is deleted or not, we need to remove the reference // because the file is no longer in use nonetheless. - self.file_ref_manager.remove_file(file_meta.file_id()); + self.file_ref_manager.remove_file(&file_meta); // TODO(discord9): consider impl a .tombstone file to reduce files needed to list } diff --git a/src/mito2/src/sst/file_ref.rs b/src/mito2/src/sst/file_ref.rs index 5929b51789b2..a281aeb5fa90 100644 --- a/src/mito2/src/sst/file_ref.rs +++ b/src/mito2/src/sst/file_ref.rs @@ -22,7 +22,7 @@ use store_api::storage::{FileRef, FileRefsManifest, RegionId}; use crate::error::Result; use crate::metrics::GC_REF_FILE_CNT; use crate::region::MitoRegionRef; -use crate::sst::file::{FileMeta, RegionFileId}; +use crate::sst::file::FileMeta; /// File references for a region. /// It contains all files referenced by the region. @@ -269,7 +269,7 @@ mod tests { expected_region_ref_manifest ); - file_ref_mgr.remove_file(file_meta.file_id()); + file_ref_mgr.remove_file(&file_meta); assert_eq!( file_ref_mgr @@ -285,7 +285,7 @@ mod tests { expected_region_ref_manifest ); - file_ref_mgr.remove_file(file_meta.file_id()); + file_ref_mgr.remove_file(&file_meta); assert!( file_ref_mgr From 799ff6005c760b6970e31f054ed2f8d96040292d Mon Sep 17 00:00:00 2001 From: discord9 Date: Tue, 18 Nov 2025 15:47:20 +0800 Subject: [PATCH 08/10] typo Signed-off-by: discord9 --- src/meta-srv/src/gc/handler.rs | 8 ++++---- src/meta-srv/src/gc/mock/retry.rs | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/src/meta-srv/src/gc/handler.rs b/src/meta-srv/src/gc/handler.rs index 16016292161c..6f07ffe4a580 100644 --- a/src/meta-srv/src/gc/handler.rs +++ b/src/meta-srv/src/gc/handler.rs @@ -251,7 +251,7 @@ impl GcScheduler { // Step 2: Create a single GcRegionProcedure for all regions on this datanode - let (gc_report, fullly_listed_regions) = { + let (gc_report, fully_listed_regions) = { // Partition regions into full listing and fast listing in a single pass let mut need_full_list_regions = Vec::new(); let mut fast_list_regions = Vec::new(); @@ -320,12 +320,12 @@ impl GcScheduler { } } } - let fullly_listed_regions = need_full_list_regions + let fully_listed_regions = need_full_list_regions .into_iter() .filter(|r| !combined_report.need_retry_regions.contains(r)) .collect::>(); - (combined_report, fullly_listed_regions) + (combined_report, fully_listed_regions) }; // Step 3: Process the combined GC report and update table reports @@ -338,7 +338,7 @@ impl GcScheduler { .or_insert_with(|| RegionGcInfo::new(now)); gc_info.last_gc_time = now; // notice need to set last_full_listing_time if full listing was used - if fullly_listed_regions.contains(region_id) { + if fully_listed_regions.contains(region_id) { gc_info.last_full_listing_time = Some(now); } } diff --git a/src/meta-srv/src/gc/mock/retry.rs b/src/meta-srv/src/gc/mock/retry.rs index 09e5439c4348..89d958095956 100644 --- a/src/meta-srv/src/gc/mock/retry.rs +++ b/src/meta-srv/src/gc/mock/retry.rs @@ -598,9 +598,9 @@ async fn test_need_retry_regions_mixed_scenarios() { // Process each region separately with its corresponding peer and check results directly for (region_id, peer) in routes { - let dn2candiate = + let dn2candidate = HashMap::from([(peer.clone(), vec![(table_id, mock_candidate(region_id))])]); - let report = scheduler.process_datanodes_with_retry(dn2candiate).await; + let report = scheduler.process_datanodes_with_retry(dn2candidate).await; let report = report.per_datanode_reports.get(&peer.id).unwrap(); From c2364bb555f0fbacbc6dca53ca94404c4457c6c7 Mon Sep 17 00:00:00 2001 From: discord9 Date: Wed, 19 Nov 2025 17:09:02 +0800 Subject: [PATCH 09/10] chore: after rebase fix Signed-off-by: discord9 --- src/meta-srv/src/gc/ctx.rs | 7 ------ src/meta-srv/src/gc/handler.rs | 42 ++++++++++++++++++++++------------ src/meta-srv/src/gc/mock.rs | 7 +++--- src/mito2/src/gc.rs | 4 ---- 4 files changed, 32 insertions(+), 28 deletions(-) diff --git a/src/meta-srv/src/gc/ctx.rs b/src/meta-srv/src/gc/ctx.rs index 4a160ef1b516..8583b6375bd6 100644 --- a/src/meta-srv/src/gc/ctx.rs +++ b/src/meta-srv/src/gc/ctx.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. - - use std::collections::{HashMap, HashSet}; use std::time::Duration; @@ -34,11 +32,6 @@ use table::metadata::TableId; use crate::cluster::MetaPeerClientRef; use crate::error::{self, Result, TableMetadataManagerSnafu, UnexpectedSnafu}; use crate::gc::Region2Peers; - -use crate::cluster::MetaPeerClientRef; -use crate::error; -use crate::error::{Result, TableMetadataManagerSnafu}; -use crate::gc::handler::Region2Peers; use crate::gc::procedure::GcRegionProcedure; use crate::handler::HeartbeatMailbox; use crate::service::mailbox::{Channel, MailboxRef}; diff --git a/src/meta-srv/src/gc/handler.rs b/src/meta-srv/src/gc/handler.rs index 6f07ffe4a580..cb3d8493743e 100644 --- a/src/meta-srv/src/gc/handler.rs +++ b/src/meta-srv/src/gc/handler.rs @@ -19,6 +19,7 @@ use common_meta::key::table_route::PhysicalTableRouteValue; use common_meta::peer::Peer; use common_telemetry::{debug, error, info, warn}; use futures::StreamExt; +use itertools::Itertools; use store_api::storage::{FileRefsManifest, GcReport, RegionId}; use table::metadata::TableId; use tokio::time::sleep; @@ -88,8 +89,8 @@ impl GcScheduler { pub(crate) async fn find_related_regions( &self, candidate_region_ids: &[RegionId], - ) -> Result> { - Ok(candidate_region_ids.to_vec()) + ) -> Result>> { + Ok(candidate_region_ids.iter().map(|&r| (r, vec![r])).collect()) } /// Aggregate GC candidates by their corresponding datanode peer. @@ -235,8 +236,10 @@ impl GcScheduler { let all_region_ids: Vec = candidates.iter().map(|(_, c)| c.region_id).collect(); + let all_related_regions = self.find_related_regions(&all_region_ids).await?; + let (region_to_peer, _) = self - .discover_datanodes_for_regions(&all_region_ids, 0) + .discover_datanodes_for_regions(&all_related_regions.keys().cloned().collect_vec(), 0) .await?; // Step 1: Get file references for all regions on this datanode @@ -244,6 +247,7 @@ impl GcScheduler { .ctx .get_file_references( &all_region_ids, + all_related_regions, ®ion_to_peer, self.config.mailbox_timeout, ) @@ -407,7 +411,7 @@ impl GcScheduler { // Step 2: Process retry regions by calling gc_regions on rediscovered datanodes (batched) let retry_result = self .process_retry_regions_by_peers(region_to_peer, peer_to_regions, retry_round) - .await; + .await?; // Merge the retry results into the final report final_report.merge(retry_result); @@ -429,19 +433,29 @@ impl GcScheduler { Ok(final_report) } - /// Discover datanodes for the given regions by fetching table routes in batches. + /// Discover datanodes for the given regions(and it's related regions) by fetching table routes in batches. /// Returns mappings from region to peer(leader, Vec) and peer to regions. async fn discover_datanodes_for_regions( &self, regions: &[RegionId], retry_round: usize, ) -> Result<(Region2Peers, Peer2Regions)> { + let all_related_regions = self + .find_related_regions(®ions) + .await? + .into_iter() + .map(|(k, mut v)| { + v.push(k); + v + }) + .flatten() + .collect_vec(); let mut region_to_peer = HashMap::new(); let mut peer_to_regions = HashMap::new(); // Group regions by table ID for batch processing let mut table_to_regions: HashMap> = HashMap::new(); - for ®ion_id in regions { + for region_id in all_related_regions { let table_id = region_id.table_id(); table_to_regions .entry(table_id) @@ -520,7 +534,7 @@ impl GcScheduler { region_to_peer: Region2Peers, peer_to_regions: Peer2Regions, retry_round: usize, - ) -> GcJobReport { + ) -> Result { let mut round_report = GcJobReport::default(); for (peer, regions) in peer_to_regions { @@ -531,17 +545,17 @@ impl GcScheduler { retry_round ); - let all_related_regions = self - .find_related_regions(®ions.iter().cloned().collect::>()) - .await - .unwrap_or_default(); + let query_regions = regions.clone().into_iter().collect::>(); + + let all_related_regions = self.find_related_regions(&query_regions).await?; // Get fresh file references for these regions let file_refs_manifest = match self .ctx .get_file_references( - &all_related_regions, - ®ion_to_peer, + &query_regions, + all_related_regions, + ®ion_to_peer, // FIXME: update region_to_peer? self.config.mailbox_timeout, ) .await @@ -576,7 +590,7 @@ impl GcScheduler { } } - round_report + Ok(round_report) } /// Process a batch of regions on a single peer. diff --git a/src/meta-srv/src/gc/mock.rs b/src/meta-srv/src/gc/mock.rs index 9a762d3bf13f..3df11c980868 100644 --- a/src/meta-srv/src/gc/mock.rs +++ b/src/meta-srv/src/gc/mock.rs @@ -212,7 +212,8 @@ impl SchedulerCtx for MockSchedulerCtx { async fn get_file_references( &self, - region_ids: &[RegionId], + query_regions: &[RegionId], + related_regions: HashMap>, region_to_peer: &Region2Peers, _timeout: Duration, ) -> Result { @@ -222,14 +223,14 @@ impl SchedulerCtx for MockSchedulerCtx { if let Some(error) = self.get_file_references_error.lock().unwrap().take() { return Err(error); } - if region_ids + if query_regions .iter() .any(|region_id| !region_to_peer.contains_key(region_id)) { UnexpectedSnafu { violated: format!( "region_to_peer{region_to_peer:?} does not contain all region_ids requested: {:?}", - region_ids + query_regions ), }.fail()?; } diff --git a/src/mito2/src/gc.rs b/src/mito2/src/gc.rs index 7c33771d94ba..b9285c183b4f 100644 --- a/src/mito2/src/gc.rs +++ b/src/mito2/src/gc.rs @@ -322,8 +322,6 @@ impl LocalGcWorker { let region_id = manifest.metadata.region_id; let current_files = &manifest.files; - let in_manifest_file_cnt = current_files.len(); - let recently_removed_files = self.get_removed_files_expel_times(&manifest).await?; if recently_removed_files.is_empty() { @@ -346,8 +344,6 @@ impl LocalGcWorker { .chain(tmp_ref_files.clone().into_iter()) .collect(); - let in_used_file_cnt = in_used.len(); - let unused_files = self .list_to_be_deleted_files(region_id, &in_used, recently_removed_files, concurrency) .await?; From 8f77c6ede8d2070d1a0a136e80b8dca05ea6ac66 Mon Sep 17 00:00:00 2001 From: discord9 Date: Wed, 19 Nov 2025 17:25:54 +0800 Subject: [PATCH 10/10] choer: more fix Signed-off-by: discord9 --- src/meta-srv/src/gc/handler.rs | 31 +++++++++++++------------- src/meta-srv/src/gc/mock/err_handle.rs | 2 +- src/meta-srv/src/gc/mock/retry.rs | 1 - 3 files changed, 16 insertions(+), 18 deletions(-) diff --git a/src/meta-srv/src/gc/handler.rs b/src/meta-srv/src/gc/handler.rs index cb3d8493743e..baa37ce9fd53 100644 --- a/src/meta-srv/src/gc/handler.rs +++ b/src/meta-srv/src/gc/handler.rs @@ -441,14 +441,13 @@ impl GcScheduler { retry_round: usize, ) -> Result<(Region2Peers, Peer2Regions)> { let all_related_regions = self - .find_related_regions(®ions) + .find_related_regions(regions) .await? .into_iter() - .map(|(k, mut v)| { + .flat_map(|(k, mut v)| { v.push(k); v }) - .flatten() .collect_vec(); let mut region_to_peer = HashMap::new(); let mut peer_to_regions = HashMap::new(); @@ -503,19 +502,19 @@ impl GcScheduler { // Find the region in the table route for region_route in &table_route.region_routes { - if region_route.region.id == region_id { - if let Some(leader_peer) = ®ion_route.leader_peer { - region_to_peer.insert( - region_id, - (leader_peer.clone(), region_route.follower_peers.clone()), - ); - peer_to_regions - .entry(leader_peer.clone()) - .or_default() - .insert(region_id); - found = true; - break; - } + if region_route.region.id == region_id + && let Some(leader_peer) = ®ion_route.leader_peer + { + region_to_peer.insert( + region_id, + (leader_peer.clone(), region_route.follower_peers.clone()), + ); + peer_to_regions + .entry(leader_peer.clone()) + .or_default() + .insert(region_id); + found = true; + break; } } diff --git a/src/meta-srv/src/gc/mock/err_handle.rs b/src/meta-srv/src/gc/mock/err_handle.rs index e31892f2bb80..0c3fd5ec7001 100644 --- a/src/meta-srv/src/gc/mock/err_handle.rs +++ b/src/meta-srv/src/gc/mock/err_handle.rs @@ -269,7 +269,7 @@ async fn test_get_table_route_failure() { let report = scheduler .process_datanodes_with_retry(datanode_to_candidates) .await; - dbg!(&report); + // Should process the datanode but handle route error gracefully assert_eq!( report.per_datanode_reports.len(), diff --git a/src/meta-srv/src/gc/mock/retry.rs b/src/meta-srv/src/gc/mock/retry.rs index 89d958095956..ae5e7216087c 100644 --- a/src/meta-srv/src/gc/mock/retry.rs +++ b/src/meta-srv/src/gc/mock/retry.rs @@ -440,7 +440,6 @@ async fn test_need_retry_regions_exponential_backoff() { // Extract the datanode report from the job report let report = job_report.per_datanode_reports.get(&peer.id).unwrap(); - dbg!(&report); // Verify results assert!(report.deleted_files.contains_key(®ion_id)); assert!(report.need_retry_regions.is_empty());