mito2/
engine.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
15//! Mito region engine.
16
17#[cfg(test)]
18mod alter_test;
19#[cfg(test)]
20mod append_mode_test;
21#[cfg(test)]
22mod basic_test;
23#[cfg(test)]
24mod batch_catchup_test;
25#[cfg(test)]
26mod batch_open_test;
27#[cfg(test)]
28mod bump_committed_sequence_test;
29#[cfg(test)]
30mod catchup_test;
31#[cfg(test)]
32mod close_test;
33#[cfg(test)]
34pub(crate) mod compaction_test;
35#[cfg(test)]
36mod create_test;
37#[cfg(test)]
38mod drop_test;
39#[cfg(test)]
40mod edit_region_test;
41#[cfg(test)]
42mod filter_deleted_test;
43#[cfg(test)]
44mod flush_test;
45#[cfg(test)]
46mod index_build_test;
47#[cfg(any(test, feature = "test"))]
48pub mod listener;
49#[cfg(test)]
50mod merge_mode_test;
51#[cfg(test)]
52mod open_test;
53#[cfg(test)]
54mod parallel_test;
55#[cfg(test)]
56mod projection_test;
57#[cfg(test)]
58mod prune_test;
59#[cfg(test)]
60mod row_selector_test;
61#[cfg(test)]
62mod scan_corrupt;
63#[cfg(test)]
64mod scan_test;
65#[cfg(test)]
66mod set_role_state_test;
67#[cfg(test)]
68mod staging_test;
69#[cfg(test)]
70mod sync_test;
71#[cfg(test)]
72mod truncate_test;
73
74#[cfg(test)]
75mod copy_region_from_test;
76#[cfg(test)]
77mod remap_manifests_test;
78
79#[cfg(test)]
80mod apply_staging_manifest_test;
81mod puffin_index;
82
83use std::any::Any;
84use std::collections::HashMap;
85use std::sync::Arc;
86use std::time::Instant;
87
88use api::region::RegionResponse;
89use async_trait::async_trait;
90use common_base::Plugins;
91use common_error::ext::BoxedError;
92use common_meta::error::UnexpectedSnafu;
93use common_meta::key::SchemaMetadataManagerRef;
94use common_recordbatch::{MemoryPermit, QueryMemoryTracker, SendableRecordBatchStream};
95use common_stat::get_total_memory_bytes;
96use common_telemetry::{info, tracing, warn};
97use common_wal::options::{WAL_OPTIONS_KEY, WalOptions};
98use futures::future::{join_all, try_join_all};
99use futures::stream::{self, Stream, StreamExt};
100use object_store::manager::ObjectStoreManagerRef;
101use snafu::{OptionExt, ResultExt, ensure};
102use store_api::ManifestVersion;
103use store_api::codec::PrimaryKeyEncoding;
104use store_api::logstore::LogStore;
105use store_api::logstore::provider::{KafkaProvider, Provider};
106use store_api::metadata::{ColumnMetadata, RegionMetadataRef};
107use store_api::metric_engine_consts::{
108    MANIFEST_INFO_EXTENSION_KEY, TABLE_COLUMN_METADATA_EXTENSION_KEY,
109};
110use store_api::region_engine::{
111    BatchResponses, MitoCopyRegionFromRequest, MitoCopyRegionFromResponse, RegionEngine,
112    RegionManifestInfo, RegionRole, RegionScannerRef, RegionStatistic, RemapManifestsRequest,
113    RemapManifestsResponse, SetRegionRoleStateResponse, SettableRegionRoleState,
114    SyncRegionFromRequest, SyncRegionFromResponse,
115};
116use store_api::region_request::{
117    AffectedRows, RegionCatchupRequest, RegionOpenRequest, RegionRequest,
118};
119use store_api::sst_entry::{ManifestSstEntry, PuffinIndexMetaEntry, StorageSstEntry};
120use store_api::storage::{FileId, FileRefsManifest, RegionId, ScanRequest, SequenceNumber};
121use tokio::sync::{Semaphore, oneshot};
122
123use crate::access_layer::RegionFilePathFactory;
124use crate::cache::{CacheManagerRef, CacheStrategy};
125use crate::config::MitoConfig;
126use crate::engine::puffin_index::{IndexEntryContext, collect_index_entries_from_puffin};
127use crate::error::{
128    InvalidRequestSnafu, JoinSnafu, MitoManifestInfoSnafu, RecvSnafu, RegionNotFoundSnafu, Result,
129    SerdeJsonSnafu, SerializeColumnMetadataSnafu, SerializeManifestSnafu,
130};
131#[cfg(feature = "enterprise")]
132use crate::extension::BoxedExtensionRangeProviderFactory;
133use crate::gc::GcLimiterRef;
134use crate::manifest::action::RegionEdit;
135use crate::memtable::MemtableStats;
136use crate::metrics::{
137    HANDLE_REQUEST_ELAPSED, SCAN_MEMORY_USAGE_BYTES, SCAN_REQUESTS_REJECTED_TOTAL,
138};
139use crate::read::scan_region::{ScanRegion, Scanner};
140use crate::read::stream::ScanBatchStream;
141use crate::region::MitoRegionRef;
142use crate::region::opener::PartitionExprFetcherRef;
143use crate::request::{RegionEditRequest, WorkerRequest};
144use crate::sst::file::{FileMeta, RegionFileId, RegionIndexId};
145use crate::sst::file_ref::FileReferenceManagerRef;
146use crate::wal::entry_distributor::{
147    DEFAULT_ENTRY_RECEIVER_BUFFER_SIZE, build_wal_entry_distributor_and_receivers,
148};
149use crate::wal::raw_entry_reader::{LogStoreRawEntryReader, RawEntryReader};
150use crate::worker::WorkerGroup;
151
152pub const MITO_ENGINE_NAME: &str = "mito";
153
154pub struct MitoEngineBuilder<'a, S: LogStore> {
155    data_home: &'a str,
156    config: MitoConfig,
157    log_store: Arc<S>,
158    object_store_manager: ObjectStoreManagerRef,
159    schema_metadata_manager: SchemaMetadataManagerRef,
160    file_ref_manager: FileReferenceManagerRef,
161    partition_expr_fetcher: PartitionExprFetcherRef,
162    plugins: Plugins,
163    max_concurrent_queries: usize,
164    #[cfg(feature = "enterprise")]
165    extension_range_provider_factory: Option<BoxedExtensionRangeProviderFactory>,
166}
167
168impl<'a, S: LogStore> MitoEngineBuilder<'a, S> {
169    #[allow(clippy::too_many_arguments)]
170    pub fn new(
171        data_home: &'a str,
172        config: MitoConfig,
173        log_store: Arc<S>,
174        object_store_manager: ObjectStoreManagerRef,
175        schema_metadata_manager: SchemaMetadataManagerRef,
176        file_ref_manager: FileReferenceManagerRef,
177        partition_expr_fetcher: PartitionExprFetcherRef,
178        plugins: Plugins,
179        max_concurrent_queries: usize,
180    ) -> Self {
181        Self {
182            data_home,
183            config,
184            log_store,
185            object_store_manager,
186            schema_metadata_manager,
187            file_ref_manager,
188            plugins,
189            partition_expr_fetcher,
190            max_concurrent_queries,
191            #[cfg(feature = "enterprise")]
192            extension_range_provider_factory: None,
193        }
194    }
195
196    #[cfg(feature = "enterprise")]
197    #[must_use]
198    pub fn with_extension_range_provider_factory(
199        self,
200        extension_range_provider_factory: Option<BoxedExtensionRangeProviderFactory>,
201    ) -> Self {
202        Self {
203            extension_range_provider_factory,
204            ..self
205        }
206    }
207
208    pub async fn try_build(mut self) -> Result<MitoEngine> {
209        self.config.sanitize(self.data_home)?;
210
211        let config = Arc::new(self.config);
212        let workers = WorkerGroup::start(
213            config.clone(),
214            self.log_store.clone(),
215            self.object_store_manager,
216            self.schema_metadata_manager,
217            self.file_ref_manager,
218            self.partition_expr_fetcher.clone(),
219            self.plugins,
220        )
221        .await?;
222        let wal_raw_entry_reader = Arc::new(LogStoreRawEntryReader::new(self.log_store));
223        let total_memory = get_total_memory_bytes().max(0) as u64;
224        let scan_memory_limit = config.scan_memory_limit.resolve(total_memory) as usize;
225        let scan_memory_tracker =
226            QueryMemoryTracker::new(scan_memory_limit, self.max_concurrent_queries)
227                .with_on_update(|usage| {
228                    SCAN_MEMORY_USAGE_BYTES.set(usage as i64);
229                })
230                .with_on_reject(|| {
231                    SCAN_REQUESTS_REJECTED_TOTAL.inc();
232                });
233
234        let inner = EngineInner {
235            workers,
236            config,
237            wal_raw_entry_reader,
238            scan_memory_tracker,
239            #[cfg(feature = "enterprise")]
240            extension_range_provider_factory: None,
241        };
242
243        #[cfg(feature = "enterprise")]
244        let inner =
245            inner.with_extension_range_provider_factory(self.extension_range_provider_factory);
246
247        Ok(MitoEngine {
248            inner: Arc::new(inner),
249        })
250    }
251}
252
253/// Region engine implementation for timeseries data.
254#[derive(Clone)]
255pub struct MitoEngine {
256    inner: Arc<EngineInner>,
257}
258
259impl MitoEngine {
260    /// Returns a new [MitoEngine] with specific `config`, `log_store` and `object_store`.
261    #[allow(clippy::too_many_arguments)]
262    pub async fn new<S: LogStore>(
263        data_home: &str,
264        config: MitoConfig,
265        log_store: Arc<S>,
266        object_store_manager: ObjectStoreManagerRef,
267        schema_metadata_manager: SchemaMetadataManagerRef,
268        file_ref_manager: FileReferenceManagerRef,
269        partition_expr_fetcher: PartitionExprFetcherRef,
270        plugins: Plugins,
271    ) -> Result<MitoEngine> {
272        let builder = MitoEngineBuilder::new(
273            data_home,
274            config,
275            log_store,
276            object_store_manager,
277            schema_metadata_manager,
278            file_ref_manager,
279            partition_expr_fetcher,
280            plugins,
281            0, // Default: no limit on concurrent queries
282        );
283        builder.try_build().await
284    }
285
286    pub fn mito_config(&self) -> &MitoConfig {
287        &self.inner.config
288    }
289
290    pub fn cache_manager(&self) -> CacheManagerRef {
291        self.inner.workers.cache_manager()
292    }
293
294    pub fn file_ref_manager(&self) -> FileReferenceManagerRef {
295        self.inner.workers.file_ref_manager()
296    }
297
298    pub fn gc_limiter(&self) -> GcLimiterRef {
299        self.inner.workers.gc_limiter()
300    }
301
302    /// Get all tmp ref files for given region ids, excluding files that's already in manifest.
303    pub async fn get_snapshot_of_file_refs(
304        &self,
305        file_handle_regions: impl IntoIterator<Item = RegionId>,
306        manifest_regions: HashMap<RegionId, Vec<RegionId>>,
307    ) -> Result<FileRefsManifest> {
308        let file_ref_mgr = self.file_ref_manager();
309
310        let file_handle_regions = file_handle_regions.into_iter().collect::<Vec<_>>();
311        // Convert region IDs to MitoRegionRef objects, ignore regions that do not exist on current datanode
312        // as regions on other datanodes are not managed by this engine.
313        let query_regions: Vec<MitoRegionRef> = file_handle_regions
314            .into_iter()
315            .filter_map(|region_id| self.find_region(region_id))
316            .collect();
317
318        let related_regions: Vec<(MitoRegionRef, Vec<RegionId>)> = manifest_regions
319            .into_iter()
320            .filter_map(|(related_region, queries)| {
321                self.find_region(related_region).map(|r| (r, queries))
322            })
323            .collect();
324
325        file_ref_mgr
326            .get_snapshot_of_file_refs(query_regions, related_regions)
327            .await
328    }
329
330    /// Returns true if the specific region exists.
331    pub fn is_region_exists(&self, region_id: RegionId) -> bool {
332        self.inner.workers.is_region_exists(region_id)
333    }
334
335    /// Returns true if the specific region exists.
336    pub fn is_region_opening(&self, region_id: RegionId) -> bool {
337        self.inner.workers.is_region_opening(region_id)
338    }
339
340    /// Returns true if the specific region is catching up.
341    pub fn is_region_catching_up(&self, region_id: RegionId) -> bool {
342        self.inner.workers.is_region_catching_up(region_id)
343    }
344
345    /// Returns the region disk/memory statistic.
346    pub fn get_region_statistic(&self, region_id: RegionId) -> Option<RegionStatistic> {
347        self.find_region(region_id)
348            .map(|region| region.region_statistic())
349    }
350
351    /// Returns primary key encoding of the region.
352    pub fn get_primary_key_encoding(&self, region_id: RegionId) -> Option<PrimaryKeyEncoding> {
353        self.find_region(region_id)
354            .map(|r| r.primary_key_encoding())
355    }
356
357    /// Handle substrait query and return a stream of record batches
358    ///
359    /// Notice that the output stream's ordering is not guranateed. If order
360    /// matter, please use [`scanner`] to build a [`Scanner`] to consume.
361    #[tracing::instrument(skip_all)]
362    pub async fn scan_to_stream(
363        &self,
364        region_id: RegionId,
365        request: ScanRequest,
366    ) -> Result<SendableRecordBatchStream, BoxedError> {
367        self.scanner(region_id, request)
368            .await
369            .map_err(BoxedError::new)?
370            .scan()
371            .await
372    }
373
374    /// Scan [`Batch`]es by [`ScanRequest`].
375    pub async fn scan_batch(
376        &self,
377        region_id: RegionId,
378        request: ScanRequest,
379        filter_deleted: bool,
380    ) -> Result<ScanBatchStream> {
381        let mut scan_region = self.scan_region(region_id, request)?;
382        scan_region.set_filter_deleted(filter_deleted);
383        scan_region.scanner().await?.scan_batch()
384    }
385
386    /// Returns a scanner to scan for `request`.
387    pub(crate) async fn scanner(
388        &self,
389        region_id: RegionId,
390        request: ScanRequest,
391    ) -> Result<Scanner> {
392        self.scan_region(region_id, request)?.scanner().await
393    }
394
395    /// Scans a region.
396    fn scan_region(&self, region_id: RegionId, request: ScanRequest) -> Result<ScanRegion> {
397        self.inner.scan_region(region_id, request)
398    }
399
400    /// Edit region's metadata by [RegionEdit] directly. Use with care.
401    /// Now we only allow adding files or removing files from region (the [RegionEdit] struct can only contain a non-empty "files_to_add" or "files_to_remove" field).
402    /// Other region editing intention will result in an "invalid request" error.
403    /// Also note that if a region is to be edited directly, we MUST not write data to it thereafter.
404    pub async fn edit_region(&self, region_id: RegionId, edit: RegionEdit) -> Result<()> {
405        let _timer = HANDLE_REQUEST_ELAPSED
406            .with_label_values(&["edit_region"])
407            .start_timer();
408
409        ensure!(
410            is_valid_region_edit(&edit),
411            InvalidRequestSnafu {
412                region_id,
413                reason: "invalid region edit"
414            }
415        );
416
417        let (tx, rx) = oneshot::channel();
418        let request = WorkerRequest::EditRegion(RegionEditRequest {
419            region_id,
420            edit,
421            tx,
422        });
423        self.inner
424            .workers
425            .submit_to_worker(region_id, request)
426            .await?;
427        rx.await.context(RecvSnafu)?
428    }
429
430    /// Handles copy region from request.
431    ///
432    /// This method is only supported for internal use and is not exposed in the trait implementation.
433    pub async fn copy_region_from(
434        &self,
435        region_id: RegionId,
436        request: MitoCopyRegionFromRequest,
437    ) -> Result<MitoCopyRegionFromResponse> {
438        self.inner.copy_region_from(region_id, request).await
439    }
440
441    #[cfg(test)]
442    pub(crate) fn get_region(&self, id: RegionId) -> Option<crate::region::MitoRegionRef> {
443        self.find_region(id)
444    }
445
446    pub fn find_region(&self, region_id: RegionId) -> Option<MitoRegionRef> {
447        self.inner.workers.get_region(region_id)
448    }
449
450    fn encode_manifest_info_to_extensions(
451        region_id: &RegionId,
452        manifest_info: RegionManifestInfo,
453        extensions: &mut HashMap<String, Vec<u8>>,
454    ) -> Result<()> {
455        let region_manifest_info = vec![(*region_id, manifest_info)];
456
457        extensions.insert(
458            MANIFEST_INFO_EXTENSION_KEY.to_string(),
459            RegionManifestInfo::encode_list(&region_manifest_info).context(SerdeJsonSnafu)?,
460        );
461        info!(
462            "Added manifest info: {:?} to extensions, region_id: {:?}",
463            region_manifest_info, region_id
464        );
465        Ok(())
466    }
467
468    fn encode_column_metadatas_to_extensions(
469        region_id: &RegionId,
470        column_metadatas: Vec<ColumnMetadata>,
471        extensions: &mut HashMap<String, Vec<u8>>,
472    ) -> Result<()> {
473        extensions.insert(
474            TABLE_COLUMN_METADATA_EXTENSION_KEY.to_string(),
475            ColumnMetadata::encode_list(&column_metadatas).context(SerializeColumnMetadataSnafu)?,
476        );
477        info!(
478            "Added column metadatas: {:?} to extensions, region_id: {:?}",
479            column_metadatas, region_id
480        );
481        Ok(())
482    }
483
484    /// Find the current version's memtables and SSTs stats by region_id.
485    /// The stats must be collected in one place one time to ensure data consistency.
486    pub fn find_memtable_and_sst_stats(
487        &self,
488        region_id: RegionId,
489    ) -> Result<(Vec<MemtableStats>, Vec<FileMeta>)> {
490        let region = self
491            .find_region(region_id)
492            .context(RegionNotFoundSnafu { region_id })?;
493
494        let version = region.version();
495        let memtable_stats = version
496            .memtables
497            .list_memtables()
498            .iter()
499            .map(|x| x.stats())
500            .collect::<Vec<_>>();
501
502        let sst_stats = version
503            .ssts
504            .levels()
505            .iter()
506            .flat_map(|level| level.files().map(|x| x.meta_ref()))
507            .cloned()
508            .collect::<Vec<_>>();
509        Ok((memtable_stats, sst_stats))
510    }
511
512    /// Lists all SSTs from the manifest of all regions in the engine.
513    pub async fn all_ssts_from_manifest(&self) -> Vec<ManifestSstEntry> {
514        let node_id = self.inner.workers.file_ref_manager().node_id();
515        let regions = self.inner.workers.all_regions();
516
517        let mut results = Vec::new();
518        for region in regions {
519            let mut entries = region.manifest_sst_entries().await;
520            for e in &mut entries {
521                e.node_id = node_id;
522            }
523            results.extend(entries);
524        }
525
526        results
527    }
528
529    /// Lists metadata about all puffin index targets stored in the engine.
530    pub async fn all_index_metas(&self) -> Vec<PuffinIndexMetaEntry> {
531        let node_id = self.inner.workers.file_ref_manager().node_id();
532        let cache_manager = self.inner.workers.cache_manager();
533        let puffin_metadata_cache = cache_manager.puffin_metadata_cache().cloned();
534        let bloom_filter_cache = cache_manager.bloom_filter_index_cache().cloned();
535        let inverted_index_cache = cache_manager.inverted_index_cache().cloned();
536
537        let mut results = Vec::new();
538
539        for region in self.inner.workers.all_regions() {
540            let manifest_entries = region.manifest_sst_entries().await;
541            let access_layer = region.access_layer.clone();
542            let table_dir = access_layer.table_dir().to_string();
543            let path_type = access_layer.path_type();
544            let object_store = access_layer.object_store().clone();
545            let puffin_factory = access_layer.puffin_manager_factory().clone();
546            let path_factory = RegionFilePathFactory::new(table_dir, path_type);
547
548            let entry_futures = manifest_entries.into_iter().map(|entry| {
549                let object_store = object_store.clone();
550                let path_factory = path_factory.clone();
551                let puffin_factory = puffin_factory.clone();
552                let puffin_metadata_cache = puffin_metadata_cache.clone();
553                let bloom_filter_cache = bloom_filter_cache.clone();
554                let inverted_index_cache = inverted_index_cache.clone();
555
556                async move {
557                    let Some(index_file_path) = entry.index_file_path.as_ref() else {
558                        return Vec::new();
559                    };
560
561                    let index_version = entry.index_version;
562                    let file_id = match FileId::parse_str(&entry.file_id) {
563                        Ok(file_id) => file_id,
564                        Err(err) => {
565                            warn!(
566                                err;
567                                "Failed to parse puffin index file id, table_dir: {}, file_id: {}",
568                                entry.table_dir,
569                                entry.file_id
570                            );
571                            return Vec::new();
572                        }
573                    };
574                    let region_index_id = RegionIndexId::new(
575                        RegionFileId::new(entry.region_id, file_id),
576                        index_version,
577                    );
578                    let context = IndexEntryContext {
579                        table_dir: &entry.table_dir,
580                        index_file_path: index_file_path.as_str(),
581                        region_id: entry.region_id,
582                        table_id: entry.table_id,
583                        region_number: entry.region_number,
584                        region_group: entry.region_group,
585                        region_sequence: entry.region_sequence,
586                        file_id: &entry.file_id,
587                        index_file_size: entry.index_file_size,
588                        node_id,
589                    };
590
591                    let manager = puffin_factory
592                        .build(object_store, path_factory)
593                        .with_puffin_metadata_cache(puffin_metadata_cache);
594
595                    collect_index_entries_from_puffin(
596                        manager,
597                        region_index_id,
598                        context,
599                        bloom_filter_cache,
600                        inverted_index_cache,
601                    )
602                    .await
603                }
604            });
605
606            let mut meta_stream = stream::iter(entry_futures).buffer_unordered(8); // Parallelism is 8.
607            while let Some(mut metas) = meta_stream.next().await {
608                results.append(&mut metas);
609            }
610        }
611
612        results
613    }
614
615    /// Lists all SSTs from the storage layer of all regions in the engine.
616    pub fn all_ssts_from_storage(&self) -> impl Stream<Item = Result<StorageSstEntry>> {
617        let node_id = self.inner.workers.file_ref_manager().node_id();
618        let regions = self.inner.workers.all_regions();
619
620        let mut layers_distinct_table_dirs = HashMap::new();
621        for region in regions {
622            let table_dir = region.access_layer.table_dir();
623            if !layers_distinct_table_dirs.contains_key(table_dir) {
624                layers_distinct_table_dirs
625                    .insert(table_dir.to_string(), region.access_layer.clone());
626            }
627        }
628
629        stream::iter(layers_distinct_table_dirs)
630            .map(|(_, access_layer)| access_layer.storage_sst_entries())
631            .flatten()
632            .map(move |entry| {
633                entry.map(move |mut entry| {
634                    entry.node_id = node_id;
635                    entry
636                })
637            })
638    }
639}
640
641/// Check whether the region edit is valid.
642///
643/// Only adding or removing files to region is considered valid now.
644fn is_valid_region_edit(edit: &RegionEdit) -> bool {
645    (!edit.files_to_add.is_empty() || !edit.files_to_remove.is_empty())
646        && matches!(
647            edit,
648            RegionEdit {
649                files_to_add: _,
650                files_to_remove: _,
651                timestamp_ms: _,
652                compaction_time_window: None,
653                flushed_entry_id: None,
654                flushed_sequence: None,
655                ..
656            }
657        )
658}
659
660/// Inner struct of [MitoEngine].
661struct EngineInner {
662    /// Region workers group.
663    workers: WorkerGroup,
664    /// Config of the engine.
665    config: Arc<MitoConfig>,
666    /// The Wal raw entry reader.
667    wal_raw_entry_reader: Arc<dyn RawEntryReader>,
668    /// Memory tracker for table scans.
669    scan_memory_tracker: QueryMemoryTracker,
670    #[cfg(feature = "enterprise")]
671    extension_range_provider_factory: Option<BoxedExtensionRangeProviderFactory>,
672}
673
674type TopicGroupedRegionOpenRequests = HashMap<String, Vec<(RegionId, RegionOpenRequest)>>;
675
676/// Returns requests([TopicGroupedRegionOpenRequests]) grouped by topic and remaining requests.
677fn prepare_batch_open_requests(
678    requests: Vec<(RegionId, RegionOpenRequest)>,
679) -> Result<(
680    TopicGroupedRegionOpenRequests,
681    Vec<(RegionId, RegionOpenRequest)>,
682)> {
683    let mut topic_to_regions: HashMap<String, Vec<(RegionId, RegionOpenRequest)>> = HashMap::new();
684    let mut remaining_regions: Vec<(RegionId, RegionOpenRequest)> = Vec::new();
685    for (region_id, request) in requests {
686        let options = if let Some(options) = request.options.get(WAL_OPTIONS_KEY) {
687            serde_json::from_str(options).context(SerdeJsonSnafu)?
688        } else {
689            WalOptions::RaftEngine
690        };
691        match options {
692            WalOptions::Kafka(options) => {
693                topic_to_regions
694                    .entry(options.topic)
695                    .or_default()
696                    .push((region_id, request));
697            }
698            WalOptions::RaftEngine | WalOptions::Noop => {
699                remaining_regions.push((region_id, request));
700            }
701        }
702    }
703
704    Ok((topic_to_regions, remaining_regions))
705}
706
707impl EngineInner {
708    #[cfg(feature = "enterprise")]
709    #[must_use]
710    fn with_extension_range_provider_factory(
711        self,
712        extension_range_provider_factory: Option<BoxedExtensionRangeProviderFactory>,
713    ) -> Self {
714        Self {
715            extension_range_provider_factory,
716            ..self
717        }
718    }
719
720    /// Stop the inner engine.
721    async fn stop(&self) -> Result<()> {
722        self.workers.stop().await
723    }
724
725    fn find_region(&self, region_id: RegionId) -> Result<MitoRegionRef> {
726        self.workers
727            .get_region(region_id)
728            .context(RegionNotFoundSnafu { region_id })
729    }
730
731    /// Get metadata of a region.
732    ///
733    /// Returns error if the region doesn't exist.
734    fn get_metadata(&self, region_id: RegionId) -> Result<RegionMetadataRef> {
735        // Reading a region doesn't need to go through the region worker thread.
736        let region = self.find_region(region_id)?;
737        Ok(region.metadata())
738    }
739
740    async fn open_topic_regions(
741        &self,
742        topic: String,
743        region_requests: Vec<(RegionId, RegionOpenRequest)>,
744    ) -> Result<Vec<(RegionId, Result<AffectedRows>)>> {
745        let now = Instant::now();
746        let region_ids = region_requests
747            .iter()
748            .map(|(region_id, _)| *region_id)
749            .collect::<Vec<_>>();
750        let provider = Provider::kafka_provider(topic);
751        let (distributor, entry_receivers) = build_wal_entry_distributor_and_receivers(
752            provider.clone(),
753            self.wal_raw_entry_reader.clone(),
754            &region_ids,
755            DEFAULT_ENTRY_RECEIVER_BUFFER_SIZE,
756        );
757
758        let mut responses = Vec::with_capacity(region_requests.len());
759        for ((region_id, request), entry_receiver) in
760            region_requests.into_iter().zip(entry_receivers)
761        {
762            let (request, receiver) =
763                WorkerRequest::new_open_region_request(region_id, request, Some(entry_receiver));
764            self.workers.submit_to_worker(region_id, request).await?;
765            responses.push(async move { receiver.await.context(RecvSnafu)? });
766        }
767
768        // Waits for entries distribution.
769        let distribution =
770            common_runtime::spawn_global(async move { distributor.distribute().await });
771        // Waits for worker returns.
772        let responses = join_all(responses).await;
773        distribution.await.context(JoinSnafu)??;
774
775        let num_failure = responses.iter().filter(|r| r.is_err()).count();
776        info!(
777            "Opened {} regions for topic '{}', failures: {}, elapsed: {:?}",
778            region_ids.len() - num_failure,
779            // Safety: provider is kafka provider.
780            provider.as_kafka_provider().unwrap(),
781            num_failure,
782            now.elapsed(),
783        );
784        Ok(region_ids.into_iter().zip(responses).collect())
785    }
786
787    async fn handle_batch_open_requests(
788        &self,
789        parallelism: usize,
790        requests: Vec<(RegionId, RegionOpenRequest)>,
791    ) -> Result<Vec<(RegionId, Result<AffectedRows>)>> {
792        let semaphore = Arc::new(Semaphore::new(parallelism));
793        let (topic_to_region_requests, remaining_region_requests) =
794            prepare_batch_open_requests(requests)?;
795        let mut responses =
796            Vec::with_capacity(topic_to_region_requests.len() + remaining_region_requests.len());
797
798        if !topic_to_region_requests.is_empty() {
799            let mut tasks = Vec::with_capacity(topic_to_region_requests.len());
800            for (topic, region_requests) in topic_to_region_requests {
801                let semaphore_moved = semaphore.clone();
802                tasks.push(async move {
803                    // Safety: semaphore must exist
804                    let _permit = semaphore_moved.acquire().await.unwrap();
805                    self.open_topic_regions(topic, region_requests).await
806                })
807            }
808            let r = try_join_all(tasks).await?;
809            responses.extend(r.into_iter().flatten());
810        }
811
812        if !remaining_region_requests.is_empty() {
813            let mut tasks = Vec::with_capacity(remaining_region_requests.len());
814            let mut region_ids = Vec::with_capacity(remaining_region_requests.len());
815            for (region_id, request) in remaining_region_requests {
816                let semaphore_moved = semaphore.clone();
817                region_ids.push(region_id);
818                tasks.push(async move {
819                    // Safety: semaphore must exist
820                    let _permit = semaphore_moved.acquire().await.unwrap();
821                    let (request, receiver) =
822                        WorkerRequest::new_open_region_request(region_id, request, None);
823
824                    self.workers.submit_to_worker(region_id, request).await?;
825
826                    receiver.await.context(RecvSnafu)?
827                })
828            }
829
830            let results = join_all(tasks).await;
831            responses.extend(region_ids.into_iter().zip(results));
832        }
833
834        Ok(responses)
835    }
836
837    async fn catchup_topic_regions(
838        &self,
839        provider: Provider,
840        region_requests: Vec<(RegionId, RegionCatchupRequest)>,
841    ) -> Result<Vec<(RegionId, Result<AffectedRows>)>> {
842        let now = Instant::now();
843        let region_ids = region_requests
844            .iter()
845            .map(|(region_id, _)| *region_id)
846            .collect::<Vec<_>>();
847        let (distributor, entry_receivers) = build_wal_entry_distributor_and_receivers(
848            provider.clone(),
849            self.wal_raw_entry_reader.clone(),
850            &region_ids,
851            DEFAULT_ENTRY_RECEIVER_BUFFER_SIZE,
852        );
853
854        let mut responses = Vec::with_capacity(region_requests.len());
855        for ((region_id, request), entry_receiver) in
856            region_requests.into_iter().zip(entry_receivers)
857        {
858            let (request, receiver) =
859                WorkerRequest::new_catchup_region_request(region_id, request, Some(entry_receiver));
860            self.workers.submit_to_worker(region_id, request).await?;
861            responses.push(async move { receiver.await.context(RecvSnafu)? });
862        }
863
864        // Wait for entries distribution.
865        let distribution =
866            common_runtime::spawn_global(async move { distributor.distribute().await });
867        // Wait for worker returns.
868        let responses = join_all(responses).await;
869        distribution.await.context(JoinSnafu)??;
870
871        let num_failure = responses.iter().filter(|r| r.is_err()).count();
872        info!(
873            "Caught up {} regions for topic '{}', failures: {}, elapsed: {:?}",
874            region_ids.len() - num_failure,
875            // Safety: provider is kafka provider.
876            provider.as_kafka_provider().unwrap(),
877            num_failure,
878            now.elapsed(),
879        );
880
881        Ok(region_ids.into_iter().zip(responses).collect())
882    }
883
884    async fn handle_batch_catchup_requests(
885        &self,
886        parallelism: usize,
887        requests: Vec<(RegionId, RegionCatchupRequest)>,
888    ) -> Result<Vec<(RegionId, Result<AffectedRows>)>> {
889        let mut responses = Vec::with_capacity(requests.len());
890        let mut topic_regions: HashMap<Arc<KafkaProvider>, Vec<_>> = HashMap::new();
891        let mut remaining_region_requests = vec![];
892
893        for (region_id, request) in requests {
894            match self.workers.get_region(region_id) {
895                Some(region) => match region.provider.as_kafka_provider() {
896                    Some(provider) => {
897                        topic_regions
898                            .entry(provider.clone())
899                            .or_default()
900                            .push((region_id, request));
901                    }
902                    None => {
903                        remaining_region_requests.push((region_id, request));
904                    }
905                },
906                None => responses.push((region_id, RegionNotFoundSnafu { region_id }.fail())),
907            }
908        }
909
910        let semaphore = Arc::new(Semaphore::new(parallelism));
911
912        if !topic_regions.is_empty() {
913            let mut tasks = Vec::with_capacity(topic_regions.len());
914            for (provider, region_requests) in topic_regions {
915                let semaphore_moved = semaphore.clone();
916                tasks.push(async move {
917                    // Safety: semaphore must exist
918                    let _permit = semaphore_moved.acquire().await.unwrap();
919                    self.catchup_topic_regions(Provider::Kafka(provider), region_requests)
920                        .await
921                })
922            }
923
924            let r = try_join_all(tasks).await?;
925            responses.extend(r.into_iter().flatten());
926        }
927
928        if !remaining_region_requests.is_empty() {
929            let mut tasks = Vec::with_capacity(remaining_region_requests.len());
930            let mut region_ids = Vec::with_capacity(remaining_region_requests.len());
931            for (region_id, request) in remaining_region_requests {
932                let semaphore_moved = semaphore.clone();
933                region_ids.push(region_id);
934                tasks.push(async move {
935                    // Safety: semaphore must exist
936                    let _permit = semaphore_moved.acquire().await.unwrap();
937                    let (request, receiver) =
938                        WorkerRequest::new_catchup_region_request(region_id, request, None);
939
940                    self.workers.submit_to_worker(region_id, request).await?;
941
942                    receiver.await.context(RecvSnafu)?
943                })
944            }
945
946            let results = join_all(tasks).await;
947            responses.extend(region_ids.into_iter().zip(results));
948        }
949
950        Ok(responses)
951    }
952
953    /// Handles [RegionRequest] and return its executed result.
954    async fn handle_request(
955        &self,
956        region_id: RegionId,
957        request: RegionRequest,
958    ) -> Result<AffectedRows> {
959        let region_metadata = self.get_metadata(region_id).ok();
960        let (request, receiver) =
961            WorkerRequest::try_from_region_request(region_id, request, region_metadata)?;
962        self.workers.submit_to_worker(region_id, request).await?;
963
964        receiver.await.context(RecvSnafu)?
965    }
966
967    /// Returns the sequence of latest committed data.
968    fn get_committed_sequence(&self, region_id: RegionId) -> Result<SequenceNumber> {
969        // Reading a region doesn't need to go through the region worker thread.
970        self.find_region(region_id)
971            .map(|r| r.find_committed_sequence())
972    }
973
974    /// Handles the scan `request` and returns a [ScanRegion].
975    fn scan_region(&self, region_id: RegionId, request: ScanRequest) -> Result<ScanRegion> {
976        let query_start = Instant::now();
977        // Reading a region doesn't need to go through the region worker thread.
978        let region = self.find_region(region_id)?;
979        let version = region.version();
980        // Get cache.
981        let cache_manager = self.workers.cache_manager();
982
983        let scan_region = ScanRegion::new(
984            version,
985            region.access_layer.clone(),
986            request,
987            CacheStrategy::EnableAll(cache_manager),
988        )
989        .with_parallel_scan_channel_size(self.config.parallel_scan_channel_size)
990        .with_max_concurrent_scan_files(self.config.max_concurrent_scan_files)
991        .with_ignore_inverted_index(self.config.inverted_index.apply_on_query.disabled())
992        .with_ignore_fulltext_index(self.config.fulltext_index.apply_on_query.disabled())
993        .with_ignore_bloom_filter(self.config.bloom_filter_index.apply_on_query.disabled())
994        .with_start_time(query_start);
995
996        #[cfg(feature = "enterprise")]
997        let scan_region = self.maybe_fill_extension_range_provider(scan_region, region);
998
999        Ok(scan_region)
1000    }
1001
1002    #[cfg(feature = "enterprise")]
1003    fn maybe_fill_extension_range_provider(
1004        &self,
1005        mut scan_region: ScanRegion,
1006        region: MitoRegionRef,
1007    ) -> ScanRegion {
1008        if region.is_follower()
1009            && let Some(factory) = self.extension_range_provider_factory.as_ref()
1010        {
1011            scan_region
1012                .set_extension_range_provider(factory.create_extension_range_provider(region));
1013        }
1014        scan_region
1015    }
1016
1017    /// Converts the [`RegionRole`].
1018    fn set_region_role(&self, region_id: RegionId, role: RegionRole) -> Result<()> {
1019        let region = self.find_region(region_id)?;
1020        region.set_role(role);
1021        Ok(())
1022    }
1023
1024    /// Sets read-only for a region and ensures no more writes in the region after it returns.
1025    async fn set_region_role_state_gracefully(
1026        &self,
1027        region_id: RegionId,
1028        region_role_state: SettableRegionRoleState,
1029    ) -> Result<SetRegionRoleStateResponse> {
1030        // Notes: It acquires the mutable ownership to ensure no other threads,
1031        // Therefore, we submit it to the worker.
1032        let (request, receiver) =
1033            WorkerRequest::new_set_readonly_gracefully(region_id, region_role_state);
1034        self.workers.submit_to_worker(region_id, request).await?;
1035
1036        receiver.await.context(RecvSnafu)
1037    }
1038
1039    async fn sync_region(
1040        &self,
1041        region_id: RegionId,
1042        manifest_info: RegionManifestInfo,
1043    ) -> Result<(ManifestVersion, bool)> {
1044        ensure!(manifest_info.is_mito(), MitoManifestInfoSnafu);
1045        let manifest_version = manifest_info.data_manifest_version();
1046        let (request, receiver) =
1047            WorkerRequest::new_sync_region_request(region_id, manifest_version);
1048        self.workers.submit_to_worker(region_id, request).await?;
1049
1050        receiver.await.context(RecvSnafu)?
1051    }
1052
1053    async fn remap_manifests(
1054        &self,
1055        request: RemapManifestsRequest,
1056    ) -> Result<RemapManifestsResponse> {
1057        let region_id = request.region_id;
1058        let (request, receiver) = WorkerRequest::try_from_remap_manifests_request(request)?;
1059        self.workers.submit_to_worker(region_id, request).await?;
1060        let manifests = receiver.await.context(RecvSnafu)??;
1061
1062        let new_manifests = manifests
1063            .into_iter()
1064            .map(|(region_id, manifest)| {
1065                Ok((
1066                    region_id,
1067                    serde_json::to_string(&manifest)
1068                        .context(SerializeManifestSnafu { region_id })?,
1069                ))
1070            })
1071            .collect::<Result<HashMap<_, _>>>()?;
1072        Ok(RemapManifestsResponse { new_manifests })
1073    }
1074
1075    async fn copy_region_from(
1076        &self,
1077        region_id: RegionId,
1078        request: MitoCopyRegionFromRequest,
1079    ) -> Result<MitoCopyRegionFromResponse> {
1080        let (request, receiver) =
1081            WorkerRequest::try_from_copy_region_from_request(region_id, request)?;
1082        self.workers.submit_to_worker(region_id, request).await?;
1083        let response = receiver.await.context(RecvSnafu)??;
1084        Ok(response)
1085    }
1086
1087    fn role(&self, region_id: RegionId) -> Option<RegionRole> {
1088        self.workers.get_region(region_id).map(|region| {
1089            if region.is_follower() {
1090                RegionRole::Follower
1091            } else {
1092                RegionRole::Leader
1093            }
1094        })
1095    }
1096}
1097
1098#[async_trait]
1099impl RegionEngine for MitoEngine {
1100    fn name(&self) -> &str {
1101        MITO_ENGINE_NAME
1102    }
1103
1104    #[tracing::instrument(skip_all)]
1105    async fn handle_batch_open_requests(
1106        &self,
1107        parallelism: usize,
1108        requests: Vec<(RegionId, RegionOpenRequest)>,
1109    ) -> Result<BatchResponses, BoxedError> {
1110        // TODO(weny): add metrics.
1111        self.inner
1112            .handle_batch_open_requests(parallelism, requests)
1113            .await
1114            .map(|responses| {
1115                responses
1116                    .into_iter()
1117                    .map(|(region_id, response)| {
1118                        (
1119                            region_id,
1120                            response.map(RegionResponse::new).map_err(BoxedError::new),
1121                        )
1122                    })
1123                    .collect::<Vec<_>>()
1124            })
1125            .map_err(BoxedError::new)
1126    }
1127
1128    #[tracing::instrument(skip_all)]
1129    async fn handle_batch_catchup_requests(
1130        &self,
1131        parallelism: usize,
1132        requests: Vec<(RegionId, RegionCatchupRequest)>,
1133    ) -> Result<BatchResponses, BoxedError> {
1134        self.inner
1135            .handle_batch_catchup_requests(parallelism, requests)
1136            .await
1137            .map(|responses| {
1138                responses
1139                    .into_iter()
1140                    .map(|(region_id, response)| {
1141                        (
1142                            region_id,
1143                            response.map(RegionResponse::new).map_err(BoxedError::new),
1144                        )
1145                    })
1146                    .collect::<Vec<_>>()
1147            })
1148            .map_err(BoxedError::new)
1149    }
1150
1151    #[tracing::instrument(skip_all)]
1152    async fn handle_request(
1153        &self,
1154        region_id: RegionId,
1155        request: RegionRequest,
1156    ) -> Result<RegionResponse, BoxedError> {
1157        let _timer = HANDLE_REQUEST_ELAPSED
1158            .with_label_values(&[request.request_type()])
1159            .start_timer();
1160
1161        let is_alter = matches!(request, RegionRequest::Alter(_));
1162        let is_create = matches!(request, RegionRequest::Create(_));
1163        let mut response = self
1164            .inner
1165            .handle_request(region_id, request)
1166            .await
1167            .map(RegionResponse::new)
1168            .map_err(BoxedError::new)?;
1169
1170        if is_alter {
1171            self.handle_alter_response(region_id, &mut response)
1172                .map_err(BoxedError::new)?;
1173        } else if is_create {
1174            self.handle_create_response(region_id, &mut response)
1175                .map_err(BoxedError::new)?;
1176        }
1177
1178        Ok(response)
1179    }
1180
1181    #[tracing::instrument(skip_all)]
1182    async fn handle_query(
1183        &self,
1184        region_id: RegionId,
1185        request: ScanRequest,
1186    ) -> Result<RegionScannerRef, BoxedError> {
1187        self.scan_region(region_id, request)
1188            .map_err(BoxedError::new)?
1189            .region_scanner()
1190            .await
1191            .map_err(BoxedError::new)
1192    }
1193
1194    fn register_query_memory_permit(&self) -> Option<Arc<MemoryPermit>> {
1195        Some(Arc::new(self.inner.scan_memory_tracker.register_permit()))
1196    }
1197
1198    async fn get_committed_sequence(
1199        &self,
1200        region_id: RegionId,
1201    ) -> Result<SequenceNumber, BoxedError> {
1202        self.inner
1203            .get_committed_sequence(region_id)
1204            .map_err(BoxedError::new)
1205    }
1206
1207    /// Retrieve region's metadata.
1208    async fn get_metadata(
1209        &self,
1210        region_id: RegionId,
1211    ) -> std::result::Result<RegionMetadataRef, BoxedError> {
1212        self.inner.get_metadata(region_id).map_err(BoxedError::new)
1213    }
1214
1215    /// Stop the engine.
1216    ///
1217    /// Stopping the engine doesn't stop the underlying log store as other components might
1218    /// still use it. (When no other components are referencing the log store, it will
1219    /// automatically shutdown.)
1220    async fn stop(&self) -> std::result::Result<(), BoxedError> {
1221        self.inner.stop().await.map_err(BoxedError::new)
1222    }
1223
1224    fn region_statistic(&self, region_id: RegionId) -> Option<RegionStatistic> {
1225        self.get_region_statistic(region_id)
1226    }
1227
1228    fn set_region_role(&self, region_id: RegionId, role: RegionRole) -> Result<(), BoxedError> {
1229        self.inner
1230            .set_region_role(region_id, role)
1231            .map_err(BoxedError::new)
1232    }
1233
1234    async fn set_region_role_state_gracefully(
1235        &self,
1236        region_id: RegionId,
1237        region_role_state: SettableRegionRoleState,
1238    ) -> Result<SetRegionRoleStateResponse, BoxedError> {
1239        let _timer = HANDLE_REQUEST_ELAPSED
1240            .with_label_values(&["set_region_role_state_gracefully"])
1241            .start_timer();
1242
1243        self.inner
1244            .set_region_role_state_gracefully(region_id, region_role_state)
1245            .await
1246            .map_err(BoxedError::new)
1247    }
1248
1249    async fn sync_region(
1250        &self,
1251        region_id: RegionId,
1252        request: SyncRegionFromRequest,
1253    ) -> Result<SyncRegionFromResponse, BoxedError> {
1254        let manifest_info = request
1255            .into_region_manifest_info()
1256            .context(UnexpectedSnafu {
1257                err_msg: "Expected a manifest info request",
1258            })
1259            .map_err(BoxedError::new)?;
1260        let (_, synced) = self
1261            .inner
1262            .sync_region(region_id, manifest_info)
1263            .await
1264            .map_err(BoxedError::new)?;
1265
1266        Ok(SyncRegionFromResponse::Mito { synced })
1267    }
1268
1269    async fn remap_manifests(
1270        &self,
1271        request: RemapManifestsRequest,
1272    ) -> Result<RemapManifestsResponse, BoxedError> {
1273        self.inner
1274            .remap_manifests(request)
1275            .await
1276            .map_err(BoxedError::new)
1277    }
1278
1279    fn role(&self, region_id: RegionId) -> Option<RegionRole> {
1280        self.inner.role(region_id)
1281    }
1282
1283    fn as_any(&self) -> &dyn Any {
1284        self
1285    }
1286}
1287
1288impl MitoEngine {
1289    fn handle_alter_response(
1290        &self,
1291        region_id: RegionId,
1292        response: &mut RegionResponse,
1293    ) -> Result<()> {
1294        if let Some(statistic) = self.region_statistic(region_id) {
1295            Self::encode_manifest_info_to_extensions(
1296                &region_id,
1297                statistic.manifest,
1298                &mut response.extensions,
1299            )?;
1300        }
1301        let column_metadatas = self
1302            .inner
1303            .find_region(region_id)
1304            .ok()
1305            .map(|r| r.metadata().column_metadatas.clone());
1306        if let Some(column_metadatas) = column_metadatas {
1307            Self::encode_column_metadatas_to_extensions(
1308                &region_id,
1309                column_metadatas,
1310                &mut response.extensions,
1311            )?;
1312        }
1313        Ok(())
1314    }
1315
1316    fn handle_create_response(
1317        &self,
1318        region_id: RegionId,
1319        response: &mut RegionResponse,
1320    ) -> Result<()> {
1321        let column_metadatas = self
1322            .inner
1323            .find_region(region_id)
1324            .ok()
1325            .map(|r| r.metadata().column_metadatas.clone());
1326        if let Some(column_metadatas) = column_metadatas {
1327            Self::encode_column_metadatas_to_extensions(
1328                &region_id,
1329                column_metadatas,
1330                &mut response.extensions,
1331            )?;
1332        }
1333        Ok(())
1334    }
1335}
1336
1337// Tests methods.
1338#[cfg(any(test, feature = "test"))]
1339#[allow(clippy::too_many_arguments)]
1340impl MitoEngine {
1341    /// Returns a new [MitoEngine] for tests.
1342    pub async fn new_for_test<S: LogStore>(
1343        data_home: &str,
1344        mut config: MitoConfig,
1345        log_store: Arc<S>,
1346        object_store_manager: ObjectStoreManagerRef,
1347        write_buffer_manager: Option<crate::flush::WriteBufferManagerRef>,
1348        listener: Option<crate::engine::listener::EventListenerRef>,
1349        time_provider: crate::time_provider::TimeProviderRef,
1350        schema_metadata_manager: SchemaMetadataManagerRef,
1351        file_ref_manager: FileReferenceManagerRef,
1352        partition_expr_fetcher: PartitionExprFetcherRef,
1353    ) -> Result<MitoEngine> {
1354        config.sanitize(data_home)?;
1355
1356        let config = Arc::new(config);
1357        let wal_raw_entry_reader = Arc::new(LogStoreRawEntryReader::new(log_store.clone()));
1358        let total_memory = get_total_memory_bytes().max(0) as u64;
1359        let scan_memory_limit = config.scan_memory_limit.resolve(total_memory) as usize;
1360        let scan_memory_tracker = QueryMemoryTracker::new(scan_memory_limit, 0)
1361            .with_on_update(|usage| {
1362                SCAN_MEMORY_USAGE_BYTES.set(usage as i64);
1363            })
1364            .with_on_reject(|| {
1365                SCAN_REQUESTS_REJECTED_TOTAL.inc();
1366            });
1367        Ok(MitoEngine {
1368            inner: Arc::new(EngineInner {
1369                workers: WorkerGroup::start_for_test(
1370                    config.clone(),
1371                    log_store,
1372                    object_store_manager,
1373                    write_buffer_manager,
1374                    listener,
1375                    schema_metadata_manager,
1376                    file_ref_manager,
1377                    time_provider,
1378                    partition_expr_fetcher,
1379                )
1380                .await?,
1381                config,
1382                wal_raw_entry_reader,
1383                scan_memory_tracker,
1384                #[cfg(feature = "enterprise")]
1385                extension_range_provider_factory: None,
1386            }),
1387        })
1388    }
1389
1390    /// Returns the purge scheduler.
1391    pub fn purge_scheduler(&self) -> &crate::schedule::scheduler::SchedulerRef {
1392        self.inner.workers.purge_scheduler()
1393    }
1394}
1395
1396#[cfg(test)]
1397mod tests {
1398    use std::time::Duration;
1399
1400    use super::*;
1401    use crate::sst::file::FileMeta;
1402
1403    #[test]
1404    fn test_is_valid_region_edit() {
1405        // Valid: has only "files_to_add"
1406        let edit = RegionEdit {
1407            files_to_add: vec![FileMeta::default()],
1408            files_to_remove: vec![],
1409            timestamp_ms: None,
1410            compaction_time_window: None,
1411            flushed_entry_id: None,
1412            flushed_sequence: None,
1413            committed_sequence: None,
1414        };
1415        assert!(is_valid_region_edit(&edit));
1416
1417        // Invalid: "files_to_add" and "files_to_remove" are both empty
1418        let edit = RegionEdit {
1419            files_to_add: vec![],
1420            files_to_remove: vec![],
1421            timestamp_ms: None,
1422            compaction_time_window: None,
1423            flushed_entry_id: None,
1424            flushed_sequence: None,
1425            committed_sequence: None,
1426        };
1427        assert!(!is_valid_region_edit(&edit));
1428
1429        // Valid: "files_to_remove" is not empty
1430        let edit = RegionEdit {
1431            files_to_add: vec![FileMeta::default()],
1432            files_to_remove: vec![FileMeta::default()],
1433            timestamp_ms: None,
1434            compaction_time_window: None,
1435            flushed_entry_id: None,
1436            flushed_sequence: None,
1437            committed_sequence: None,
1438        };
1439        assert!(is_valid_region_edit(&edit));
1440
1441        // Invalid: other fields are not all "None"s
1442        let edit = RegionEdit {
1443            files_to_add: vec![FileMeta::default()],
1444            files_to_remove: vec![],
1445            timestamp_ms: None,
1446            compaction_time_window: Some(Duration::from_secs(1)),
1447            flushed_entry_id: None,
1448            flushed_sequence: None,
1449            committed_sequence: None,
1450        };
1451        assert!(!is_valid_region_edit(&edit));
1452        let edit = RegionEdit {
1453            files_to_add: vec![FileMeta::default()],
1454            files_to_remove: vec![],
1455            timestamp_ms: None,
1456            compaction_time_window: None,
1457            flushed_entry_id: Some(1),
1458            flushed_sequence: None,
1459            committed_sequence: None,
1460        };
1461        assert!(!is_valid_region_edit(&edit));
1462        let edit = RegionEdit {
1463            files_to_add: vec![FileMeta::default()],
1464            files_to_remove: vec![],
1465            timestamp_ms: None,
1466            compaction_time_window: None,
1467            flushed_entry_id: None,
1468            flushed_sequence: Some(1),
1469            committed_sequence: None,
1470        };
1471        assert!(!is_valid_region_edit(&edit));
1472    }
1473}