1use std::fmt::{Debug, Formatter};
16use std::sync::Arc;
17use std::time::Instant;
18
19use common_memory_manager::OnExhaustedPolicy;
20use common_telemetry::{error, info, warn};
21use itertools::Itertools;
22use snafu::ResultExt;
23use tokio::sync::mpsc;
24
25use crate::compaction::compactor::{CompactionRegion, Compactor};
26use crate::compaction::memory_manager::{CompactionMemoryGuard, CompactionMemoryManager};
27use crate::compaction::picker::{CompactionTask, PickerOutput};
28use crate::error::{CompactRegionSnafu, CompactionMemoryExhaustedSnafu};
29use crate::manifest::action::{RegionEdit, RegionMetaAction, RegionMetaActionList};
30use crate::metrics::{COMPACTION_FAILURE_COUNT, COMPACTION_MEMORY_WAIT, COMPACTION_STAGE_ELAPSED};
31use crate::region::RegionRoleState;
32use crate::request::{
33 BackgroundNotify, CompactionFailed, CompactionFinished, OutputTx, RegionEditResult,
34 WorkerRequest, WorkerRequestWithTime,
35};
36use crate::sst::file::FileMeta;
37use crate::worker::WorkerListener;
38use crate::{error, metrics};
39
40pub const MAX_PARALLEL_COMPACTION: usize = 1;
42
43pub(crate) struct CompactionTaskImpl {
44 pub compaction_region: CompactionRegion,
45 pub(crate) request_sender: mpsc::Sender<WorkerRequestWithTime>,
47 pub waiters: Vec<OutputTx>,
49 pub start_time: Instant,
51 pub(crate) listener: WorkerListener,
53 pub(crate) compactor: Arc<dyn Compactor>,
55 pub(crate) picker_output: PickerOutput,
57 pub(crate) memory_manager: Arc<CompactionMemoryManager>,
59 pub(crate) memory_policy: OnExhaustedPolicy,
61 pub(crate) estimated_memory_bytes: u64,
63}
64
65impl Debug for CompactionTaskImpl {
66 fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
67 f.debug_struct("TwcsCompactionTask")
68 .field("region_id", &self.compaction_region.region_id)
69 .field("picker_output", &self.picker_output)
70 .field(
71 "append_mode",
72 &self.compaction_region.region_options.append_mode,
73 )
74 .finish()
75 }
76}
77
78impl Drop for CompactionTaskImpl {
79 fn drop(&mut self) {
80 self.mark_files_compacting(false)
81 }
82}
83
84impl CompactionTaskImpl {
85 fn mark_files_compacting(&self, compacting: bool) {
86 self.picker_output
87 .outputs
88 .iter()
89 .for_each(|o| o.inputs.iter().for_each(|f| f.set_compacting(compacting)));
90 }
91
92 async fn acquire_memory_with_policy(&self) -> error::Result<CompactionMemoryGuard> {
96 let region_id = self.compaction_region.region_id;
97 let requested_bytes = self.estimated_memory_bytes;
98 let policy = self.memory_policy;
99
100 let _timer = COMPACTION_MEMORY_WAIT.start_timer();
101 self.memory_manager
102 .acquire_with_policy(requested_bytes, policy)
103 .await
104 .context(CompactionMemoryExhaustedSnafu {
105 region_id,
106 policy: format!("{policy:?}"),
107 })
108 }
109
110 async fn remove_expired(
115 &self,
116 compaction_region: &CompactionRegion,
117 expired_files: Vec<FileMeta>,
118 ) {
119 let region_id = compaction_region.region_id;
120 let expired_files_str = expired_files.iter().map(|f| f.file_id).join(",");
121 let (expire_delete_sender, expire_delete_listener) = tokio::sync::oneshot::channel();
122 let edit = RegionEdit {
124 files_to_add: Vec::new(),
125 files_to_remove: expired_files,
126 timestamp_ms: Some(chrono::Utc::now().timestamp_millis()),
127 compaction_time_window: None,
128 flushed_entry_id: None,
129 flushed_sequence: None,
130 committed_sequence: None,
131 };
132
133 let action_list = RegionMetaActionList::with_action(RegionMetaAction::Edit(edit.clone()));
135 let RegionRoleState::Leader(current_region_state) =
136 compaction_region.manifest_ctx.current_state()
137 else {
138 warn!(
139 "Region {} not in leader state, skip removing expired files",
140 region_id
141 );
142 return;
143 };
144 if let Err(e) = compaction_region
145 .manifest_ctx
146 .update_manifest(current_region_state, action_list, false)
147 .await
148 {
149 warn!(
150 e;
151 "Failed to update manifest for expired files removal, region: {region_id}, files: [{expired_files_str}]. Compaction will continue."
152 );
153 return;
154 }
155
156 self.send_to_worker(WorkerRequest::Background {
158 region_id,
159 notify: BackgroundNotify::RegionEdit(RegionEditResult {
160 region_id,
161 sender: expire_delete_sender,
162 edit,
163 result: Ok(()),
164 update_region_state: false,
165 }),
166 })
167 .await;
168
169 if let Err(e) = expire_delete_listener
170 .await
171 .context(error::RecvSnafu)
172 .flatten()
173 {
174 warn!(
175 e;
176 "Failed to remove expired files from region version, region: {region_id}, files: [{expired_files_str}]. Compaction will continue."
177 );
178 return;
179 }
180
181 info!(
182 "Successfully removed expired files, region: {region_id}, files: [{expired_files_str}]"
183 );
184 }
185
186 async fn handle_expiration_and_compaction(&mut self) -> error::Result<RegionEdit> {
187 self.mark_files_compacting(true);
188
189 if !self.picker_output.expired_ssts.is_empty() {
191 let remove_timer = COMPACTION_STAGE_ELAPSED
192 .with_label_values(&["remove_expired"])
193 .start_timer();
194 let expired_ssts = self
195 .picker_output
196 .expired_ssts
197 .drain(..)
198 .map(|f| f.meta_ref().clone())
199 .collect();
200 self.remove_expired(&self.compaction_region, expired_ssts)
202 .await;
203 remove_timer.observe_duration();
204 }
205
206 let merge_timer = COMPACTION_STAGE_ELAPSED
208 .with_label_values(&["merge"])
209 .start_timer();
210
211 let compaction_result = match self
212 .compactor
213 .merge_ssts(&self.compaction_region, self.picker_output.clone())
214 .await
215 {
216 Ok(v) => v,
217 Err(e) => {
218 error!(e; "Failed to compact region: {}", self.compaction_region.region_id);
219 merge_timer.stop_and_discard();
220 return Err(e);
221 }
222 };
223 let merge_time = merge_timer.stop_and_record();
224
225 metrics::COMPACTION_INPUT_BYTES.inc_by(compaction_result.input_file_size() as f64);
226 metrics::COMPACTION_OUTPUT_BYTES.inc_by(compaction_result.output_file_size() as f64);
227 info!(
228 "Compacted SST files, region_id: {}, input: {:?}, output: {:?}, window: {:?}, waiter_num: {}, merge_time: {}s",
229 self.compaction_region.region_id,
230 compaction_result.files_to_remove,
231 compaction_result.files_to_add,
232 compaction_result.compaction_time_window,
233 self.waiters.len(),
234 merge_time,
235 );
236
237 self.listener
238 .on_merge_ssts_finished(self.compaction_region.region_id)
239 .await;
240
241 let _manifest_timer = COMPACTION_STAGE_ELAPSED
242 .with_label_values(&["write_manifest"])
243 .start_timer();
244
245 self.compactor
246 .update_manifest(&self.compaction_region, compaction_result)
247 .await
248 }
249
250 pub(crate) fn on_failure(&mut self, err: Arc<error::Error>) {
252 COMPACTION_FAILURE_COUNT.inc();
253 for waiter in self.waiters.drain(..) {
254 waiter.send(Err(err.clone()).context(CompactRegionSnafu {
255 region_id: self.compaction_region.region_id,
256 }));
257 }
258 }
259
260 async fn send_to_worker(&self, request: WorkerRequest) {
262 if let Err(e) = self
263 .request_sender
264 .send(WorkerRequestWithTime::new(request))
265 .await
266 {
267 error!(
268 "Failed to notify compaction job status for region {}, request: {:?}",
269 self.compaction_region.region_id, e.0
270 );
271 }
272 }
273}
274
275#[async_trait::async_trait]
276impl CompactionTask for CompactionTaskImpl {
277 async fn run(&mut self) {
278 let _memory_guard = match self.acquire_memory_with_policy().await {
280 Ok(guard) => guard,
281 Err(e) => {
282 error!(e; "Failed to acquire memory for compaction, region id: {}", self.compaction_region.region_id);
283 let err = Arc::new(e);
284 self.on_failure(err.clone());
285 let notify = BackgroundNotify::CompactionFailed(CompactionFailed {
286 region_id: self.compaction_region.region_id,
287 err,
288 });
289 self.send_to_worker(WorkerRequest::Background {
290 region_id: self.compaction_region.region_id,
291 notify,
292 })
293 .await;
294 return;
295 }
296 };
297
298 let notify = match self.handle_expiration_and_compaction().await {
299 Ok(edit) => BackgroundNotify::CompactionFinished(CompactionFinished {
300 region_id: self.compaction_region.region_id,
301 senders: std::mem::take(&mut self.waiters),
302 start_time: self.start_time,
303 edit,
304 }),
305 Err(e) => {
306 error!(e; "Failed to compact region, region id: {}", self.compaction_region.region_id);
307 let err = Arc::new(e);
308 self.on_failure(err.clone());
310 BackgroundNotify::CompactionFailed(CompactionFailed {
311 region_id: self.compaction_region.region_id,
312 err,
313 })
314 }
315 };
316
317 self.send_to_worker(WorkerRequest::Background {
318 region_id: self.compaction_region.region_id,
319 notify,
320 })
321 .await;
322 }
323}
324
325#[cfg(test)]
326mod tests {
327 use store_api::storage::FileId;
328
329 use crate::compaction::picker::PickerOutput;
330 use crate::compaction::test_util::new_file_handle;
331
332 #[test]
333 fn test_picker_output_with_expired_ssts() {
334 let file_ids = (0..3).map(|_| FileId::random()).collect::<Vec<_>>();
339 let expired_ssts = vec![
340 new_file_handle(file_ids[0], 0, 999, 0),
341 new_file_handle(file_ids[1], 1000, 1999, 0),
342 ];
343
344 let picker_output = PickerOutput {
345 outputs: vec![],
346 expired_ssts: expired_ssts.clone(),
347 time_window_size: 3600,
348 max_file_size: None,
349 };
350
351 assert_eq!(picker_output.expired_ssts.len(), 2);
353 assert_eq!(
354 picker_output.expired_ssts[0].file_id(),
355 expired_ssts[0].file_id()
356 );
357 assert_eq!(
358 picker_output.expired_ssts[1].file_id(),
359 expired_ssts[1].file_id()
360 );
361 }
362
363 #[test]
364 fn test_picker_output_without_expired_ssts() {
365 let picker_output = PickerOutput {
367 outputs: vec![],
368 expired_ssts: vec![],
369 time_window_size: 3600,
370 max_file_size: None,
371 };
372
373 assert!(picker_output.expired_ssts.is_empty());
375 }
376
377 }