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::default()),
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)?;
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
862        // This serves as a workaround of https://github.com/GreptimeTeam/greptimedb/issues/3944
863        // by converting time ranges into predicate.
864        if let Some(time_range) = self.time_range {
865            scan_input =
866                scan_input.with_predicate(time_range_to_predicate(time_range, &self.metadata)?);
867        }
868
869        Ok(scan_input)
870    }
871}
872
873/// Converts time range to predicates so that rows outside the range will be filtered.
874fn time_range_to_predicate(
875    range: TimestampRange,
876    metadata: &RegionMetadataRef,
877) -> Result<PredicateGroup> {
878    let ts_col = metadata.time_index_column();
879
880    // safety: time index column's type must be a valid timestamp type.
881    let ts_col_unit = ts_col
882        .column_schema
883        .data_type
884        .as_timestamp()
885        .unwrap()
886        .unit();
887
888    let exprs = match (range.start(), range.end()) {
889        (Some(start), Some(end)) => {
890            vec![
891                datafusion_expr::col(ts_col.column_schema.name.clone())
892                    .gt_eq(ts_to_lit(*start, ts_col_unit)?),
893                datafusion_expr::col(ts_col.column_schema.name.clone())
894                    .lt(ts_to_lit(*end, ts_col_unit)?),
895            ]
896        }
897        (Some(start), None) => {
898            vec![
899                datafusion_expr::col(ts_col.column_schema.name.clone())
900                    .gt_eq(ts_to_lit(*start, ts_col_unit)?),
901            ]
902        }
903
904        (None, Some(end)) => {
905            vec![
906                datafusion_expr::col(ts_col.column_schema.name.clone())
907                    .lt(ts_to_lit(*end, ts_col_unit)?),
908            ]
909        }
910        (None, None) => {
911            return Ok(PredicateGroup::default());
912        }
913    };
914
915    let predicate = PredicateGroup::new(metadata, &exprs)?;
916    Ok(predicate)
917}
918
919fn ts_to_lit(ts: Timestamp, ts_col_unit: TimeUnit) -> Result<Expr> {
920    let ts = ts
921        .convert_to(ts_col_unit)
922        .context(TimeRangePredicateOverflowSnafu {
923            timestamp: ts,
924            unit: ts_col_unit,
925        })?;
926    let val = ts.value();
927    let scalar_value = match ts_col_unit {
928        TimeUnit::Second => ScalarValue::TimestampSecond(Some(val), None),
929        TimeUnit::Millisecond => ScalarValue::TimestampMillisecond(Some(val), None),
930        TimeUnit::Microsecond => ScalarValue::TimestampMicrosecond(Some(val), None),
931        TimeUnit::Nanosecond => ScalarValue::TimestampNanosecond(Some(val), None),
932    };
933    Ok(datafusion_expr::lit(scalar_value))
934}
935
936/// Finds all expired SSTs across levels.
937fn get_expired_ssts(
938    levels: &[LevelMeta],
939    ttl: Option<TimeToLive>,
940    now: Timestamp,
941) -> Vec<FileHandle> {
942    let Some(ttl) = ttl else {
943        return vec![];
944    };
945
946    levels
947        .iter()
948        .flat_map(|l| l.get_expired_files(&now, &ttl).into_iter())
949        .collect()
950}
951
952/// Estimates compaction memory as the sum of all input files' maximum row-group
953/// uncompressed sizes.
954fn estimate_compaction_bytes(picker_output: &PickerOutput) -> u64 {
955    picker_output
956        .outputs
957        .iter()
958        .flat_map(|output| output.inputs.iter())
959        .map(|file: &FileHandle| {
960            let meta = file.meta_ref();
961            meta.max_row_group_uncompressed_size
962        })
963        .sum()
964}
965
966/// Pending compaction request that is supposed to run after current task is finished,
967/// typically used for manual compactions.
968struct PendingCompaction {
969    /// Compaction options. Currently, it can only be [StrictWindow].
970    pub(crate) options: compact_request::Options,
971    /// Waiters of pending requests.
972    pub(crate) waiter: OptionOutputTx,
973    /// Max parallelism for pending compaction.
974    pub(crate) max_parallelism: usize,
975}
976
977#[cfg(test)]
978mod tests {
979    use std::assert_matches;
980    use std::time::Duration;
981
982    use api::v1::region::StrictWindow;
983    use common_datasource::compression::CompressionType;
984    use common_meta::key::schema_name::SchemaNameValue;
985    use common_time::DatabaseTimeToLive;
986    use tokio::sync::{Barrier, oneshot};
987
988    use super::*;
989    use crate::compaction::memory_manager::{CompactionMemoryGuard, new_compaction_memory_manager};
990    use crate::error::InvalidSchedulerStateSnafu;
991    use crate::manifest::manager::{RegionManifestManager, RegionManifestOptions};
992    use crate::region::ManifestContext;
993    use crate::schedule::scheduler::{Job, Scheduler};
994    use crate::sst::FormatType;
995    use crate::test_util::mock_schema_metadata_manager;
996    use crate::test_util::scheduler_util::{SchedulerEnv, VecScheduler};
997    use crate::test_util::version_util::{VersionControlBuilder, apply_edit};
998
999    struct FailingScheduler;
1000
1001    #[async_trait::async_trait]
1002    impl Scheduler for FailingScheduler {
1003        fn schedule(&self, _job: Job) -> Result<()> {
1004            InvalidSchedulerStateSnafu.fail()
1005        }
1006
1007        async fn stop(&self, _await_termination: bool) -> Result<()> {
1008            Ok(())
1009        }
1010    }
1011
1012    #[tokio::test]
1013    async fn test_find_compaction_options_db_level() {
1014        let env = SchedulerEnv::new().await;
1015        let builder = VersionControlBuilder::new();
1016        let (schema_metadata_manager, kv_backend) = mock_schema_metadata_manager();
1017        let region_id = builder.region_id();
1018        let table_id = region_id.table_id();
1019        // Register table without ttl but with db-level compaction options
1020        let mut schema_value = SchemaNameValue {
1021            ttl: Some(DatabaseTimeToLive::default()),
1022            ..Default::default()
1023        };
1024        schema_value
1025            .extra_options
1026            .insert("compaction.type".to_string(), "twcs".to_string());
1027        schema_value
1028            .extra_options
1029            .insert("compaction.twcs.time_window".to_string(), "2h".to_string());
1030        schema_metadata_manager
1031            .register_region_table_info(
1032                table_id,
1033                "t",
1034                "c",
1035                "s",
1036                Some(schema_value),
1037                kv_backend.clone(),
1038            )
1039            .await;
1040
1041        let version_control = Arc::new(builder.build());
1042        let region_opts = version_control.current().version.options.clone();
1043        let (opts, _) = find_dynamic_options(table_id, &region_opts, &schema_metadata_manager)
1044            .await
1045            .unwrap();
1046        match opts {
1047            crate::region::options::CompactionOptions::Twcs(t) => {
1048                assert_eq!(t.time_window_seconds(), Some(2 * 3600));
1049            }
1050        }
1051        let manifest_ctx = env
1052            .mock_manifest_context(version_control.current().version.metadata.clone())
1053            .await;
1054        let (tx, _rx) = mpsc::channel(4);
1055        let mut scheduler = env.mock_compaction_scheduler(tx);
1056        let (otx, _orx) = oneshot::channel();
1057        let request = scheduler
1058            .region_status
1059            .entry(region_id)
1060            .or_insert_with(|| {
1061                crate::compaction::CompactionStatus::new(
1062                    region_id,
1063                    version_control.clone(),
1064                    env.access_layer.clone(),
1065                )
1066            })
1067            .new_compaction_request(
1068                scheduler.request_sender.clone(),
1069                OptionOutputTx::new(Some(OutputTx::new(otx))),
1070                scheduler.engine_config.clone(),
1071                scheduler.cache_manager.clone(),
1072                &manifest_ctx,
1073                scheduler.listener.clone(),
1074                schema_metadata_manager.clone(),
1075                1,
1076            );
1077        scheduler
1078            .schedule_compaction_request(
1079                request,
1080                compact_request::Options::Regular(Default::default()),
1081            )
1082            .await
1083            .unwrap();
1084    }
1085
1086    #[tokio::test]
1087    async fn test_find_compaction_options_priority() {
1088        fn schema_value_with_twcs(time_window: &str) -> SchemaNameValue {
1089            let mut schema_value = SchemaNameValue {
1090                ttl: Some(DatabaseTimeToLive::default()),
1091                ..Default::default()
1092            };
1093            schema_value
1094                .extra_options
1095                .insert("compaction.type".to_string(), "twcs".to_string());
1096            schema_value.extra_options.insert(
1097                "compaction.twcs.time_window".to_string(),
1098                time_window.to_string(),
1099            );
1100            schema_value
1101        }
1102
1103        let cases = [
1104            (
1105                "db options set and table override set",
1106                Some(schema_value_with_twcs("2h")),
1107                true,
1108                Some(Duration::from_secs(5 * 3600)),
1109                Some(5 * 3600),
1110            ),
1111            (
1112                "db options set and table override not set",
1113                Some(schema_value_with_twcs("2h")),
1114                false,
1115                None,
1116                Some(2 * 3600),
1117            ),
1118            (
1119                "db options not set and table override set",
1120                None,
1121                true,
1122                Some(Duration::from_secs(4 * 3600)),
1123                Some(4 * 3600),
1124            ),
1125            (
1126                "db options not set and table override not set",
1127                None,
1128                false,
1129                None,
1130                None,
1131            ),
1132        ];
1133
1134        for (case_name, schema_value, override_set, table_window, expected_window) in cases {
1135            let builder = VersionControlBuilder::new();
1136            let (schema_metadata_manager, kv_backend) = mock_schema_metadata_manager();
1137            let table_id = builder.region_id().table_id();
1138            schema_metadata_manager
1139                .register_region_table_info(
1140                    table_id,
1141                    "t",
1142                    "c",
1143                    "s",
1144                    schema_value,
1145                    kv_backend.clone(),
1146                )
1147                .await;
1148
1149            let version_control = Arc::new(builder.build());
1150            let mut region_opts = version_control.current().version.options.clone();
1151            region_opts.compaction_override = override_set;
1152            if let Some(window) = table_window {
1153                let crate::region::options::CompactionOptions::Twcs(twcs) =
1154                    &mut region_opts.compaction;
1155                twcs.time_window = Some(window);
1156            }
1157
1158            let (opts, _) = find_dynamic_options(table_id, &region_opts, &schema_metadata_manager)
1159                .await
1160                .unwrap();
1161            match opts {
1162                crate::region::options::CompactionOptions::Twcs(t) => {
1163                    assert_eq!(t.time_window_seconds(), expected_window, "{case_name}");
1164                }
1165            }
1166        }
1167    }
1168
1169    #[tokio::test]
1170    async fn test_schedule_empty() {
1171        let env = SchedulerEnv::new().await;
1172        let (tx, _rx) = mpsc::channel(4);
1173        let mut scheduler = env.mock_compaction_scheduler(tx);
1174        let mut builder = VersionControlBuilder::new();
1175        let (schema_metadata_manager, kv_backend) = mock_schema_metadata_manager();
1176        schema_metadata_manager
1177            .register_region_table_info(
1178                builder.region_id().table_id(),
1179                "test_table",
1180                "test_catalog",
1181                "test_schema",
1182                None,
1183                kv_backend,
1184            )
1185            .await;
1186        // Nothing to compact.
1187        let version_control = Arc::new(builder.build());
1188        let (output_tx, output_rx) = oneshot::channel();
1189        let waiter = OptionOutputTx::from(output_tx);
1190        let manifest_ctx = env
1191            .mock_manifest_context(version_control.current().version.metadata.clone())
1192            .await;
1193        scheduler
1194            .schedule_compaction(
1195                builder.region_id(),
1196                compact_request::Options::Regular(Default::default()),
1197                &version_control,
1198                &env.access_layer,
1199                waiter,
1200                &manifest_ctx,
1201                schema_metadata_manager.clone(),
1202                1,
1203            )
1204            .await
1205            .unwrap();
1206        let output = output_rx.await.unwrap().unwrap();
1207        assert_eq!(output, 0);
1208        assert!(scheduler.region_status.is_empty());
1209
1210        // Only one file, picker won't compact it.
1211        let version_control = Arc::new(builder.push_l0_file(0, 1000).build());
1212        let (output_tx, output_rx) = oneshot::channel();
1213        let waiter = OptionOutputTx::from(output_tx);
1214        scheduler
1215            .schedule_compaction(
1216                builder.region_id(),
1217                compact_request::Options::Regular(Default::default()),
1218                &version_control,
1219                &env.access_layer,
1220                waiter,
1221                &manifest_ctx,
1222                schema_metadata_manager,
1223                1,
1224            )
1225            .await
1226            .unwrap();
1227        let output = output_rx.await.unwrap().unwrap();
1228        assert_eq!(output, 0);
1229        assert!(scheduler.region_status.is_empty());
1230    }
1231
1232    #[tokio::test]
1233    async fn test_schedule_on_finished() {
1234        common_telemetry::init_default_ut_logging();
1235        let job_scheduler = Arc::new(VecScheduler::default());
1236        let env = SchedulerEnv::new().await.scheduler(job_scheduler.clone());
1237        let (tx, _rx) = mpsc::channel(4);
1238        let mut scheduler = env.mock_compaction_scheduler(tx);
1239        let mut builder = VersionControlBuilder::new();
1240        let purger = builder.file_purger();
1241        let region_id = builder.region_id();
1242
1243        let (schema_metadata_manager, kv_backend) = mock_schema_metadata_manager();
1244        schema_metadata_manager
1245            .register_region_table_info(
1246                builder.region_id().table_id(),
1247                "test_table",
1248                "test_catalog",
1249                "test_schema",
1250                None,
1251                kv_backend,
1252            )
1253            .await;
1254
1255        // 5 files to compact.
1256        let end = 1000 * 1000;
1257        let version_control = Arc::new(
1258            builder
1259                .push_l0_file(0, end)
1260                .push_l0_file(10, end)
1261                .push_l0_file(50, end)
1262                .push_l0_file(80, end)
1263                .push_l0_file(90, end)
1264                .build(),
1265        );
1266        let manifest_ctx = env
1267            .mock_manifest_context(version_control.current().version.metadata.clone())
1268            .await;
1269        scheduler
1270            .schedule_compaction(
1271                region_id,
1272                compact_request::Options::Regular(Default::default()),
1273                &version_control,
1274                &env.access_layer,
1275                OptionOutputTx::none(),
1276                &manifest_ctx,
1277                schema_metadata_manager.clone(),
1278                1,
1279            )
1280            .await
1281            .unwrap();
1282        // Should schedule 1 compaction.
1283        assert_eq!(1, scheduler.region_status.len());
1284        assert_eq!(1, job_scheduler.num_jobs());
1285        let data = version_control.current();
1286        let file_metas: Vec<_> = data.version.ssts.levels()[0]
1287            .files
1288            .values()
1289            .map(|file| file.meta_ref().clone())
1290            .collect();
1291
1292        // 5 files for next compaction and removes old files.
1293        apply_edit(
1294            &version_control,
1295            &[(0, end), (20, end), (40, end), (60, end), (80, end)],
1296            &file_metas,
1297            purger.clone(),
1298        );
1299        // The task is pending.
1300        let (tx, _rx) = oneshot::channel();
1301        scheduler
1302            .schedule_compaction(
1303                region_id,
1304                compact_request::Options::Regular(Default::default()),
1305                &version_control,
1306                &env.access_layer,
1307                OptionOutputTx::new(Some(OutputTx::new(tx))),
1308                &manifest_ctx,
1309                schema_metadata_manager.clone(),
1310                1,
1311            )
1312            .await
1313            .unwrap();
1314        assert_eq!(1, scheduler.region_status.len());
1315        assert_eq!(1, job_scheduler.num_jobs());
1316        assert!(
1317            !scheduler
1318                .region_status
1319                .get(&builder.region_id())
1320                .unwrap()
1321                .waiters
1322                .is_empty()
1323        );
1324
1325        // On compaction finished and schedule next compaction.
1326        scheduler
1327            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager.clone())
1328            .await;
1329        assert_eq!(1, scheduler.region_status.len());
1330        assert_eq!(2, job_scheduler.num_jobs());
1331
1332        // 5 files for next compaction.
1333        apply_edit(
1334            &version_control,
1335            &[(0, end), (20, end), (40, end), (60, end), (80, end)],
1336            &[],
1337            purger.clone(),
1338        );
1339        let (tx, _rx) = oneshot::channel();
1340        // The task is pending.
1341        scheduler
1342            .schedule_compaction(
1343                region_id,
1344                compact_request::Options::Regular(Default::default()),
1345                &version_control,
1346                &env.access_layer,
1347                OptionOutputTx::new(Some(OutputTx::new(tx))),
1348                &manifest_ctx,
1349                schema_metadata_manager,
1350                1,
1351            )
1352            .await
1353            .unwrap();
1354        assert_eq!(2, job_scheduler.num_jobs());
1355        assert!(
1356            !scheduler
1357                .region_status
1358                .get(&builder.region_id())
1359                .unwrap()
1360                .waiters
1361                .is_empty()
1362        );
1363    }
1364
1365    #[tokio::test]
1366    async fn test_manual_compaction_when_compaction_in_progress() {
1367        common_telemetry::init_default_ut_logging();
1368        let job_scheduler = Arc::new(VecScheduler::default());
1369        let env = SchedulerEnv::new().await.scheduler(job_scheduler.clone());
1370        let (tx, _rx) = mpsc::channel(4);
1371        let mut scheduler = env.mock_compaction_scheduler(tx);
1372        let mut builder = VersionControlBuilder::new();
1373        let purger = builder.file_purger();
1374        let region_id = builder.region_id();
1375
1376        let (schema_metadata_manager, kv_backend) = mock_schema_metadata_manager();
1377        schema_metadata_manager
1378            .register_region_table_info(
1379                builder.region_id().table_id(),
1380                "test_table",
1381                "test_catalog",
1382                "test_schema",
1383                None,
1384                kv_backend,
1385            )
1386            .await;
1387
1388        // 5 files to compact.
1389        let end = 1000 * 1000;
1390        let version_control = Arc::new(
1391            builder
1392                .push_l0_file(0, end)
1393                .push_l0_file(10, end)
1394                .push_l0_file(50, end)
1395                .push_l0_file(80, end)
1396                .push_l0_file(90, end)
1397                .build(),
1398        );
1399        let manifest_ctx = env
1400            .mock_manifest_context(version_control.current().version.metadata.clone())
1401            .await;
1402
1403        let file_metas: Vec<_> = version_control.current().version.ssts.levels()[0]
1404            .files
1405            .values()
1406            .map(|file| file.meta_ref().clone())
1407            .collect();
1408
1409        // 5 files for next compaction and removes old files.
1410        apply_edit(
1411            &version_control,
1412            &[(0, end), (20, end), (40, end), (60, end), (80, end)],
1413            &file_metas,
1414            purger.clone(),
1415        );
1416
1417        scheduler
1418            .schedule_compaction(
1419                region_id,
1420                compact_request::Options::Regular(Default::default()),
1421                &version_control,
1422                &env.access_layer,
1423                OptionOutputTx::none(),
1424                &manifest_ctx,
1425                schema_metadata_manager.clone(),
1426                1,
1427            )
1428            .await
1429            .unwrap();
1430        // Should schedule 1 compaction.
1431        assert_eq!(1, scheduler.region_status.len());
1432        assert_eq!(1, job_scheduler.num_jobs());
1433        assert!(
1434            scheduler
1435                .region_status
1436                .get(&region_id)
1437                .unwrap()
1438                .pending_request
1439                .is_none()
1440        );
1441
1442        // Schedule another manual compaction.
1443        let (tx, _rx) = oneshot::channel();
1444        scheduler
1445            .schedule_compaction(
1446                region_id,
1447                compact_request::Options::StrictWindow(StrictWindow { window_seconds: 60 }),
1448                &version_control,
1449                &env.access_layer,
1450                OptionOutputTx::new(Some(OutputTx::new(tx))),
1451                &manifest_ctx,
1452                schema_metadata_manager.clone(),
1453                1,
1454            )
1455            .await
1456            .unwrap();
1457        assert_eq!(1, scheduler.region_status.len());
1458        // Current job num should be 1 since compaction is in progress.
1459        assert_eq!(1, job_scheduler.num_jobs());
1460        let status = scheduler.region_status.get(&builder.region_id()).unwrap();
1461        assert!(status.pending_request.is_some());
1462
1463        // On compaction finished and schedule next compaction.
1464        scheduler
1465            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager.clone())
1466            .await;
1467        assert_eq!(1, scheduler.region_status.len());
1468        assert_eq!(2, job_scheduler.num_jobs());
1469
1470        let status = scheduler.region_status.get(&builder.region_id()).unwrap();
1471        assert!(status.pending_request.is_none());
1472    }
1473
1474    #[tokio::test]
1475    async fn test_compaction_bypass_in_staging_mode() {
1476        let env = SchedulerEnv::new().await;
1477        let (tx, _rx) = mpsc::channel(4);
1478        let mut scheduler = env.mock_compaction_scheduler(tx);
1479
1480        // Create version control and manifest context for staging mode
1481        let builder = VersionControlBuilder::new();
1482        let version_control = Arc::new(builder.build());
1483        let region_id = version_control.current().version.metadata.region_id;
1484
1485        // Create staging manifest context using the same pattern as SchedulerEnv
1486        let staging_manifest_ctx = {
1487            let manager = RegionManifestManager::new(
1488                version_control.current().version.metadata.clone(),
1489                0,
1490                RegionManifestOptions {
1491                    manifest_dir: "".to_string(),
1492                    object_store: env.access_layer.object_store().clone(),
1493                    compress_type: CompressionType::Uncompressed,
1494                    checkpoint_distance: 10,
1495                    remove_file_options: Default::default(),
1496                    manifest_cache: None,
1497                },
1498                FormatType::PrimaryKey,
1499                &Default::default(),
1500            )
1501            .await
1502            .unwrap();
1503            Arc::new(ManifestContext::new(
1504                manager,
1505                RegionRoleState::Leader(RegionLeaderState::Staging),
1506            ))
1507        };
1508
1509        let (schema_metadata_manager, _kv_backend) = mock_schema_metadata_manager();
1510
1511        // Test regular compaction bypass in staging mode
1512        let (tx, rx) = oneshot::channel();
1513        scheduler
1514            .schedule_compaction(
1515                region_id,
1516                compact_request::Options::Regular(Default::default()),
1517                &version_control,
1518                &env.access_layer,
1519                OptionOutputTx::new(Some(OutputTx::new(tx))),
1520                &staging_manifest_ctx,
1521                schema_metadata_manager,
1522                1,
1523            )
1524            .await
1525            .unwrap();
1526
1527        let result = rx.await.unwrap();
1528        assert_eq!(result.unwrap(), 0); // is there a better way to check this?
1529        assert_eq!(0, scheduler.region_status.len());
1530    }
1531
1532    #[tokio::test]
1533    async fn test_add_ddl_request_to_pending() {
1534        let env = SchedulerEnv::new().await;
1535        let (tx, _rx) = mpsc::channel(4);
1536        let mut scheduler = env.mock_compaction_scheduler(tx);
1537        let builder = VersionControlBuilder::new();
1538        let version_control = Arc::new(builder.build());
1539        let region_id = builder.region_id();
1540
1541        scheduler.region_status.insert(
1542            region_id,
1543            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1544        );
1545
1546        let (output_tx, _output_rx) = oneshot::channel();
1547        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1548            region_id,
1549            sender: OptionOutputTx::from(output_tx),
1550            request: crate::request::DdlRequest::EnterStaging(
1551                store_api::region_request::EnterStagingRequest {
1552                    partition_directive:
1553                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1554                },
1555            ),
1556        });
1557
1558        assert!(scheduler.has_pending_ddls(region_id));
1559    }
1560
1561    #[tokio::test]
1562    async fn test_pending_ddl_request_failed_on_compaction_failed() {
1563        let env = SchedulerEnv::new().await;
1564        let (tx, _rx) = mpsc::channel(4);
1565        let mut scheduler = env.mock_compaction_scheduler(tx);
1566        let builder = VersionControlBuilder::new();
1567        let version_control = Arc::new(builder.build());
1568        let region_id = builder.region_id();
1569
1570        scheduler.region_status.insert(
1571            region_id,
1572            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1573        );
1574
1575        let (output_tx, output_rx) = oneshot::channel();
1576        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1577            region_id,
1578            sender: OptionOutputTx::from(output_tx),
1579            request: crate::request::DdlRequest::EnterStaging(
1580                store_api::region_request::EnterStagingRequest {
1581                    partition_directive:
1582                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1583                },
1584            ),
1585        });
1586
1587        assert!(scheduler.has_pending_ddls(region_id));
1588        scheduler
1589            .on_compaction_failed(region_id, Arc::new(RegionClosedSnafu { region_id }.build()));
1590
1591        assert!(!scheduler.has_pending_ddls(region_id));
1592        let result = output_rx.await.unwrap();
1593        assert_matches!(result, Err(_));
1594    }
1595
1596    #[tokio::test]
1597    async fn test_pending_ddl_request_failed_on_region_closed() {
1598        let env = SchedulerEnv::new().await;
1599        let (tx, _rx) = mpsc::channel(4);
1600        let mut scheduler = env.mock_compaction_scheduler(tx);
1601        let builder = VersionControlBuilder::new();
1602        let version_control = Arc::new(builder.build());
1603        let region_id = builder.region_id();
1604
1605        scheduler.region_status.insert(
1606            region_id,
1607            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1608        );
1609
1610        let (output_tx, output_rx) = oneshot::channel();
1611        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1612            region_id,
1613            sender: OptionOutputTx::from(output_tx),
1614            request: crate::request::DdlRequest::EnterStaging(
1615                store_api::region_request::EnterStagingRequest {
1616                    partition_directive:
1617                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1618                },
1619            ),
1620        });
1621
1622        assert!(scheduler.has_pending_ddls(region_id));
1623        scheduler.on_region_closed(region_id);
1624
1625        assert!(!scheduler.has_pending_ddls(region_id));
1626        let result = output_rx.await.unwrap();
1627        assert_matches!(result, Err(_));
1628    }
1629
1630    #[tokio::test]
1631    async fn test_pending_ddl_request_failed_on_region_dropped() {
1632        let env = SchedulerEnv::new().await;
1633        let (tx, _rx) = mpsc::channel(4);
1634        let mut scheduler = env.mock_compaction_scheduler(tx);
1635        let builder = VersionControlBuilder::new();
1636        let version_control = Arc::new(builder.build());
1637        let region_id = builder.region_id();
1638
1639        scheduler.region_status.insert(
1640            region_id,
1641            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1642        );
1643
1644        let (output_tx, output_rx) = oneshot::channel();
1645        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1646            region_id,
1647            sender: OptionOutputTx::from(output_tx),
1648            request: crate::request::DdlRequest::EnterStaging(
1649                store_api::region_request::EnterStagingRequest {
1650                    partition_directive:
1651                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1652                },
1653            ),
1654        });
1655
1656        assert!(scheduler.has_pending_ddls(region_id));
1657        scheduler.on_region_dropped(region_id);
1658
1659        assert!(!scheduler.has_pending_ddls(region_id));
1660        let result = output_rx.await.unwrap();
1661        assert_matches!(result, Err(_));
1662    }
1663
1664    #[tokio::test]
1665    async fn test_pending_ddl_request_failed_on_region_truncated() {
1666        let env = SchedulerEnv::new().await;
1667        let (tx, _rx) = mpsc::channel(4);
1668        let mut scheduler = env.mock_compaction_scheduler(tx);
1669        let builder = VersionControlBuilder::new();
1670        let version_control = Arc::new(builder.build());
1671        let region_id = builder.region_id();
1672
1673        scheduler.region_status.insert(
1674            region_id,
1675            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1676        );
1677
1678        let (output_tx, output_rx) = oneshot::channel();
1679        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1680            region_id,
1681            sender: OptionOutputTx::from(output_tx),
1682            request: crate::request::DdlRequest::EnterStaging(
1683                store_api::region_request::EnterStagingRequest {
1684                    partition_directive:
1685                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1686                },
1687            ),
1688        });
1689
1690        assert!(scheduler.has_pending_ddls(region_id));
1691        scheduler.on_region_truncated(region_id);
1692
1693        assert!(!scheduler.has_pending_ddls(region_id));
1694        let result = output_rx.await.unwrap();
1695        assert_matches!(result, Err(_));
1696    }
1697
1698    #[tokio::test]
1699    async fn test_on_compaction_finished_returns_pending_ddl_requests() {
1700        let job_scheduler = Arc::new(VecScheduler::default());
1701        let env = SchedulerEnv::new().await.scheduler(job_scheduler.clone());
1702        let (tx, _rx) = mpsc::channel(4);
1703        let mut scheduler = env.mock_compaction_scheduler(tx);
1704        let builder = VersionControlBuilder::new();
1705        let version_control = Arc::new(builder.build());
1706        let region_id = builder.region_id();
1707        let manifest_ctx = env
1708            .mock_manifest_context(version_control.current().version.metadata.clone())
1709            .await;
1710        let (schema_metadata_manager, _kv_backend) = mock_schema_metadata_manager();
1711
1712        scheduler.region_status.insert(
1713            region_id,
1714            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1715        );
1716
1717        let (output_tx, _output_rx) = oneshot::channel();
1718        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1719            region_id,
1720            sender: OptionOutputTx::from(output_tx),
1721            request: crate::request::DdlRequest::EnterStaging(
1722                store_api::region_request::EnterStagingRequest {
1723                    partition_directive:
1724                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1725                },
1726            ),
1727        });
1728
1729        let pending_ddls = scheduler
1730            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager)
1731            .await;
1732
1733        assert_eq!(pending_ddls.len(), 1);
1734        assert!(!scheduler.has_pending_ddls(region_id));
1735        assert!(!scheduler.region_status.contains_key(&region_id));
1736        assert_eq!(job_scheduler.num_jobs(), 0);
1737    }
1738
1739    #[tokio::test]
1740    async fn test_on_compaction_finished_replays_pending_ddl_after_manual_noop() {
1741        let env = SchedulerEnv::new().await;
1742        let (tx, _rx) = mpsc::channel(4);
1743        let mut scheduler = env.mock_compaction_scheduler(tx);
1744        let builder = VersionControlBuilder::new();
1745        let version_control = Arc::new(builder.build());
1746        let region_id = builder.region_id();
1747        let manifest_ctx = env
1748            .mock_manifest_context(version_control.current().version.metadata.clone())
1749            .await;
1750        let (schema_metadata_manager, _kv_backend) = mock_schema_metadata_manager();
1751
1752        let (manual_tx, manual_rx) = oneshot::channel();
1753        let mut status =
1754            CompactionStatus::new(region_id, version_control.clone(), env.access_layer.clone());
1755        status.set_pending_request(PendingCompaction {
1756            options: compact_request::Options::Regular(Default::default()),
1757            waiter: OptionOutputTx::from(manual_tx),
1758            max_parallelism: 1,
1759        });
1760        scheduler.region_status.insert(region_id, status);
1761
1762        let (ddl_tx, _ddl_rx) = oneshot::channel();
1763        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1764            region_id,
1765            sender: OptionOutputTx::from(ddl_tx),
1766            request: crate::request::DdlRequest::EnterStaging(
1767                store_api::region_request::EnterStagingRequest {
1768                    partition_directive:
1769                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1770                },
1771            ),
1772        });
1773
1774        let pending_ddls = scheduler
1775            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager)
1776            .await;
1777
1778        assert_eq!(pending_ddls.len(), 1);
1779        assert!(!scheduler.region_status.contains_key(&region_id));
1780        assert_eq!(manual_rx.await.unwrap().unwrap(), 0);
1781    }
1782
1783    #[tokio::test]
1784    async fn test_on_compaction_finished_returns_empty_when_region_absent() {
1785        let env = SchedulerEnv::new().await;
1786        let (tx, _rx) = mpsc::channel(4);
1787        let mut scheduler = env.mock_compaction_scheduler(tx);
1788        let builder = VersionControlBuilder::new();
1789        let region_id = builder.region_id();
1790        let version_control = Arc::new(builder.build());
1791        let manifest_ctx = env
1792            .mock_manifest_context(version_control.current().version.metadata.clone())
1793            .await;
1794        let (schema_metadata_manager, _kv_backend) = mock_schema_metadata_manager();
1795
1796        let pending_ddls = scheduler
1797            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager)
1798            .await;
1799
1800        assert!(pending_ddls.is_empty());
1801    }
1802
1803    #[tokio::test]
1804    async fn test_on_compaction_finished_manual_schedule_error_cleans_status() {
1805        let env = SchedulerEnv::new()
1806            .await
1807            .scheduler(Arc::new(FailingScheduler));
1808        let (tx, _rx) = mpsc::channel(4);
1809        let mut scheduler = env.mock_compaction_scheduler(tx);
1810        let mut builder = VersionControlBuilder::new();
1811        let end = 1000 * 1000;
1812        let version_control = Arc::new(
1813            builder
1814                .push_l0_file(0, end)
1815                .push_l0_file(10, end)
1816                .push_l0_file(50, end)
1817                .push_l0_file(80, end)
1818                .push_l0_file(90, end)
1819                .build(),
1820        );
1821        let region_id = builder.region_id();
1822        let manifest_ctx = env
1823            .mock_manifest_context(version_control.current().version.metadata.clone())
1824            .await;
1825        let (schema_metadata_manager, _kv_backend) = mock_schema_metadata_manager();
1826
1827        let (manual_tx, manual_rx) = oneshot::channel();
1828        let mut status =
1829            CompactionStatus::new(region_id, version_control.clone(), env.access_layer.clone());
1830        status.set_pending_request(PendingCompaction {
1831            options: compact_request::Options::Regular(Default::default()),
1832            waiter: OptionOutputTx::from(manual_tx),
1833            max_parallelism: 1,
1834        });
1835        scheduler.region_status.insert(region_id, status);
1836
1837        let (ddl_tx, ddl_rx) = oneshot::channel();
1838        scheduler.add_ddl_request_to_pending(SenderDdlRequest {
1839            region_id,
1840            sender: OptionOutputTx::from(ddl_tx),
1841            request: crate::request::DdlRequest::EnterStaging(
1842                store_api::region_request::EnterStagingRequest {
1843                    partition_directive:
1844                        store_api::region_request::StagingPartitionDirective::RejectAllWrites,
1845                },
1846            ),
1847        });
1848
1849        let pending_ddls = scheduler
1850            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager)
1851            .await;
1852
1853        assert!(pending_ddls.is_empty());
1854        assert!(!scheduler.region_status.contains_key(&region_id));
1855        assert!(manual_rx.await.is_err());
1856        assert_matches!(ddl_rx.await.unwrap(), Err(_));
1857    }
1858
1859    #[tokio::test]
1860    async fn test_on_compaction_finished_next_schedule_noop_removes_status() {
1861        let env = SchedulerEnv::new().await;
1862        let (tx, _rx) = mpsc::channel(4);
1863        let mut scheduler = env.mock_compaction_scheduler(tx);
1864        let builder = VersionControlBuilder::new();
1865        let version_control = Arc::new(builder.build());
1866        let region_id = builder.region_id();
1867        let manifest_ctx = env
1868            .mock_manifest_context(version_control.current().version.metadata.clone())
1869            .await;
1870        let (schema_metadata_manager, _kv_backend) = mock_schema_metadata_manager();
1871
1872        scheduler.region_status.insert(
1873            region_id,
1874            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1875        );
1876
1877        let pending_ddls = scheduler
1878            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager)
1879            .await;
1880
1881        assert!(pending_ddls.is_empty());
1882        assert!(!scheduler.region_status.contains_key(&region_id));
1883    }
1884
1885    #[tokio::test]
1886    async fn test_on_compaction_finished_next_schedule_error_cleans_status() {
1887        let env = SchedulerEnv::new()
1888            .await
1889            .scheduler(Arc::new(FailingScheduler));
1890        let (tx, _rx) = mpsc::channel(4);
1891        let mut scheduler = env.mock_compaction_scheduler(tx);
1892        let mut builder = VersionControlBuilder::new();
1893        let end = 1000 * 1000;
1894        let version_control = Arc::new(
1895            builder
1896                .push_l0_file(0, end)
1897                .push_l0_file(10, end)
1898                .push_l0_file(50, end)
1899                .push_l0_file(80, end)
1900                .push_l0_file(90, end)
1901                .build(),
1902        );
1903        let region_id = builder.region_id();
1904        let manifest_ctx = env
1905            .mock_manifest_context(version_control.current().version.metadata.clone())
1906            .await;
1907        let (schema_metadata_manager, _kv_backend) = mock_schema_metadata_manager();
1908
1909        scheduler.region_status.insert(
1910            region_id,
1911            CompactionStatus::new(region_id, version_control, env.access_layer.clone()),
1912        );
1913
1914        let pending_ddls = scheduler
1915            .on_compaction_finished(region_id, &manifest_ctx, schema_metadata_manager)
1916            .await;
1917
1918        assert!(pending_ddls.is_empty());
1919        assert!(!scheduler.region_status.contains_key(&region_id));
1920    }
1921
1922    #[tokio::test]
1923    async fn test_concurrent_memory_competition() {
1924        let manager = Arc::new(new_compaction_memory_manager(3 * 1024 * 1024)); // 3MB
1925        let barrier = Arc::new(Barrier::new(3));
1926        let mut handles = vec![];
1927
1928        // Spawn 3 tasks competing for memory, each trying to acquire 2MB
1929        for _i in 0..3 {
1930            let mgr = manager.clone();
1931            let bar = barrier.clone();
1932            let handle = tokio::spawn(async move {
1933                bar.wait().await; // Synchronize start
1934                mgr.try_acquire(2 * 1024 * 1024)
1935            });
1936            handles.push(handle);
1937        }
1938
1939        let results: Vec<Option<CompactionMemoryGuard>> = futures::future::join_all(handles)
1940            .await
1941            .into_iter()
1942            .map(|r| r.unwrap())
1943            .collect();
1944
1945        // Only 1 should succeed (3MB limit, 2MB request, can only fit one)
1946        let succeeded = results.iter().filter(|r| r.is_some()).count();
1947        let failed = results.iter().filter(|r| r.is_none()).count();
1948
1949        assert_eq!(succeeded, 1, "Expected exactly 1 task to acquire memory");
1950        assert_eq!(failed, 2, "Expected 2 tasks to fail");
1951
1952        // Clean up
1953        drop(results);
1954        assert_eq!(manager.used_bytes(), 0);
1955    }
1956}