Skip to main content

mito2/
compaction.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
15mod buckets;
16pub mod compactor;
17pub mod memory_manager;
18pub mod picker;
19pub mod run;
20mod task;
21#[cfg(test)]
22mod test_util;
23mod twcs;
24mod window;
25
26use std::collections::HashMap;
27use std::sync::Arc;
28use std::time::Instant;
29
30use api::v1::region::compact_request;
31use api::v1::region::compact_request::Options;
32use common_base::Plugins;
33use common_memory_manager::OnExhaustedPolicy;
34use common_meta::key::SchemaMetadataManagerRef;
35use common_telemetry::{debug, error, info, warn};
36use common_time::range::TimestampRange;
37use common_time::timestamp::TimeUnit;
38use common_time::{TimeToLive, Timestamp};
39use datafusion_common::ScalarValue;
40use datafusion_expr::Expr;
41use serde::{Deserialize, Serialize};
42use snafu::{OptionExt, ResultExt};
43use store_api::metadata::RegionMetadataRef;
44use store_api::storage::{RegionId, TableId};
45use task::MAX_PARALLEL_COMPACTION;
46use tokio::sync::mpsc::{self, Sender};
47
48use crate::access_layer::AccessLayerRef;
49use crate::cache::{CacheManagerRef, CacheStrategy};
50use crate::compaction::compactor::{CompactionRegion, CompactionVersion, DefaultCompactor};
51use crate::compaction::memory_manager::CompactionMemoryManager;
52use crate::compaction::picker::{CompactionTask, PickerOutput, new_picker};
53use crate::compaction::task::CompactionTaskImpl;
54use crate::config::MitoConfig;
55use crate::error::{
56    CompactRegionSnafu, Error, GetSchemaMetadataSnafu, ManualCompactionOverrideSnafu,
57    RegionClosedSnafu, RegionDroppedSnafu, RegionTruncatedSnafu, RemoteCompactionSnafu, Result,
58    TimeRangePredicateOverflowSnafu, TimeoutSnafu,
59};
60use crate::metrics::{COMPACTION_STAGE_ELAPSED, INFLIGHT_COMPACTION_COUNT};
61use crate::read::BoxedRecordBatchStream;
62use crate::read::projection::ProjectionMapper;
63use crate::read::scan_region::{PredicateGroup, ScanInput};
64use crate::read::seq_scan::SeqScan;
65use crate::region::options::{MergeMode, RegionOptions};
66use crate::region::version::VersionControlRef;
67use crate::region::{ManifestContextRef, RegionLeaderState, RegionRoleState};
68use crate::request::{OptionOutputTx, OutputTx, SenderDdlRequest, WorkerRequestWithTime};
69use crate::schedule::remote_job_scheduler::{
70    CompactionJob, DefaultNotifier, RemoteJob, RemoteJobSchedulerRef,
71};
72use crate::schedule::scheduler::SchedulerRef;
73use crate::sst::file::{FileHandle, FileMeta, Level};
74use crate::sst::version::LevelMeta;
75use crate::worker::WorkerListener;
76
77/// Region compaction request.
78pub struct CompactionRequest {
79    pub(crate) engine_config: Arc<MitoConfig>,
80    pub(crate) current_version: CompactionVersion,
81    pub(crate) access_layer: AccessLayerRef,
82    /// Sender to send notification to the region worker.
83    pub(crate) request_sender: mpsc::Sender<WorkerRequestWithTime>,
84    /// Waiters of the compaction request.
85    pub(crate) waiters: Vec<OutputTx>,
86    /// Start time of compaction task.
87    pub(crate) start_time: Instant,
88    pub(crate) cache_manager: CacheManagerRef,
89    pub(crate) manifest_ctx: ManifestContextRef,
90    pub(crate) listener: WorkerListener,
91    pub(crate) schema_metadata_manager: SchemaMetadataManagerRef,
92    pub(crate) max_parallelism: usize,
93}
94
95impl CompactionRequest {
96    pub(crate) fn region_id(&self) -> RegionId {
97        self.current_version.metadata.region_id
98    }
99}
100
101/// Compaction scheduler tracks and manages compaction tasks.
102pub(crate) struct CompactionScheduler {
103    scheduler: SchedulerRef,
104    /// Compacting regions.
105    region_status: HashMap<RegionId, CompactionStatus>,
106    /// Request sender of the worker that this scheduler belongs to.
107    request_sender: Sender<WorkerRequestWithTime>,
108    cache_manager: CacheManagerRef,
109    engine_config: Arc<MitoConfig>,
110    memory_manager: Arc<CompactionMemoryManager>,
111    memory_policy: OnExhaustedPolicy,
112    listener: WorkerListener,
113    /// Plugins for the compaction scheduler.
114    plugins: Plugins,
115}
116
117impl CompactionScheduler {
118    #[allow(clippy::too_many_arguments)]
119    pub(crate) fn new(
120        scheduler: SchedulerRef,
121        request_sender: Sender<WorkerRequestWithTime>,
122        cache_manager: CacheManagerRef,
123        engine_config: Arc<MitoConfig>,
124        listener: WorkerListener,
125        plugins: Plugins,
126        memory_manager: Arc<CompactionMemoryManager>,
127        memory_policy: OnExhaustedPolicy,
128    ) -> Self {
129        Self {
130            scheduler,
131            region_status: HashMap::new(),
132            request_sender,
133            cache_manager,
134            engine_config,
135            memory_manager,
136            memory_policy,
137            listener,
138            plugins,
139        }
140    }
141
142    /// Schedules a compaction for the region.
143    #[allow(clippy::too_many_arguments)]
144    pub(crate) async fn schedule_compaction(
145        &mut self,
146        region_id: RegionId,
147        compact_options: compact_request::Options,
148        version_control: &VersionControlRef,
149        access_layer: &AccessLayerRef,
150        waiter: OptionOutputTx,
151        manifest_ctx: &ManifestContextRef,
152        schema_metadata_manager: SchemaMetadataManagerRef,
153        max_parallelism: usize,
154    ) -> Result<()> {
155        // skip compaction if region is in staging state
156        let current_state = manifest_ctx.current_state();
157        if current_state == RegionRoleState::Leader(RegionLeaderState::Staging) {
158            info!(
159                "Skipping compaction for region {} in staging mode, options: {:?}",
160                region_id, compact_options
161            );
162            waiter.send(Ok(0));
163            return Ok(());
164        }
165
166        if let Some(status) = self.region_status.get_mut(&region_id) {
167            match compact_options {
168                Options::Regular(_) => {
169                    // Region is compacting. Add the waiter to pending list.
170                    status.merge_waiter(waiter);
171                }
172                options @ Options::StrictWindow(_) => {
173                    // Incoming compaction request is manually triggered.
174                    status.set_pending_request(PendingCompaction {
175                        options,
176                        waiter,
177                        max_parallelism,
178                    });
179                    info!(
180                        "Region {} is compacting, manually compaction will be re-scheduled.",
181                        region_id
182                    );
183                }
184            }
185            return Ok(());
186        }
187
188        // The region can compact directly.
189        let mut status: CompactionStatus =
190            CompactionStatus::new(region_id, version_control.clone(), access_layer.clone());
191        let request = status.new_compaction_request(
192            self.request_sender.clone(),
193            waiter,
194            self.engine_config.clone(),
195            self.cache_manager.clone(),
196            manifest_ctx,
197            self.listener.clone(),
198            schema_metadata_manager,
199            max_parallelism,
200        );
201
202        let result = self
203            .schedule_compaction_request(request, compact_options)
204            .await;
205        if matches!(result, Ok(true)) {
206            // Only if the compaction request is scheduled successfully,
207            // we insert the region into the status map.
208            self.region_status.insert(region_id, status);
209        }
210
211        self.listener.on_compaction_scheduled(region_id);
212        result.map(|_| ())
213    }
214
215    // Handle pending manual compaction request for the region.
216    //
217    // Returns true if should early return, false otherwise.
218    pub(crate) async fn handle_pending_compaction_request(
219        &mut self,
220        region_id: RegionId,
221        manifest_ctx: &ManifestContextRef,
222        schema_metadata_manager: SchemaMetadataManagerRef,
223    ) -> bool {
224        let Some(status) = self.region_status.get_mut(&region_id) else {
225            return true;
226        };
227
228        // If there is a pending manual compaction request, schedule it.
229        // and defer returning the pending DDL requests to the caller.
230        let Some(pending_request) = std::mem::take(&mut status.pending_request) else {
231            return false;
232        };
233
234        let PendingCompaction {
235            options,
236            waiter,
237            max_parallelism,
238        } = pending_request;
239
240        let request = {
241            status.new_compaction_request(
242                self.request_sender.clone(),
243                waiter,
244                self.engine_config.clone(),
245                self.cache_manager.clone(),
246                manifest_ctx,
247                self.listener.clone(),
248                schema_metadata_manager,
249                max_parallelism,
250            )
251        };
252
253        match self.schedule_compaction_request(request, options).await {
254            Ok(true) => {
255                debug!(
256                    "Successfully scheduled manual compaction for region id: {}",
257                    region_id
258                );
259                true
260            }
261            Ok(false) => {
262                // We still need to handle the pending DDL requests.
263                // So we can't return early here.
264                false
265            }
266            Err(e) => {
267                error!(e; "Failed to continue pending manual compaction for region id: {}", region_id);
268                self.remove_region_on_failure(region_id, Arc::new(e));
269                true
270            }
271        }
272    }
273
274    /// Notifies the scheduler that the compaction job is finished successfully.
275    pub(crate) async fn on_compaction_finished(
276        &mut self,
277        region_id: RegionId,
278        manifest_ctx: &ManifestContextRef,
279        schema_metadata_manager: SchemaMetadataManagerRef,
280    ) -> Vec<SenderDdlRequest> {
281        // If there a pending compaction request, handle it first
282        // and defer returning the pending DDL requests to the caller.
283        if self
284            .handle_pending_compaction_request(
285                region_id,
286                manifest_ctx,
287                schema_metadata_manager.clone(),
288            )
289            .await
290        {
291            return Vec::new();
292        }
293
294        let Some(status) = self.region_status.get_mut(&region_id) else {
295            // The region status might be removed by the previous steps.
296            // So we return empty DDL requests.
297            return Vec::new();
298        };
299
300        // Notify all waiters that compaction is finished.
301        for waiter in std::mem::take(&mut status.waiters) {
302            waiter.send(Ok(0));
303        }
304
305        // If there are pending DDL requests, run them.
306        let pending_ddl_requests = std::mem::take(&mut status.pending_ddl_requests);
307        if !pending_ddl_requests.is_empty() {
308            self.region_status.remove(&region_id);
309            // If there are pending DDL requests, we should return them to the caller.
310            // And skip try to schedule next compaction task.
311            return pending_ddl_requests;
312        }
313
314        // We should always try to compact the region until picker returns None.
315        let request = status.new_compaction_request(
316            self.request_sender.clone(),
317            OptionOutputTx::none(),
318            self.engine_config.clone(),
319            self.cache_manager.clone(),
320            manifest_ctx,
321            self.listener.clone(),
322            schema_metadata_manager,
323            MAX_PARALLEL_COMPACTION,
324        );
325
326        // Try to schedule next compaction task for this region.
327        match self
328            .schedule_compaction_request(
329                request,
330                compact_request::Options::Regular(Default::default()),
331            )
332            .await
333        {
334            Ok(true) => {
335                debug!(
336                    "Successfully scheduled next compaction for region id: {}",
337                    region_id
338                );
339            }
340            Ok(false) => {
341                // No further compaction tasks can be scheduled; cleanup the `CompactionStatus` for this region.
342                // All DDL requests and pending compaction requests have already been processed.
343                // Safe to remove the region from status tracking.
344                self.region_status.remove(&region_id);
345            }
346            Err(e) => {
347                error!(e; "Failed to schedule next compaction for region {}", region_id);
348                self.remove_region_on_failure(region_id, Arc::new(e));
349            }
350        }
351
352        Vec::new()
353    }
354
355    /// Notifies the scheduler that the compaction job is failed.
356    pub(crate) fn on_compaction_failed(&mut self, region_id: RegionId, err: Arc<Error>) {
357        error!(err; "Region {} failed to compact, cancel all pending tasks", region_id);
358        self.remove_region_on_failure(region_id, err);
359    }
360
361    /// Notifies the scheduler that the region is dropped.
362    pub(crate) fn on_region_dropped(&mut self, region_id: RegionId) {
363        self.remove_region_on_failure(
364            region_id,
365            Arc::new(RegionDroppedSnafu { region_id }.build()),
366        );
367    }
368
369    /// Notifies the scheduler that the region is closed.
370    pub(crate) fn on_region_closed(&mut self, region_id: RegionId) {
371        self.remove_region_on_failure(region_id, Arc::new(RegionClosedSnafu { region_id }.build()));
372    }
373
374    /// Notifies the scheduler that the region is truncated.
375    pub(crate) fn on_region_truncated(&mut self, region_id: RegionId) {
376        self.remove_region_on_failure(
377            region_id,
378            Arc::new(RegionTruncatedSnafu { region_id }.build()),
379        );
380    }
381
382    /// Add ddl request to pending queue.
383    ///
384    /// # Panics
385    /// Panics if region didn't request compaction.
386    pub(crate) fn add_ddl_request_to_pending(&mut self, request: SenderDdlRequest) {
387        debug!(
388            "Added pending DDL request for region: {}, ddl: {:?}",
389            request.region_id, request.request
390        );
391        let status = self.region_status.get_mut(&request.region_id).unwrap();
392        status.pending_ddl_requests.push(request);
393    }
394
395    #[cfg(test)]
396    pub(crate) fn has_pending_ddls(&self, region_id: RegionId) -> bool {
397        let has_pending = self
398            .region_status
399            .get(&region_id)
400            .map(|status| !status.pending_ddl_requests.is_empty())
401            .unwrap_or(false);
402        debug!(
403            "Checked pending DDL requests for region: {}, has_pending: {}",
404            region_id, has_pending
405        );
406        has_pending
407    }
408
409    /// Returns true if the region is compacting.
410    pub(crate) fn is_compacting(&self, region_id: RegionId) -> bool {
411        self.region_status.contains_key(&region_id)
412    }
413
414    /// Schedules a compaction request.
415    ///
416    /// Returns true if the compaction request is scheduled successfully.
417    /// Returns false if no compaction task can be scheduled for this region.
418    async fn schedule_compaction_request(
419        &mut self,
420        request: CompactionRequest,
421        options: compact_request::Options,
422    ) -> Result<bool> {
423        let region_id = request.region_id();
424        let (dynamic_compaction_opts, ttl) = find_dynamic_options(
425            region_id.table_id(),
426            &request.current_version.options,
427            &request.schema_metadata_manager,
428        )
429        .await
430        .unwrap_or_else(|e| {
431            warn!(e; "Failed to find dynamic options for region: {}", region_id);
432            (
433                request.current_version.options.compaction.clone(),
434                request.current_version.options.ttl.unwrap_or_default(),
435            )
436        });
437
438        let picker = new_picker(
439            &options,
440            &dynamic_compaction_opts,
441            request.current_version.options.append_mode,
442            Some(self.engine_config.max_background_compactions),
443        );
444        let region_id = request.region_id();
445        let CompactionRequest {
446            engine_config,
447            current_version,
448            access_layer,
449            request_sender,
450            waiters,
451            start_time,
452            cache_manager,
453            manifest_ctx,
454            listener,
455            schema_metadata_manager: _,
456            max_parallelism,
457        } = request;
458
459        debug!(
460            "Pick compaction strategy {:?} for region: {}, ttl: {:?}",
461            picker, region_id, ttl
462        );
463
464        let compaction_region = CompactionRegion {
465            region_id,
466            current_version: current_version.clone(),
467            region_options: RegionOptions {
468                compaction: dynamic_compaction_opts.clone(),
469                ..current_version.options.clone()
470            },
471            engine_config: engine_config.clone(),
472            region_metadata: current_version.metadata.clone(),
473            cache_manager: cache_manager.clone(),
474            access_layer: access_layer.clone(),
475            manifest_ctx: manifest_ctx.clone(),
476            file_purger: None,
477            ttl: Some(ttl),
478            max_parallelism,
479        };
480
481        let picker_output = {
482            let _pick_timer = COMPACTION_STAGE_ELAPSED
483                .with_label_values(&["pick"])
484                .start_timer();
485            picker.pick(&compaction_region)
486        };
487
488        let picker_output = if let Some(picker_output) = picker_output {
489            picker_output
490        } else {
491            // Nothing to compact, we are done. Notifies all waiters as we consume the compaction request.
492            for waiter in waiters {
493                waiter.send(Ok(0));
494            }
495            return Ok(false);
496        };
497
498        // If specified to run compaction remotely, we schedule the compaction job remotely.
499        // It will fall back to local compaction if there is no remote job scheduler.
500        let waiters = if dynamic_compaction_opts.remote_compaction() {
501            if let Some(remote_job_scheduler) = &self.plugins.get::<RemoteJobSchedulerRef>() {
502                let remote_compaction_job = CompactionJob {
503                    compaction_region: compaction_region.clone(),
504                    picker_output: picker_output.clone(),
505                    start_time,
506                    waiters,
507                    ttl,
508                };
509
510                let result = remote_job_scheduler
511                    .schedule(
512                        RemoteJob::CompactionJob(remote_compaction_job),
513                        Box::new(DefaultNotifier {
514                            request_sender: request_sender.clone(),
515                        }),
516                    )
517                    .await;
518
519                match result {
520                    Ok(job_id) => {
521                        info!(
522                            "Scheduled remote compaction job {} for region {}",
523                            job_id, region_id
524                        );
525                        INFLIGHT_COMPACTION_COUNT.inc();
526                        return Ok(true);
527                    }
528                    Err(e) => {
529                        if !dynamic_compaction_opts.fallback_to_local() {
530                            error!(e; "Failed to schedule remote compaction job for region {}", region_id);
531                            return RemoteCompactionSnafu {
532                                region_id,
533                                job_id: None,
534                                reason: e.reason,
535                            }
536                            .fail();
537                        }
538
539                        error!(e; "Failed to schedule remote compaction job for region {}, fallback to local compaction", region_id);
540
541                        // Return the waiters back to the caller for local compaction.
542                        e.waiters
543                    }
544                }
545            } else {
546                debug!(
547                    "Remote compaction is not enabled, fallback to local compaction for region {}",
548                    region_id
549                );
550                waiters
551            }
552        } else {
553            waiters
554        };
555
556        // Create a local compaction task.
557        let estimated_bytes = estimate_compaction_bytes(&picker_output);
558        let local_compaction_task = Box::new(CompactionTaskImpl {
559            request_sender,
560            waiters,
561            start_time,
562            listener,
563            picker_output,
564            compaction_region,
565            compactor: Arc::new(DefaultCompactor {}),
566            memory_manager: self.memory_manager.clone(),
567            memory_policy: self.memory_policy,
568            estimated_memory_bytes: estimated_bytes,
569        });
570
571        self.submit_compaction_task(local_compaction_task, region_id)
572            .map(|_| true)
573    }
574
575    fn submit_compaction_task(
576        &mut self,
577        mut task: Box<CompactionTaskImpl>,
578        region_id: RegionId,
579    ) -> Result<()> {
580        self.scheduler
581            .schedule(Box::pin(async move {
582                INFLIGHT_COMPACTION_COUNT.inc();
583                task.run().await;
584                INFLIGHT_COMPACTION_COUNT.dec();
585            }))
586            .inspect_err(
587                |e| error!(e; "Failed to submit compaction request for region {}", region_id),
588            )
589    }
590
591    fn remove_region_on_failure(&mut self, region_id: RegionId, err: Arc<Error>) {
592        // Remove this region.
593        let Some(status) = self.region_status.remove(&region_id) else {
594            return;
595        };
596
597        // Notifies all pending tasks.
598        status.on_failure(err);
599    }
600}
601
602impl Drop for CompactionScheduler {
603    fn drop(&mut self) {
604        for (region_id, status) in self.region_status.drain() {
605            // We are shutting down so notify all pending tasks.
606            status.on_failure(Arc::new(RegionClosedSnafu { region_id }.build()));
607        }
608    }
609}
610
611/// Finds compaction options and TTL together with a single metadata fetch to reduce RTT.
612async fn find_dynamic_options(
613    table_id: TableId,
614    region_options: &crate::region::options::RegionOptions,
615    schema_metadata_manager: &SchemaMetadataManagerRef,
616) -> Result<(crate::region::options::CompactionOptions, TimeToLive)> {
617    if let (true, Some(ttl)) = (region_options.compaction_override, region_options.ttl) {
618        debug!(
619            "Use region options directly for table {}: compaction={:?}, ttl={:?}",
620            table_id, region_options.compaction, region_options.ttl
621        );
622        return Ok((region_options.compaction.clone(), ttl));
623    }
624
625    let db_options = tokio::time::timeout(
626        crate::config::FETCH_OPTION_TIMEOUT,
627        schema_metadata_manager.get_schema_options_by_table_id(table_id),
628    )
629    .await
630    .context(TimeoutSnafu)?
631    .context(GetSchemaMetadataSnafu)?;
632
633    let ttl = if let Some(ttl) = region_options.ttl {
634        debug!(
635            "Use region TTL directly for table {}: ttl={:?}",
636            table_id, region_options.ttl
637        );
638        ttl
639    } else {
640        db_options
641            .as_ref()
642            .and_then(|options| options.ttl)
643            .unwrap_or_default()
644            .into()
645    };
646
647    let compaction = if !region_options.compaction_override {
648        if let Some(schema_opts) = db_options {
649            let map: HashMap<String, String> = schema_opts
650                .extra_options
651                .iter()
652                .filter_map(|(k, v)| {
653                    if k.starts_with("compaction.") {
654                        Some((k.clone(), v.clone()))
655                    } else {
656                        None
657                    }
658                })
659                .collect();
660            if map.is_empty() {
661                region_options.compaction.clone()
662            } else {
663                crate::region::options::RegionOptions::try_from(&map)
664                    .map(|o| o.compaction)
665                    .unwrap_or_else(|e| {
666                        error!(e; "Failed to create RegionOptions from map");
667                        region_options.compaction.clone()
668                    })
669            }
670        } else {
671            debug!(
672                "DB options is None for table {}, use region compaction: compaction={:?}",
673                table_id, region_options.compaction
674            );
675            region_options.compaction.clone()
676        }
677    } else {
678        debug!(
679            "No schema options for table {}, use region compaction: compaction={:?}",
680            table_id, region_options.compaction
681        );
682        region_options.compaction.clone()
683    };
684
685    debug!(
686        "Resolved dynamic options for table {}: compaction={:?}, ttl={:?}",
687        table_id, compaction, ttl
688    );
689    Ok((compaction, ttl))
690}
691
692/// Status of running and pending region compaction tasks.
693struct CompactionStatus {
694    /// Id of the region.
695    region_id: RegionId,
696    /// Version control of the region.
697    version_control: VersionControlRef,
698    /// Access layer of the region.
699    access_layer: AccessLayerRef,
700    /// Pending waiters for compaction.
701    waiters: Vec<OutputTx>,
702    /// Pending compactions that are supposed to run as soon as current compaction task finished.
703    pending_request: Option<PendingCompaction>,
704    /// Pending DDL requests that should run when compaction is done.
705    pending_ddl_requests: Vec<SenderDdlRequest>,
706}
707
708impl CompactionStatus {
709    /// Creates a new [CompactionStatus]
710    fn new(
711        region_id: RegionId,
712        version_control: VersionControlRef,
713        access_layer: AccessLayerRef,
714    ) -> CompactionStatus {
715        CompactionStatus {
716            region_id,
717            version_control,
718            access_layer,
719            waiters: Vec::new(),
720            pending_request: None,
721            pending_ddl_requests: Vec::new(),
722        }
723    }
724
725    /// Merge the waiter to the pending compaction.
726    fn merge_waiter(&mut self, mut waiter: OptionOutputTx) {
727        if let Some(waiter) = waiter.take_inner() {
728            self.waiters.push(waiter);
729        }
730    }
731
732    /// Set pending compaction request or replace current value if already exist.
733    fn set_pending_request(&mut self, pending: PendingCompaction) {
734        if let Some(prev) = self.pending_request.replace(pending) {
735            debug!(
736                "Replace pending compaction options with new request {:?} for region: {}",
737                prev.options, self.region_id
738            );
739            prev.waiter.send(ManualCompactionOverrideSnafu.fail());
740        }
741    }
742
743    fn on_failure(mut self, err: Arc<Error>) {
744        for waiter in self.waiters.drain(..) {
745            waiter.send(Err(err.clone()).context(CompactRegionSnafu {
746                region_id: self.region_id,
747            }));
748        }
749
750        if let Some(pending_compaction) = self.pending_request {
751            pending_compaction
752                .waiter
753                .send(Err(err.clone()).context(CompactRegionSnafu {
754                    region_id: self.region_id,
755                }));
756        }
757
758        for pending_ddl in self.pending_ddl_requests {
759            pending_ddl
760                .sender
761                .send(Err(err.clone()).context(CompactRegionSnafu {
762                    region_id: self.region_id,
763                }));
764        }
765    }
766
767    /// Creates a new compaction request for compaction picker.
768    ///
769    /// It consumes all pending compaction waiters.
770    #[allow(clippy::too_many_arguments)]
771    fn new_compaction_request(
772        &mut self,
773        request_sender: Sender<WorkerRequestWithTime>,
774        mut waiter: OptionOutputTx,
775        engine_config: Arc<MitoConfig>,
776        cache_manager: CacheManagerRef,
777        manifest_ctx: &ManifestContextRef,
778        listener: WorkerListener,
779        schema_metadata_manager: SchemaMetadataManagerRef,
780        max_parallelism: usize,
781    ) -> CompactionRequest {
782        let current_version = CompactionVersion::from(self.version_control.current().version);
783        let start_time = Instant::now();
784        let mut waiters = Vec::with_capacity(self.waiters.len() + 1);
785        waiters.extend(std::mem::take(&mut self.waiters));
786
787        if let Some(waiter) = waiter.take_inner() {
788            waiters.push(waiter);
789        }
790
791        CompactionRequest {
792            engine_config,
793            current_version,
794            access_layer: self.access_layer.clone(),
795            request_sender: request_sender.clone(),
796            waiters,
797            start_time,
798            cache_manager,
799            manifest_ctx: manifest_ctx.clone(),
800            listener,
801            schema_metadata_manager,
802            max_parallelism,
803        }
804    }
805}
806
807#[derive(Debug, Clone)]
808pub struct CompactionOutput {
809    /// Compaction output file level.
810    pub output_level: Level,
811    /// Compaction input files.
812    pub inputs: Vec<FileHandle>,
813    /// Whether to remove deletion markers.
814    pub filter_deleted: bool,
815    /// Compaction output time range. Only windowed compaction specifies output time range.
816    pub output_time_range: Option<TimestampRange>,
817}
818
819/// SerializedCompactionOutput is a serialized version of [CompactionOutput] by replacing [FileHandle] with [FileMeta].
820#[derive(Debug, Clone, Serialize, Deserialize)]
821pub struct SerializedCompactionOutput {
822    output_level: Level,
823    inputs: Vec<FileMeta>,
824    filter_deleted: bool,
825    output_time_range: Option<TimestampRange>,
826}
827
828/// Builders to create [BoxedRecordBatchStream] for compaction.
829struct CompactionSstReaderBuilder<'a> {
830    metadata: RegionMetadataRef,
831    sst_layer: AccessLayerRef,
832    cache: CacheManagerRef,
833    inputs: &'a [FileHandle],
834    append_mode: bool,
835    filter_deleted: bool,
836    time_range: Option<TimestampRange>,
837    merge_mode: MergeMode,
838}
839
840impl CompactionSstReaderBuilder<'_> {
841    /// Builds [BoxedRecordBatchStream] that reads all SST files and yields batches in flat format for compaction.
842    async fn build_flat_sst_reader(self) -> Result<BoxedRecordBatchStream> {
843        let scan_input = self.build_scan_input()?.with_compaction(true);
844
845        SeqScan::new(scan_input)
846            .build_flat_reader_for_compaction()
847            .await
848    }
849
850    fn build_scan_input(self) -> Result<ScanInput> {
851        let mapper = ProjectionMapper::all(&self.metadata, true)?;
852        let mut scan_input = ScanInput::new(self.sst_layer, mapper)
853            .with_files(self.inputs.to_vec())
854            .with_append_mode(self.append_mode)
855            // We use special cache strategy for compaction.
856            .with_cache(CacheStrategy::Compaction(self.cache))
857            .with_filter_deleted(self.filter_deleted)
858            // We ignore file not found error during compaction.
859            .with_ignore_file_not_found(true)
860            .with_merge_mode(self.merge_mode)
861            .with_flat_format(true);
862
863        // This serves as a workaround of https://github.com/GreptimeTeam/greptimedb/issues/3944
864        // by converting time ranges into predicate.
865        if let Some(time_range) = self.time_range {
866            scan_input =
867                scan_input.with_predicate(time_range_to_predicate(time_range, &self.metadata)?);
868        }
869
870        Ok(scan_input)
871    }
872}
873
874/// Converts time range to predicates so that rows outside the range will be filtered.
875fn time_range_to_predicate(
876    range: TimestampRange,
877    metadata: &RegionMetadataRef,
878) -> Result<PredicateGroup> {
879    let ts_col = metadata.time_index_column();
880
881    // safety: time index column's type must be a valid timestamp type.
882    let ts_col_unit = ts_col
883        .column_schema
884        .data_type
885        .as_timestamp()
886        .unwrap()
887        .unit();
888
889    let exprs = match (range.start(), range.end()) {
890        (Some(start), Some(end)) => {
891            vec![
892                datafusion_expr::col(ts_col.column_schema.name.clone())
893                    .gt_eq(ts_to_lit(*start, ts_col_unit)?),
894                datafusion_expr::col(ts_col.column_schema.name.clone())
895                    .lt(ts_to_lit(*end, ts_col_unit)?),
896            ]
897        }
898        (Some(start), None) => {
899            vec![
900                datafusion_expr::col(ts_col.column_schema.name.clone())
901                    .gt_eq(ts_to_lit(*start, ts_col_unit)?),
902            ]
903        }
904
905        (None, Some(end)) => {
906            vec![
907                datafusion_expr::col(ts_col.column_schema.name.clone())
908                    .lt(ts_to_lit(*end, ts_col_unit)?),
909            ]
910        }
911        (None, None) => {
912            return Ok(PredicateGroup::default());
913        }
914    };
915
916    let predicate = PredicateGroup::new(metadata, &exprs)?;
917    Ok(predicate)
918}
919
920fn ts_to_lit(ts: Timestamp, ts_col_unit: TimeUnit) -> Result<Expr> {
921    let ts = ts
922        .convert_to(ts_col_unit)
923        .context(TimeRangePredicateOverflowSnafu {
924            timestamp: ts,
925            unit: ts_col_unit,
926        })?;
927    let val = ts.value();
928    let scalar_value = match ts_col_unit {
929        TimeUnit::Second => ScalarValue::TimestampSecond(Some(val), None),
930        TimeUnit::Millisecond => ScalarValue::TimestampMillisecond(Some(val), None),
931        TimeUnit::Microsecond => ScalarValue::TimestampMicrosecond(Some(val), None),
932        TimeUnit::Nanosecond => ScalarValue::TimestampNanosecond(Some(val), None),
933    };
934    Ok(datafusion_expr::lit(scalar_value))
935}
936
937/// Finds all expired SSTs across levels.
938fn get_expired_ssts(
939    levels: &[LevelMeta],
940    ttl: Option<TimeToLive>,
941    now: Timestamp,
942) -> Vec<FileHandle> {
943    let Some(ttl) = ttl else {
944        return vec![];
945    };
946
947    levels
948        .iter()
949        .flat_map(|l| l.get_expired_files(&now, &ttl).into_iter())
950        .collect()
951}
952
953/// Estimates compaction memory as the sum of all input files' maximum row-group
954/// uncompressed sizes.
955fn estimate_compaction_bytes(picker_output: &PickerOutput) -> u64 {
956    picker_output
957        .outputs
958        .iter()
959        .flat_map(|output| output.inputs.iter())
960        .map(|file: &FileHandle| {
961            let meta = file.meta_ref();
962            meta.max_row_group_uncompressed_size
963        })
964        .sum()
965}
966
967/// Pending compaction request that is supposed to run after current task is finished,
968/// typically used for manual compactions.
969struct PendingCompaction {
970    /// Compaction options. Currently, it can only be [StrictWindow].
971    pub(crate) options: compact_request::Options,
972    /// Waiters of pending requests.
973    pub(crate) waiter: OptionOutputTx,
974    /// Max parallelism for pending compaction.
975    pub(crate) max_parallelism: usize,
976}
977
978#[cfg(test)]
979mod tests {
980    use std::assert_matches;
981    use std::time::Duration;
982
983    use api::v1::region::StrictWindow;
984    use common_datasource::compression::CompressionType;
985    use common_meta::key::schema_name::SchemaNameValue;
986    use common_time::DatabaseTimeToLive;
987    use tokio::sync::{Barrier, oneshot};
988
989    use super::*;
990    use crate::compaction::memory_manager::{CompactionMemoryGuard, new_compaction_memory_manager};
991    use crate::error::InvalidSchedulerStateSnafu;
992    use crate::manifest::manager::{RegionManifestManager, RegionManifestOptions};
993    use crate::region::ManifestContext;
994    use crate::schedule::scheduler::{Job, Scheduler};
995    use crate::sst::FormatType;
996    use crate::test_util::mock_schema_metadata_manager;
997    use crate::test_util::scheduler_util::{SchedulerEnv, VecScheduler};
998    use crate::test_util::version_util::{VersionControlBuilder, apply_edit};
999
1000    struct FailingScheduler;
1001
1002    #[async_trait::async_trait]
1003    impl Scheduler for FailingScheduler {
1004        fn schedule(&self, _job: Job) -> Result<()> {
1005            InvalidSchedulerStateSnafu.fail()
1006        }
1007
1008        async fn stop(&self, _await_termination: bool) -> Result<()> {
1009            Ok(())
1010        }
1011    }
1012
1013    #[tokio::test]
1014    async fn test_find_compaction_options_db_level() {
1015        let env = SchedulerEnv::new().await;
1016        let builder = VersionControlBuilder::new();
1017        let (schema_metadata_manager, kv_backend) = mock_schema_metadata_manager();
1018        let region_id = builder.region_id();
1019        let table_id = region_id.table_id();
1020        // Register table without ttl but with db-level compaction options
1021        let mut schema_value = SchemaNameValue {
1022            ttl: Some(DatabaseTimeToLive::default()),
1023            ..Default::default()
1024        };
1025        schema_value
1026            .extra_options
1027            .insert("compaction.type".to_string(), "twcs".to_string());
1028        schema_value
1029            .extra_options
1030            .insert("compaction.twcs.time_window".to_string(), "2h".to_string());
1031        schema_metadata_manager
1032            .register_region_table_info(
1033                table_id,
1034                "t",
1035                "c",
1036                "s",
1037                Some(schema_value),
1038                kv_backend.clone(),
1039            )
1040            .await;
1041
1042        let version_control = Arc::new(builder.build());
1043        let region_opts = version_control.current().version.options.clone();
1044        let (opts, _) = find_dynamic_options(table_id, &region_opts, &schema_metadata_manager)
1045            .await
1046            .unwrap();
1047        match opts {
1048            crate::region::options::CompactionOptions::Twcs(t) => {
1049                assert_eq!(t.time_window_seconds(), Some(2 * 3600));
1050            }
1051        }
1052        let manifest_ctx = env
1053            .mock_manifest_context(version_control.current().version.metadata.clone())
1054            .await;
1055        let (tx, _rx) = mpsc::channel(4);
1056        let mut scheduler = env.mock_compaction_scheduler(tx);
1057        let (otx, _orx) = oneshot::channel();
1058        let request = scheduler
1059            .region_status
1060            .entry(region_id)
1061            .or_insert_with(|| {
1062                crate::compaction::CompactionStatus::new(
1063                    region_id,
1064                    version_control.clone(),
1065                    env.access_layer.clone(),
1066                )
1067            })
1068            .new_compaction_request(
1069                scheduler.request_sender.clone(),
1070                OptionOutputTx::new(Some(OutputTx::new(otx))),
1071                scheduler.engine_config.clone(),
1072                scheduler.cache_manager.clone(),
1073                &manifest_ctx,
1074                scheduler.listener.clone(),
1075                schema_metadata_manager.clone(),
1076                1,
1077            );
1078        scheduler
1079            .schedule_compaction_request(
1080                request,
1081                compact_request::Options::Regular(Default::default()),
1082            )
1083            .await
1084            .unwrap();
1085    }
1086
1087    #[tokio::test]
1088    async fn test_find_compaction_options_priority() {
1089        fn schema_value_with_twcs(time_window: &str) -> SchemaNameValue {
1090            let mut schema_value = SchemaNameValue {
1091                ttl: Some(DatabaseTimeToLive::default()),
1092                ..Default::default()
1093            };
1094            schema_value
1095                .extra_options
1096                .insert("compaction.type".to_string(), "twcs".to_string());
1097            schema_value.extra_options.insert(
1098                "compaction.twcs.time_window".to_string(),
1099                time_window.to_string(),
1100            );
1101            schema_value
1102        }
1103
1104        let cases = [
1105            (
1106                "db options set and table override set",
1107                Some(schema_value_with_twcs("2h")),
1108                true,
1109                Some(Duration::from_secs(5 * 3600)),
1110                Some(5 * 3600),
1111            ),
1112            (
1113                "db options set and table override not set",
1114                Some(schema_value_with_twcs("2h")),
1115                false,
1116                None,
1117                Some(2 * 3600),
1118            ),
1119            (
1120                "db options not set and table override set",
1121                None,
1122                true,
1123                Some(Duration::from_secs(4 * 3600)),
1124                Some(4 * 3600),
1125            ),
1126            (
1127                "db options not set and table override not set",
1128                None,
1129                false,
1130                None,
1131                None,
1132            ),
1133        ];
1134
1135        for (case_name, schema_value, override_set, table_window, expected_window) in cases {
1136            let builder = VersionControlBuilder::new();
1137            let (schema_metadata_manager, kv_backend) = mock_schema_metadata_manager();
1138            let table_id = builder.region_id().table_id();
1139            schema_metadata_manager
1140                .register_region_table_info(
1141                    table_id,
1142                    "t",
1143                    "c",
1144                    "s",
1145                    schema_value,
1146                    kv_backend.clone(),
1147                )
1148                .await;
1149
1150            let version_control = Arc::new(builder.build());
1151            let mut region_opts = version_control.current().version.options.clone();
1152            region_opts.compaction_override = override_set;
1153            if let Some(window) = table_window {
1154                let crate::region::options::CompactionOptions::Twcs(twcs) =
1155                    &mut region_opts.compaction;
1156                twcs.time_window = Some(window);
1157            }
1158
1159            let (opts, _) = find_dynamic_options(table_id, &region_opts, &schema_metadata_manager)
1160                .await
1161                .unwrap();
1162            match opts {
1163                crate::region::options::CompactionOptions::Twcs(t) => {
1164                    assert_eq!(t.time_window_seconds(), expected_window, "{case_name}");
1165                }
1166            }
1167        }
1168    }
1169
1170    #[tokio::test]
1171    async fn test_schedule_empty() {
1172        let env = SchedulerEnv::new().await;
1173        let (tx, _rx) = mpsc::channel(4);
1174        let mut scheduler = env.mock_compaction_scheduler(tx);
1175        let mut builder = VersionControlBuilder::new();
1176        let (schema_metadata_manager, kv_backend) = mock_schema_metadata_manager();
1177        schema_metadata_manager
1178            .register_region_table_info(
1179                builder.region_id().table_id(),
1180                "test_table",
1181                "test_catalog",
1182                "test_schema",
1183                None,
1184                kv_backend,
1185            )
1186            .await;
1187        // Nothing to compact.
1188        let version_control = Arc::new(builder.build());
1189        let (output_tx, output_rx) = oneshot::channel();
1190        let waiter = OptionOutputTx::from(output_tx);
1191        let manifest_ctx = env
1192            .mock_manifest_context(version_control.current().version.metadata.clone())
1193            .await;
1194        scheduler
1195            .schedule_compaction(
1196                builder.region_id(),
1197                compact_request::Options::Regular(Default::default()),
1198                &version_control,
1199                &env.access_layer,
1200                waiter,
1201                &manifest_ctx,
1202                schema_metadata_manager.clone(),
1203                1,
1204            )
1205            .await
1206            .unwrap();
1207        let output = output_rx.await.unwrap().unwrap();
1208        assert_eq!(output, 0);
1209        assert!(scheduler.region_status.is_empty());
1210
1211        // Only one file, picker won't compact it.
1212        let version_control = Arc::new(builder.push_l0_file(0, 1000).build());
1213        let (output_tx, output_rx) = oneshot::channel();
1214        let waiter = OptionOutputTx::from(output_tx);
1215        scheduler
1216            .schedule_compaction(
1217                builder.region_id(),
1218                compact_request::Options::Regular(Default::default()),
1219                &version_control,
1220                &env.access_layer,
1221                waiter,
1222                &manifest_ctx,
1223                schema_metadata_manager,
1224                1,
1225            )
1226            .await
1227            .unwrap();
1228        let output = output_rx.await.unwrap().unwrap();
1229        assert_eq!(output, 0);
1230        assert!(scheduler.region_status.is_empty());
1231    }
1232
1233    #[tokio::test]
1234    async fn test_schedule_on_finished() {
1235        common_telemetry::init_default_ut_logging();
1236        let job_scheduler = Arc::new(VecScheduler::default());
1237        let env = SchedulerEnv::new().await.scheduler(job_scheduler.clone());
1238        let (tx, _rx) = mpsc::channel(4);
1239        let mut scheduler = env.mock_compaction_scheduler(tx);
1240        let mut builder = VersionControlBuilder::new();
1241        let purger = builder.file_purger();
1242        let region_id = builder.region_id();
1243
1244        let (schema_metadata_manager, kv_backend) = mock_schema_metadata_manager();
1245        schema_metadata_manager
1246            .register_region_table_info(
1247                builder.region_id().table_id(),
1248                "test_table",
1249                "test_catalog",
1250                "test_schema",
1251                None,
1252                kv_backend,
1253            )
1254            .await;
1255
1256        // 5 files to compact.
1257        let end = 1000 * 1000;
1258        let version_control = Arc::new(
1259            builder
1260                .push_l0_file(0, end)
1261                .push_l0_file(10, end)
1262                .push_l0_file(50, end)
1263                .push_l0_file(80, end)
1264                .push_l0_file(90, end)
1265                .build(),
1266        );
1267        let manifest_ctx = env
1268            .mock_manifest_context(version_control.current().version.metadata.clone())
1269            .await;
1270        scheduler
1271            .schedule_compaction(
1272                region_id,
1273                compact_request::Options::Regular(Default::default()),
1274                &version_control,
1275                &env.access_layer,
1276                OptionOutputTx::none(),
1277                &manifest_ctx,
1278                schema_metadata_manager.clone(),
1279                1,
1280            )
1281            .await
1282            .unwrap();
1283        // Should schedule 1 compaction.
1284        assert_eq!(1, scheduler.region_status.len());
1285        assert_eq!(1, job_scheduler.num_jobs());
1286        let data = version_control.current();
1287        let file_metas: Vec<_> = data.version.ssts.levels()[0]
1288            .files
1289            .values()
1290            .map(|file| file.meta_ref().clone())
1291            .collect();
1292
1293        // 5 files for next compaction and removes old files.
1294        apply_edit(
1295            &version_control,
1296            &[(0, end), (20, end), (40, end), (60, end), (80, end)],
1297            &file_metas,
1298            purger.clone(),
1299        );
1300        // The task is pending.
1301        let (tx, _rx) = oneshot::channel();
1302        scheduler
1303            .schedule_compaction(
1304                region_id,
1305                compact_request::Options::Regular(Default::default()),
1306                &version_control,
1307                &env.access_layer,
1308                OptionOutputTx::new(Some(OutputTx::new(tx))),
1309                &manifest_ctx,
1310                schema_metadata_manager.clone(),
1311                1,
1312            )
1313            .await
1314            .unwrap();
1315        assert_eq!(1, scheduler.region_status.len());
1316        assert_eq!(1, job_scheduler.num_jobs());
1317        assert!(
1318            !scheduler
1319                .region_status
1320                .get(&builder.region_id())
1321                .unwrap()
1322                .waiters
1323                .is_empty()
1324        );
1325
1326        // On compaction finished and schedule next compaction.
1327        scheduler
1328            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager.clone())
1329            .await;
1330        assert_eq!(1, scheduler.region_status.len());
1331        assert_eq!(2, job_scheduler.num_jobs());
1332
1333        // 5 files for next compaction.
1334        apply_edit(
1335            &version_control,
1336            &[(0, end), (20, end), (40, end), (60, end), (80, end)],
1337            &[],
1338            purger.clone(),
1339        );
1340        let (tx, _rx) = oneshot::channel();
1341        // The task is pending.
1342        scheduler
1343            .schedule_compaction(
1344                region_id,
1345                compact_request::Options::Regular(Default::default()),
1346                &version_control,
1347                &env.access_layer,
1348                OptionOutputTx::new(Some(OutputTx::new(tx))),
1349                &manifest_ctx,
1350                schema_metadata_manager,
1351                1,
1352            )
1353            .await
1354            .unwrap();
1355        assert_eq!(2, job_scheduler.num_jobs());
1356        assert!(
1357            !scheduler
1358                .region_status
1359                .get(&builder.region_id())
1360                .unwrap()
1361                .waiters
1362                .is_empty()
1363        );
1364    }
1365
1366    #[tokio::test]
1367    async fn test_manual_compaction_when_compaction_in_progress() {
1368        common_telemetry::init_default_ut_logging();
1369        let job_scheduler = Arc::new(VecScheduler::default());
1370        let env = SchedulerEnv::new().await.scheduler(job_scheduler.clone());
1371        let (tx, _rx) = mpsc::channel(4);
1372        let mut scheduler = env.mock_compaction_scheduler(tx);
1373        let mut builder = VersionControlBuilder::new();
1374        let purger = builder.file_purger();
1375        let region_id = builder.region_id();
1376
1377        let (schema_metadata_manager, kv_backend) = mock_schema_metadata_manager();
1378        schema_metadata_manager
1379            .register_region_table_info(
1380                builder.region_id().table_id(),
1381                "test_table",
1382                "test_catalog",
1383                "test_schema",
1384                None,
1385                kv_backend,
1386            )
1387            .await;
1388
1389        // 5 files to compact.
1390        let end = 1000 * 1000;
1391        let version_control = Arc::new(
1392            builder
1393                .push_l0_file(0, end)
1394                .push_l0_file(10, end)
1395                .push_l0_file(50, end)
1396                .push_l0_file(80, end)
1397                .push_l0_file(90, end)
1398                .build(),
1399        );
1400        let manifest_ctx = env
1401            .mock_manifest_context(version_control.current().version.metadata.clone())
1402            .await;
1403
1404        let file_metas: Vec<_> = version_control.current().version.ssts.levels()[0]
1405            .files
1406            .values()
1407            .map(|file| file.meta_ref().clone())
1408            .collect();
1409
1410        // 5 files for next compaction and removes old files.
1411        apply_edit(
1412            &version_control,
1413            &[(0, end), (20, end), (40, end), (60, end), (80, end)],
1414            &file_metas,
1415            purger.clone(),
1416        );
1417
1418        scheduler
1419            .schedule_compaction(
1420                region_id,
1421                compact_request::Options::Regular(Default::default()),
1422                &version_control,
1423                &env.access_layer,
1424                OptionOutputTx::none(),
1425                &manifest_ctx,
1426                schema_metadata_manager.clone(),
1427                1,
1428            )
1429            .await
1430            .unwrap();
1431        // Should schedule 1 compaction.
1432        assert_eq!(1, scheduler.region_status.len());
1433        assert_eq!(1, job_scheduler.num_jobs());
1434        assert!(
1435            scheduler
1436                .region_status
1437                .get(&region_id)
1438                .unwrap()
1439                .pending_request
1440                .is_none()
1441        );
1442
1443        // Schedule another manual compaction.
1444        let (tx, _rx) = oneshot::channel();
1445        scheduler
1446            .schedule_compaction(
1447                region_id,
1448                compact_request::Options::StrictWindow(StrictWindow { window_seconds: 60 }),
1449                &version_control,
1450                &env.access_layer,
1451                OptionOutputTx::new(Some(OutputTx::new(tx))),
1452                &manifest_ctx,
1453                schema_metadata_manager.clone(),
1454                1,
1455            )
1456            .await
1457            .unwrap();
1458        assert_eq!(1, scheduler.region_status.len());
1459        // Current job num should be 1 since compaction is in progress.
1460        assert_eq!(1, job_scheduler.num_jobs());
1461        let status = scheduler.region_status.get(&builder.region_id()).unwrap();
1462        assert!(status.pending_request.is_some());
1463
1464        // On compaction finished and schedule next compaction.
1465        scheduler
1466            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager.clone())
1467            .await;
1468        assert_eq!(1, scheduler.region_status.len());
1469        assert_eq!(2, job_scheduler.num_jobs());
1470
1471        let status = scheduler.region_status.get(&builder.region_id()).unwrap();
1472        assert!(status.pending_request.is_none());
1473    }
1474
1475    #[tokio::test]
1476    async fn test_compaction_bypass_in_staging_mode() {
1477        let env = SchedulerEnv::new().await;
1478        let (tx, _rx) = mpsc::channel(4);
1479        let mut scheduler = env.mock_compaction_scheduler(tx);
1480
1481        // Create version control and manifest context for staging mode
1482        let builder = VersionControlBuilder::new();
1483        let version_control = Arc::new(builder.build());
1484        let region_id = version_control.current().version.metadata.region_id;
1485
1486        // Create staging manifest context using the same pattern as SchedulerEnv
1487        let staging_manifest_ctx = {
1488            let manager = RegionManifestManager::new(
1489                version_control.current().version.metadata.clone(),
1490                0,
1491                RegionManifestOptions {
1492                    manifest_dir: "".to_string(),
1493                    object_store: env.access_layer.object_store().clone(),
1494                    compress_type: CompressionType::Uncompressed,
1495                    checkpoint_distance: 10,
1496                    remove_file_options: Default::default(),
1497                    manifest_cache: None,
1498                },
1499                FormatType::PrimaryKey,
1500                &Default::default(),
1501            )
1502            .await
1503            .unwrap();
1504            Arc::new(ManifestContext::new(
1505                manager,
1506                RegionRoleState::Leader(RegionLeaderState::Staging),
1507            ))
1508        };
1509
1510        let (schema_metadata_manager, _kv_backend) = mock_schema_metadata_manager();
1511
1512        // Test regular compaction bypass in staging mode
1513        let (tx, rx) = oneshot::channel();
1514        scheduler
1515            .schedule_compaction(
1516                region_id,
1517                compact_request::Options::Regular(Default::default()),
1518                &version_control,
1519                &env.access_layer,
1520                OptionOutputTx::new(Some(OutputTx::new(tx))),
1521                &staging_manifest_ctx,
1522                schema_metadata_manager,
1523                1,
1524            )
1525            .await
1526            .unwrap();
1527
1528        let result = rx.await.unwrap();
1529        assert_eq!(result.unwrap(), 0); // is there a better way to check this?
1530        assert_eq!(0, scheduler.region_status.len());
1531    }
1532
1533    #[tokio::test]
1534    async fn test_add_ddl_request_to_pending() {
1535        let env = SchedulerEnv::new().await;
1536        let (tx, _rx) = mpsc::channel(4);
1537        let mut scheduler = env.mock_compaction_scheduler(tx);
1538        let builder = VersionControlBuilder::new();
1539        let version_control = Arc::new(builder.build());
1540        let region_id = builder.region_id();
1541
1542        scheduler.region_status.insert(
1543            region_id,
1544            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1545        );
1546
1547        let (output_tx, _output_rx) = oneshot::channel();
1548        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1549            region_id,
1550            sender: OptionOutputTx::from(output_tx),
1551            request: crate::request::DdlRequest::EnterStaging(
1552                store_api::region_request::EnterStagingRequest {
1553                    partition_directive:
1554                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1555                },
1556            ),
1557        });
1558
1559        assert!(scheduler.has_pending_ddls(region_id));
1560    }
1561
1562    #[tokio::test]
1563    async fn test_pending_ddl_request_failed_on_compaction_failed() {
1564        let env = SchedulerEnv::new().await;
1565        let (tx, _rx) = mpsc::channel(4);
1566        let mut scheduler = env.mock_compaction_scheduler(tx);
1567        let builder = VersionControlBuilder::new();
1568        let version_control = Arc::new(builder.build());
1569        let region_id = builder.region_id();
1570
1571        scheduler.region_status.insert(
1572            region_id,
1573            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1574        );
1575
1576        let (output_tx, output_rx) = oneshot::channel();
1577        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1578            region_id,
1579            sender: OptionOutputTx::from(output_tx),
1580            request: crate::request::DdlRequest::EnterStaging(
1581                store_api::region_request::EnterStagingRequest {
1582                    partition_directive:
1583                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1584                },
1585            ),
1586        });
1587
1588        assert!(scheduler.has_pending_ddls(region_id));
1589        scheduler
1590            .on_compaction_failed(region_id, Arc::new(RegionClosedSnafu { region_id }.build()));
1591
1592        assert!(!scheduler.has_pending_ddls(region_id));
1593        let result = output_rx.await.unwrap();
1594        assert_matches!(result, Err(_));
1595    }
1596
1597    #[tokio::test]
1598    async fn test_pending_ddl_request_failed_on_region_closed() {
1599        let env = SchedulerEnv::new().await;
1600        let (tx, _rx) = mpsc::channel(4);
1601        let mut scheduler = env.mock_compaction_scheduler(tx);
1602        let builder = VersionControlBuilder::new();
1603        let version_control = Arc::new(builder.build());
1604        let region_id = builder.region_id();
1605
1606        scheduler.region_status.insert(
1607            region_id,
1608            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1609        );
1610
1611        let (output_tx, output_rx) = oneshot::channel();
1612        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1613            region_id,
1614            sender: OptionOutputTx::from(output_tx),
1615            request: crate::request::DdlRequest::EnterStaging(
1616                store_api::region_request::EnterStagingRequest {
1617                    partition_directive:
1618                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1619                },
1620            ),
1621        });
1622
1623        assert!(scheduler.has_pending_ddls(region_id));
1624        scheduler.on_region_closed(region_id);
1625
1626        assert!(!scheduler.has_pending_ddls(region_id));
1627        let result = output_rx.await.unwrap();
1628        assert_matches!(result, Err(_));
1629    }
1630
1631    #[tokio::test]
1632    async fn test_pending_ddl_request_failed_on_region_dropped() {
1633        let env = SchedulerEnv::new().await;
1634        let (tx, _rx) = mpsc::channel(4);
1635        let mut scheduler = env.mock_compaction_scheduler(tx);
1636        let builder = VersionControlBuilder::new();
1637        let version_control = Arc::new(builder.build());
1638        let region_id = builder.region_id();
1639
1640        scheduler.region_status.insert(
1641            region_id,
1642            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1643        );
1644
1645        let (output_tx, output_rx) = oneshot::channel();
1646        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1647            region_id,
1648            sender: OptionOutputTx::from(output_tx),
1649            request: crate::request::DdlRequest::EnterStaging(
1650                store_api::region_request::EnterStagingRequest {
1651                    partition_directive:
1652                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1653                },
1654            ),
1655        });
1656
1657        assert!(scheduler.has_pending_ddls(region_id));
1658        scheduler.on_region_dropped(region_id);
1659
1660        assert!(!scheduler.has_pending_ddls(region_id));
1661        let result = output_rx.await.unwrap();
1662        assert_matches!(result, Err(_));
1663    }
1664
1665    #[tokio::test]
1666    async fn test_pending_ddl_request_failed_on_region_truncated() {
1667        let env = SchedulerEnv::new().await;
1668        let (tx, _rx) = mpsc::channel(4);
1669        let mut scheduler = env.mock_compaction_scheduler(tx);
1670        let builder = VersionControlBuilder::new();
1671        let version_control = Arc::new(builder.build());
1672        let region_id = builder.region_id();
1673
1674        scheduler.region_status.insert(
1675            region_id,
1676            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1677        );
1678
1679        let (output_tx, output_rx) = oneshot::channel();
1680        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1681            region_id,
1682            sender: OptionOutputTx::from(output_tx),
1683            request: crate::request::DdlRequest::EnterStaging(
1684                store_api::region_request::EnterStagingRequest {
1685                    partition_directive:
1686                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1687                },
1688            ),
1689        });
1690
1691        assert!(scheduler.has_pending_ddls(region_id));
1692        scheduler.on_region_truncated(region_id);
1693
1694        assert!(!scheduler.has_pending_ddls(region_id));
1695        let result = output_rx.await.unwrap();
1696        assert_matches!(result, Err(_));
1697    }
1698
1699    #[tokio::test]
1700    async fn test_on_compaction_finished_returns_pending_ddl_requests() {
1701        let job_scheduler = Arc::new(VecScheduler::default());
1702        let env = SchedulerEnv::new().await.scheduler(job_scheduler.clone());
1703        let (tx, _rx) = mpsc::channel(4);
1704        let mut scheduler = env.mock_compaction_scheduler(tx);
1705        let builder = VersionControlBuilder::new();
1706        let version_control = Arc::new(builder.build());
1707        let region_id = builder.region_id();
1708        let manifest_ctx = env
1709            .mock_manifest_context(version_control.current().version.metadata.clone())
1710            .await;
1711        let (schema_metadata_manager, _kv_backend) = mock_schema_metadata_manager();
1712
1713        scheduler.region_status.insert(
1714            region_id,
1715            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1716        );
1717
1718        let (output_tx, _output_rx) = oneshot::channel();
1719        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1720            region_id,
1721            sender: OptionOutputTx::from(output_tx),
1722            request: crate::request::DdlRequest::EnterStaging(
1723                store_api::region_request::EnterStagingRequest {
1724                    partition_directive:
1725                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1726                },
1727            ),
1728        });
1729
1730        let pending_ddls = scheduler
1731            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager)
1732            .await;
1733
1734        assert_eq!(pending_ddls.len(), 1);
1735        assert!(!scheduler.has_pending_ddls(region_id));
1736        assert!(!scheduler.region_status.contains_key(&region_id));
1737        assert_eq!(job_scheduler.num_jobs(), 0);
1738    }
1739
1740    #[tokio::test]
1741    async fn test_on_compaction_finished_replays_pending_ddl_after_manual_noop() {
1742        let env = SchedulerEnv::new().await;
1743        let (tx, _rx) = mpsc::channel(4);
1744        let mut scheduler = env.mock_compaction_scheduler(tx);
1745        let builder = VersionControlBuilder::new();
1746        let version_control = Arc::new(builder.build());
1747        let region_id = builder.region_id();
1748        let manifest_ctx = env
1749            .mock_manifest_context(version_control.current().version.metadata.clone())
1750            .await;
1751        let (schema_metadata_manager, _kv_backend) = mock_schema_metadata_manager();
1752
1753        let (manual_tx, manual_rx) = oneshot::channel();
1754        let mut status =
1755            CompactionStatus::new(region_id, version_control.clone(), env.access_layer.clone());
1756        status.set_pending_request(PendingCompaction {
1757            options: compact_request::Options::Regular(Default::default()),
1758            waiter: OptionOutputTx::from(manual_tx),
1759            max_parallelism: 1,
1760        });
1761        scheduler.region_status.insert(region_id, status);
1762
1763        let (ddl_tx, _ddl_rx) = oneshot::channel();
1764        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1765            region_id,
1766            sender: OptionOutputTx::from(ddl_tx),
1767            request: crate::request::DdlRequest::EnterStaging(
1768                store_api::region_request::EnterStagingRequest {
1769                    partition_directive:
1770                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1771                },
1772            ),
1773        });
1774
1775        let pending_ddls = scheduler
1776            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager)
1777            .await;
1778
1779        assert_eq!(pending_ddls.len(), 1);
1780        assert!(!scheduler.region_status.contains_key(&region_id));
1781        assert_eq!(manual_rx.await.unwrap().unwrap(), 0);
1782    }
1783
1784    #[tokio::test]
1785    async fn test_on_compaction_finished_returns_empty_when_region_absent() {
1786        let env = SchedulerEnv::new().await;
1787        let (tx, _rx) = mpsc::channel(4);
1788        let mut scheduler = env.mock_compaction_scheduler(tx);
1789        let builder = VersionControlBuilder::new();
1790        let region_id = builder.region_id();
1791        let version_control = Arc::new(builder.build());
1792        let manifest_ctx = env
1793            .mock_manifest_context(version_control.current().version.metadata.clone())
1794            .await;
1795        let (schema_metadata_manager, _kv_backend) = mock_schema_metadata_manager();
1796
1797        let pending_ddls = scheduler
1798            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager)
1799            .await;
1800
1801        assert!(pending_ddls.is_empty());
1802    }
1803
1804    #[tokio::test]
1805    async fn test_on_compaction_finished_manual_schedule_error_cleans_status() {
1806        let env = SchedulerEnv::new()
1807            .await
1808            .scheduler(Arc::new(FailingScheduler));
1809        let (tx, _rx) = mpsc::channel(4);
1810        let mut scheduler = env.mock_compaction_scheduler(tx);
1811        let mut builder = VersionControlBuilder::new();
1812        let end = 1000 * 1000;
1813        let version_control = Arc::new(
1814            builder
1815                .push_l0_file(0, end)
1816                .push_l0_file(10, end)
1817                .push_l0_file(50, end)
1818                .push_l0_file(80, end)
1819                .push_l0_file(90, end)
1820                .build(),
1821        );
1822        let region_id = builder.region_id();
1823        let manifest_ctx = env
1824            .mock_manifest_context(version_control.current().version.metadata.clone())
1825            .await;
1826        let (schema_metadata_manager, _kv_backend) = mock_schema_metadata_manager();
1827
1828        let (manual_tx, manual_rx) = oneshot::channel();
1829        let mut status =
1830            CompactionStatus::new(region_id, version_control.clone(), env.access_layer.clone());
1831        status.set_pending_request(PendingCompaction {
1832            options: compact_request::Options::Regular(Default::default()),
1833            waiter: OptionOutputTx::from(manual_tx),
1834            max_parallelism: 1,
1835        });
1836        scheduler.region_status.insert(region_id, status);
1837
1838        let (ddl_tx, ddl_rx) = oneshot::channel();
1839        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1840            region_id,
1841            sender: OptionOutputTx::from(ddl_tx),
1842            request: crate::request::DdlRequest::EnterStaging(
1843                store_api::region_request::EnterStagingRequest {
1844                    partition_directive:
1845                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1846                },
1847            ),
1848        });
1849
1850        let pending_ddls = scheduler
1851            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager)
1852            .await;
1853
1854        assert!(pending_ddls.is_empty());
1855        assert!(!scheduler.region_status.contains_key(&region_id));
1856        assert!(manual_rx.await.is_err());
1857        assert_matches!(ddl_rx.await.unwrap(), Err(_));
1858    }
1859
1860    #[tokio::test]
1861    async fn test_on_compaction_finished_next_schedule_noop_removes_status() {
1862        let env = SchedulerEnv::new().await;
1863        let (tx, _rx) = mpsc::channel(4);
1864        let mut scheduler = env.mock_compaction_scheduler(tx);
1865        let builder = VersionControlBuilder::new();
1866        let version_control = Arc::new(builder.build());
1867        let region_id = builder.region_id();
1868        let manifest_ctx = env
1869            .mock_manifest_context(version_control.current().version.metadata.clone())
1870            .await;
1871        let (schema_metadata_manager, _kv_backend) = mock_schema_metadata_manager();
1872
1873        scheduler.region_status.insert(
1874            region_id,
1875            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1876        );
1877
1878        let pending_ddls = scheduler
1879            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager)
1880            .await;
1881
1882        assert!(pending_ddls.is_empty());
1883        assert!(!scheduler.region_status.contains_key(&region_id));
1884    }
1885
1886    #[tokio::test]
1887    async fn test_on_compaction_finished_next_schedule_error_cleans_status() {
1888        let env = SchedulerEnv::new()
1889            .await
1890            .scheduler(Arc::new(FailingScheduler));
1891        let (tx, _rx) = mpsc::channel(4);
1892        let mut scheduler = env.mock_compaction_scheduler(tx);
1893        let mut builder = VersionControlBuilder::new();
1894        let end = 1000 * 1000;
1895        let version_control = Arc::new(
1896            builder
1897                .push_l0_file(0, end)
1898                .push_l0_file(10, end)
1899                .push_l0_file(50, end)
1900                .push_l0_file(80, end)
1901                .push_l0_file(90, end)
1902                .build(),
1903        );
1904        let region_id = builder.region_id();
1905        let manifest_ctx = env
1906            .mock_manifest_context(version_control.current().version.metadata.clone())
1907            .await;
1908        let (schema_metadata_manager, _kv_backend) = mock_schema_metadata_manager();
1909
1910        scheduler.region_status.insert(
1911            region_id,
1912            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1913        );
1914
1915        let pending_ddls = scheduler
1916            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager)
1917            .await;
1918
1919        assert!(pending_ddls.is_empty());
1920        assert!(!scheduler.region_status.contains_key(&region_id));
1921    }
1922
1923    #[tokio::test]
1924    async fn test_concurrent_memory_competition() {
1925        let manager = Arc::new(new_compaction_memory_manager(3 * 1024 * 1024)); // 3MB
1926        let barrier = Arc::new(Barrier::new(3));
1927        let mut handles = vec![];
1928
1929        // Spawn 3 tasks competing for memory, each trying to acquire 2MB
1930        for _i in 0..3 {
1931            let mgr = manager.clone();
1932            let bar = barrier.clone();
1933            let handle = tokio::spawn(async move {
1934                bar.wait().await; // Synchronize start
1935                mgr.try_acquire(2 * 1024 * 1024)
1936            });
1937            handles.push(handle);
1938        }
1939
1940        let results: Vec<Option<CompactionMemoryGuard>> = futures::future::join_all(handles)
1941            .await
1942            .into_iter()
1943            .map(|r| r.unwrap())
1944            .collect();
1945
1946        // Only 1 should succeed (3MB limit, 2MB request, can only fit one)
1947        let succeeded = results.iter().filter(|r| r.is_some()).count();
1948        let failed = results.iter().filter(|r| r.is_none()).count();
1949
1950        assert_eq!(succeeded, 1, "Expected exactly 1 task to acquire memory");
1951        assert_eq!(failed, 2, "Expected 2 tasks to fail");
1952
1953        // Clean up
1954        drop(results);
1955        assert_eq!(manager.used_bytes(), 0);
1956    }
1957}