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