mito2/compaction/
task.rs

1// Copyright 2023 Greptime Team
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use 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
40/// Maximum number of compaction tasks in parallel.
41pub const MAX_PARALLEL_COMPACTION: usize = 1;
42
43pub(crate) struct CompactionTaskImpl {
44    pub compaction_region: CompactionRegion,
45    /// Request sender to notify the worker.
46    pub(crate) request_sender: mpsc::Sender<WorkerRequestWithTime>,
47    /// Senders that are used to notify waiters waiting for pending compaction tasks.
48    pub waiters: Vec<OutputTx>,
49    /// Start time of compaction task
50    pub start_time: Instant,
51    /// Event listener.
52    pub(crate) listener: WorkerListener,
53    /// Compactor to handle compaction.
54    pub(crate) compactor: Arc<dyn Compactor>,
55    /// Output of the picker.
56    pub(crate) picker_output: PickerOutput,
57    /// Memory manager to acquire memory budget.
58    pub(crate) memory_manager: Arc<CompactionMemoryManager>,
59    /// Policy when memory is exhausted.
60    pub(crate) memory_policy: OnExhaustedPolicy,
61    /// Estimated memory bytes needed for this compaction.
62    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    /// Acquires memory budget based on the configured policy.
93    ///
94    /// Returns an error if memory cannot be acquired according to the policy.
95    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    /// Remove expired ssts files, update manifest immediately
111    /// and apply the edit to region version.
112    ///
113    /// This function logs errors but does not stop the compaction process if removal fails.
114    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        // Update manifest to remove expired SSTs
123        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        // 1. Update manifest
134        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        // 2. Notify region worker loop to remove expired files from region version.
157        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        // 1. In case of local compaction, we can delete expired ssts in advance.
190        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            // remove_expired logs errors but doesn't stop compaction
201            self.remove_expired(&self.compaction_region, expired_ssts)
202                .await;
203            remove_timer.observe_duration();
204        }
205
206        // 2. Merge inputs
207        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    /// Handles compaction failure, notifies all waiters.
251    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    /// Notifies region worker to handle post-compaction tasks.
261    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        // Acquire memory budget before starting compaction
279        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                // notify compaction waiters
309                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        // Test that PickerOutput correctly includes expired_ssts
335        // This verifies that expired SSTs are properly identified and included
336        // in the picker output, which is then handled by handle_expiration_and_compaction
337
338        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        // Verify expired_ssts are included
352        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        // Test that PickerOutput works correctly when there are no expired SSTs
366        let picker_output = PickerOutput {
367            outputs: vec![],
368            expired_ssts: vec![],
369            time_window_size: 3600,
370            max_file_size: None,
371        };
372
373        // Verify empty expired_ssts
374        assert!(picker_output.expired_ssts.is_empty());
375    }
376
377    // Note: Testing remove_expired() directly requires extensive mocking of:
378    // - manifest_ctx (ManifestContext)
379    // - request_sender (mpsc::Sender<WorkerRequestWithTime>)
380    // - WorkerRequest handling
381    //
382    // The behavior is tested indirectly through integration tests:
383    // - remove_expired() logs errors but doesn't stop compaction
384    // - handle_expiration_and_compaction() continues even if remove_expired() encounters errors
385    // - The function is designed to be non-blocking for compaction
386}