Skip to main content

datanode/
region_server.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 catalog;
16
17use std::collections::HashMap;
18use std::fmt::Debug;
19use std::ops::Deref;
20use std::pin::Pin;
21use std::sync::atomic::{AtomicBool, Ordering};
22use std::sync::{Arc, RwLock};
23use std::task::{Context, Poll};
24use std::time::Duration;
25
26use api::region::RegionResponse;
27use api::v1::meta::TopicStat;
28use api::v1::region::remote_dyn_filter_request::Action;
29use api::v1::region::sync_request::ManifestInfo;
30use api::v1::region::{
31    ListMetadataRequest, RegionResponse as RegionResponseV1, RemoteDynFilterRequest, SyncRequest,
32    region_request,
33};
34use api::v1::{ResponseHeader, Status};
35use arrow_flight::{FlightData, Ticket};
36use async_trait::async_trait;
37use bytes::Bytes;
38use common_error::ext::{BoxedError, ErrorExt};
39use common_error::status_code::StatusCode;
40use common_meta::datanode::TopicStatsReporter;
41use common_query::OutputData;
42use common_query::request::QueryRequest;
43use common_recordbatch::adapter::RecordBatchMetrics;
44use common_recordbatch::{OrderOption, RecordBatch, RecordBatchStream, SendableRecordBatchStream};
45use common_runtime::Runtime;
46use common_telemetry::tracing::{self, info_span};
47use common_telemetry::tracing_context::{FutureExt, TracingContext};
48use common_telemetry::{debug, error, info, warn};
49use dashmap::DashMap;
50use datafusion::datasource::TableProvider;
51use datafusion_common::tree_node::TreeNode;
52use datatypes::schema::SchemaRef;
53use either::Either;
54use futures_util::Stream;
55use futures_util::future::try_join_all;
56use metric_engine::engine::MetricEngine;
57use mito2::engine::{MITO_ENGINE_NAME, MitoEngine};
58use prost::Message;
59use query::QueryEngineRef;
60pub use query::dummy_catalog::{
61    DummyCatalogList, DummyTableProviderFactory, TableProviderFactoryRef,
62};
63use query::options::should_collect_region_watermark_from_extensions;
64use serde_json;
65use servers::error::{
66    self as servers_error, ExecuteGrpcRequestSnafu, Result as ServerResult, SuspendedSnafu,
67};
68use servers::grpc::FlightCompression;
69use servers::grpc::flight::{FlightCraft, FlightRecordBatchStream, TonicStream};
70use servers::grpc::region_server::RegionServerHandler;
71use session::context::{QueryContext, QueryContextBuilder, QueryContextRef};
72use snafu::{OptionExt, ResultExt, ensure};
73use store_api::metric_engine_consts::{
74    FILE_ENGINE_NAME, LOGICAL_TABLE_METADATA_KEY, METRIC_ENGINE_NAME,
75};
76use store_api::region_engine::{
77    RegionEngineRef, RegionManifestInfo, RegionRole, RegionStatistic, RemapManifestsRequest,
78    RemapManifestsResponse, SetRegionRoleStateResponse, SettableRegionRoleState,
79    SyncRegionFromRequest,
80};
81use store_api::region_request::{
82    AffectedRows, BatchRegionDdlRequest, RegionCatchupRequest, RegionCloseRequest,
83    RegionOpenRequest, RegionRequest,
84};
85use store_api::storage::RegionId;
86use tokio::sync::{OwnedSemaphorePermit, Semaphore};
87use tokio::time::timeout;
88use tonic::{Request, Response, Result as TonicResult};
89
90use crate::error::{
91    self, BuildRegionRequestsSnafu, ConcurrentQueryLimiterClosedSnafu,
92    ConcurrentQueryLimiterTimeoutSnafu, DataFusionSnafu, DecodeLogicalPlanSnafu,
93    ExecuteLogicalPlanSnafu, FindLogicalRegionsSnafu, GetRegionMetadataSnafu,
94    HandleBatchDdlRequestSnafu, HandleBatchOpenRequestSnafu, HandleRegionRequestSnafu,
95    NewPlanDecoderSnafu, NotYetImplementedSnafu, RegionEngineNotFoundSnafu, RegionNotFoundSnafu,
96    RegionNotReadySnafu, Result, SerializeJsonSnafu, StopRegionEngineSnafu, UnexpectedSnafu,
97    UnsupportedOutputSnafu,
98};
99use crate::event_listener::RegionServerEventListenerRef;
100use crate::region_server::catalog::{NameAwareCatalogList, NameAwareDataSourceInjectorBuilder};
101
102#[derive(Clone)]
103pub struct RegionServer {
104    inner: Arc<RegionServerInner>,
105    flight_compression: FlightCompression,
106    suspend: Arc<AtomicBool>,
107}
108
109pub struct RegionStat {
110    pub region_id: RegionId,
111    pub engine: String,
112    pub role: RegionRole,
113}
114
115impl RegionServer {
116    pub fn new(
117        query_engine: QueryEngineRef,
118        runtime: Runtime,
119        event_listener: RegionServerEventListenerRef,
120        flight_compression: FlightCompression,
121    ) -> Self {
122        Self::with_table_provider(
123            query_engine,
124            runtime,
125            event_listener,
126            Arc::new(DummyTableProviderFactory),
127            0,
128            Duration::from_millis(0),
129            flight_compression,
130        )
131    }
132
133    pub fn with_table_provider(
134        query_engine: QueryEngineRef,
135        runtime: Runtime,
136        event_listener: RegionServerEventListenerRef,
137        table_provider_factory: TableProviderFactoryRef,
138        max_concurrent_queries: usize,
139        concurrent_query_limiter_timeout: Duration,
140        flight_compression: FlightCompression,
141    ) -> Self {
142        Self {
143            inner: Arc::new(RegionServerInner::new(
144                query_engine,
145                runtime,
146                event_listener,
147                table_provider_factory,
148                RegionServerParallelism::from_opts(
149                    max_concurrent_queries,
150                    concurrent_query_limiter_timeout,
151                ),
152            )),
153            flight_compression,
154            suspend: Arc::new(AtomicBool::new(false)),
155        }
156    }
157
158    /// Registers an engine.
159    pub fn register_engine(&mut self, engine: RegionEngineRef) {
160        self.inner.register_engine(engine);
161    }
162
163    /// Sets the topic stats.
164    pub fn set_topic_stats_reporter(&mut self, topic_stats_reporter: Box<dyn TopicStatsReporter>) {
165        self.inner.set_topic_stats_reporter(topic_stats_reporter);
166    }
167
168    /// Finds the region's engine by its id. If the region is not ready, returns `None`.
169    pub fn find_engine(&self, region_id: RegionId) -> Result<Option<RegionEngineRef>> {
170        match self.inner.get_engine(region_id, &RegionChange::None) {
171            Ok(CurrentEngine::Engine(engine)) => Ok(Some(engine)),
172            Ok(CurrentEngine::EarlyReturn(_)) => Ok(None),
173            Err(error::Error::RegionNotFound { .. }) => Ok(None),
174            Err(err) => Err(err),
175        }
176    }
177
178    /// Gets the MitoEngine if it's registered.
179    pub fn mito_engine(&self) -> Option<MitoEngine> {
180        if let Some(mito) = self.inner.mito_engine.read().unwrap().clone() {
181            Some(mito)
182        } else {
183            self.inner
184                .engines
185                .read()
186                .unwrap()
187                .get(MITO_ENGINE_NAME)
188                .cloned()
189                .and_then(|e| {
190                    let mito = e.as_any().downcast_ref::<MitoEngine>().cloned();
191                    if mito.is_none() {
192                        warn!("Mito engine not found in region server engines");
193                    }
194                    mito
195                })
196        }
197    }
198
199    #[tracing::instrument(skip_all)]
200    pub async fn handle_batch_open_requests(
201        &self,
202        parallelism: usize,
203        requests: Vec<(RegionId, RegionOpenRequest)>,
204        ignore_nonexistent_region: bool,
205    ) -> Result<Vec<RegionId>> {
206        self.inner
207            .handle_batch_open_requests(parallelism, requests, ignore_nonexistent_region)
208            .await
209    }
210
211    #[tracing::instrument(skip_all)]
212    pub async fn handle_batch_catchup_requests(
213        &self,
214        parallelism: usize,
215        requests: Vec<(RegionId, RegionCatchupRequest)>,
216    ) -> Result<Vec<(RegionId, std::result::Result<(), BoxedError>)>> {
217        self.inner
218            .handle_batch_catchup_requests(parallelism, requests)
219            .await
220    }
221
222    #[tracing::instrument(skip_all, fields(request_type = request.request_type()))]
223    pub async fn handle_request(
224        &self,
225        region_id: RegionId,
226        request: RegionRequest,
227    ) -> Result<RegionResponse> {
228        self.inner.handle_request(region_id, request).await
229    }
230
231    /// Returns a table provider for the region. Will set snapshot sequence if available in the context.
232    async fn table_provider(
233        &self,
234        region_id: RegionId,
235        ctx: Option<QueryContextRef>,
236    ) -> Result<Arc<dyn TableProvider>> {
237        let status = self
238            .inner
239            .region_map
240            .get(&region_id)
241            .context(RegionNotFoundSnafu { region_id })?
242            .clone();
243        ensure!(
244            matches!(status, RegionEngineWithStatus::Ready(_)),
245            RegionNotReadySnafu { region_id }
246        );
247
248        self.inner
249            .table_provider_factory
250            .create(region_id, status.into_engine(), ctx)
251            .await
252            .context(ExecuteLogicalPlanSnafu)
253    }
254
255    /// Handle reads from remote. They're often query requests received by our Arrow Flight service.
256    pub async fn handle_remote_read(
257        &self,
258        request: api::v1::region::QueryRequest,
259        query_ctx: QueryContextRef,
260    ) -> Result<SendableRecordBatchStream> {
261        let permit = if let Some(p) = &self.inner.parallelism {
262            Some(p.acquire().await?)
263        } else {
264            None
265        };
266
267        let region_id = RegionId::from_u64(request.region_id);
268        let catalog_list = Arc::new(NameAwareCatalogList::new(
269            self.clone(),
270            region_id,
271            query_ctx.clone(),
272        ));
273
274        if query_ctx.explain_verbose() {
275            common_telemetry::info!("Handle remote read for region: {}", region_id);
276        }
277
278        let decoder = self
279            .inner
280            .query_engine
281            .engine_context(query_ctx.clone())
282            .new_plan_decoder()
283            .context(NewPlanDecoderSnafu)?;
284
285        let plan = decoder
286            .decode(Bytes::from(request.plan), catalog_list, false)
287            .await
288            .context(DecodeLogicalPlanSnafu)?;
289
290        let stream = self
291            .inner
292            .handle_read(
293                QueryRequest {
294                    header: request.header,
295                    region_id,
296                    plan,
297                },
298                query_ctx.clone(),
299            )
300            .await?;
301        let stream = wrap_flow_region_watermark_stream(stream, region_id, &query_ctx);
302        Ok(maybe_guard_stream(stream, permit))
303    }
304
305    #[tracing::instrument(skip_all)]
306    pub async fn handle_read(&self, request: QueryRequest) -> Result<SendableRecordBatchStream> {
307        let permit = if let Some(p) = &self.inner.parallelism {
308            Some(p.acquire().await?)
309        } else {
310            None
311        };
312
313        let ctx = request.header.as_ref().map(|h| h.into());
314        let query_ctx = Arc::new(ctx.unwrap_or_else(|| QueryContextBuilder::default().build()));
315
316        let region_id = request.region_id;
317        let injector_builder = NameAwareDataSourceInjectorBuilder::from_plan(&request.plan)
318            .context(DataFusionSnafu)?;
319        let mut injector = injector_builder
320            .build(self, request.region_id, query_ctx.clone())
321            .await?;
322
323        let plan = request
324            .plan
325            .rewrite(&mut injector)
326            .context(DataFusionSnafu)?
327            .data;
328
329        let stream = self
330            .inner
331            .handle_read(QueryRequest { plan, ..request }, query_ctx.clone())
332            .await?;
333
334        let stream = wrap_flow_region_watermark_stream(stream, region_id, &query_ctx);
335        Ok(maybe_guard_stream(stream, permit))
336    }
337
338    /// Returns all opened and reportable regions.
339    ///
340    /// Notes: except all metrics regions.
341    pub fn reportable_regions(&self) -> Vec<RegionStat> {
342        self.inner
343            .region_map
344            .iter()
345            .filter_map(|e| {
346                let region_id = *e.key();
347                // Filters out any regions whose role equals None.
348                e.role(region_id).map(|role| RegionStat {
349                    region_id,
350                    engine: e.value().name().to_string(),
351                    role,
352                })
353            })
354            .collect()
355    }
356
357    /// Returns the reportable topics.
358    pub fn topic_stats(&self) -> Vec<TopicStat> {
359        let mut reporter = self.inner.topic_stats_reporter.write().unwrap();
360        let Some(reporter) = reporter.as_mut() else {
361            return vec![];
362        };
363        reporter
364            .reportable_topics()
365            .into_iter()
366            .map(|stat| TopicStat {
367                topic_name: stat.topic,
368                record_size: stat.record_size,
369                record_num: stat.record_num,
370                latest_entry_id: stat.latest_entry_id,
371            })
372            .collect()
373    }
374
375    pub fn is_region_leader(&self, region_id: RegionId) -> Option<bool> {
376        self.inner.region_map.get(&region_id).and_then(|engine| {
377            engine.role(region_id).map(|role| match role {
378                RegionRole::Follower => false,
379                RegionRole::Leader => true,
380                RegionRole::StagingLeader => true,
381                RegionRole::DowngradingLeader => true,
382            })
383        })
384    }
385
386    pub fn set_region_role(&self, region_id: RegionId, role: RegionRole) -> Result<()> {
387        let engine = self
388            .inner
389            .region_map
390            .get(&region_id)
391            .with_context(|| RegionNotFoundSnafu { region_id })?;
392        engine
393            .set_region_role(region_id, role)
394            .with_context(|_| HandleRegionRequestSnafu { region_id })
395    }
396
397    /// Set region role state gracefully.
398    ///
399    /// For [SettableRegionRoleState::Follower]:
400    /// After the call returns, the engine ensures that
401    /// no **further** write or flush operations will succeed in this region.
402    ///
403    /// For [SettableRegionRoleState::DowngradingLeader]:
404    /// After the call returns, the engine ensures that
405    /// no **further** write operations will succeed in this region.
406    pub async fn set_region_role_state_gracefully(
407        &self,
408        region_id: RegionId,
409        state: SettableRegionRoleState,
410    ) -> Result<SetRegionRoleStateResponse> {
411        match self.inner.region_map.get(&region_id) {
412            Some(engine) => Ok(engine
413                .set_region_role_state_gracefully(region_id, state)
414                .await
415                .with_context(|_| HandleRegionRequestSnafu { region_id })?),
416            None => Ok(SetRegionRoleStateResponse::NotFound),
417        }
418    }
419
420    pub fn runtime(&self) -> Runtime {
421        self.inner.runtime.clone()
422    }
423
424    pub fn region_statistic(&self, region_id: RegionId) -> Option<RegionStatistic> {
425        match self.inner.region_map.get(&region_id) {
426            Some(e) => e.region_statistic(region_id),
427            None => None,
428        }
429    }
430
431    /// Stop the region server.
432    pub async fn stop(&self) -> Result<()> {
433        self.inner.stop().await
434    }
435
436    #[cfg(test)]
437    /// Registers a region for test purpose.
438    pub(crate) fn register_test_region(&self, region_id: RegionId, engine: RegionEngineRef) {
439        {
440            let mut engines = self.inner.engines.write().unwrap();
441            if !engines.contains_key(engine.name()) {
442                debug!("Registering test engine: {}", engine.name());
443                engines.insert(engine.name().to_string(), engine.clone());
444            }
445        }
446
447        self.inner
448            .region_map
449            .insert(region_id, RegionEngineWithStatus::Ready(engine));
450    }
451
452    async fn handle_batch_ddl_requests(
453        &self,
454        request: region_request::Body,
455    ) -> Result<RegionResponse> {
456        // Safety: we have already checked the request type in `RegionServer::handle()`.
457        let batch_request = BatchRegionDdlRequest::try_from_request_body(request)
458            .context(BuildRegionRequestsSnafu)?
459            .unwrap();
460        let tracing_context = TracingContext::from_current_span();
461
462        let span = tracing_context.attach(info_span!("RegionServer::handle_batch_ddl_requests"));
463        self.inner
464            .handle_batch_request(batch_request)
465            .trace(span)
466            .await
467    }
468
469    async fn handle_requests_in_parallel(
470        &self,
471        request: region_request::Body,
472    ) -> Result<RegionResponse> {
473        let requests =
474            RegionRequest::try_from_request_body(request).context(BuildRegionRequestsSnafu)?;
475
476        // Try to optimize batch Put requests for metric engine
477        // Returns either Some(response) or None(requests_back)
478        match self.try_handle_metric_batch_puts(requests).await? {
479            Either::Left(response) => Ok(response),
480            Either::Right(requests) => {
481                // Fallback: original parallel processing
482                let tracing_context = TracingContext::from_current_span();
483                let join_tasks =
484                    requests
485                        .into_iter()
486                        .map(|(region_id, req): (RegionId, RegionRequest)| {
487                            let self_to_move = self;
488                            let span = tracing_context.attach(info_span!(
489                                "RegionServer::handle_region_request",
490                                region_id = region_id.to_string()
491                            ));
492                            async move {
493                                self_to_move
494                                    .handle_request(region_id, req)
495                                    .trace(span)
496                                    .await
497                            }
498                        });
499
500                let results = try_join_all(join_tasks).await?;
501                let mut affected_rows = 0;
502                let mut extensions = HashMap::new();
503                for result in results {
504                    affected_rows += result.affected_rows;
505                    extensions.extend(result.extensions);
506                }
507
508                Ok(RegionResponse {
509                    affected_rows,
510                    extensions,
511                    metadata: Vec::new(),
512                })
513            }
514        }
515    }
516
517    async fn handle_requests_in_serial(
518        &self,
519        request: region_request::Body,
520    ) -> Result<RegionResponse> {
521        let requests =
522            RegionRequest::try_from_request_body(request).context(BuildRegionRequestsSnafu)?;
523        let tracing_context = TracingContext::from_current_span();
524
525        let mut affected_rows = 0;
526        let mut extensions = HashMap::new();
527        for (region_id, req) in requests {
528            let span = tracing_context.attach(info_span!(
529                "RegionServer::handle_region_request",
530                region_id = region_id.to_string()
531            ));
532            let result = self.handle_request(region_id, req).trace(span).await?;
533
534            affected_rows += result.affected_rows;
535            extensions.extend(result.extensions);
536        }
537
538        Ok(RegionResponse {
539            affected_rows,
540            extensions,
541            metadata: Vec::new(),
542        })
543    }
544
545    /// Attempts to optimize batch Put requests for metric engine.
546    ///
547    /// Returns Either::Left(response) if optimization succeeded,
548    /// or Either::Right(original_requests) to fall back to parallel processing.
549    ///
550    /// This avoids cloning requests when optimization cannot be applied.
551    async fn try_handle_metric_batch_puts(
552        &self,
553        requests: Vec<(RegionId, RegionRequest)>,
554    ) -> Result<Either<RegionResponse, Vec<(RegionId, RegionRequest)>>> {
555        if requests.is_empty() {
556            return Ok(Either::Right(requests));
557        }
558
559        // Quick check: verify first request is Put and is metric engine
560        if !matches!(requests[0].1, RegionRequest::Put(_)) {
561            return Ok(Either::Right(requests));
562        }
563        let first_region_id = requests[0].0;
564        let request_type = requests[0].1.request_type();
565
566        // SAFETY: If the first request belongs to metric engine, then ALL requests
567        // in this batch are guaranteed to belong to metric engine. This invariant
568        // is maintained by the request batching logic upstream.
569        let engine = match self
570            .inner
571            .get_engine(first_region_id, &RegionChange::None)?
572        {
573            CurrentEngine::Engine(e) => e,
574            _ => return Ok(Either::Right(requests)),
575        };
576
577        if engine.name() != METRIC_ENGINE_NAME {
578            return Ok(Either::Right(requests));
579        }
580
581        // Check if ALL requests are Put (now we know it's worth checking)
582        let mut all_puts = true;
583        for (_, req) in &requests {
584            if !matches!(req, RegionRequest::Put(_)) {
585                all_puts = false;
586                break;
587            }
588        }
589
590        if !all_puts {
591            return Ok(Either::Right(requests));
592        }
593
594        // Now extract Put requests by consuming ownership (zero clone!)
595        let put_requests = requests.into_iter().map(|(region_id, req)| {
596            if let RegionRequest::Put(put) = req {
597                (region_id, put)
598            } else {
599                unreachable!("Already checked all are Put")
600            }
601        });
602
603        // Downcast to MetricEngine and call batch API
604        let metric_engine =
605            engine
606                .as_any()
607                .downcast_ref::<MetricEngine>()
608                .context(UnexpectedSnafu {
609                    violated: "Failed to downcast to MetricEngine",
610                })?;
611
612        let tracing_context = TracingContext::from_current_span();
613        let batch_size = put_requests.len();
614        let span = tracing_context.attach(info_span!(
615            "RegionServer::handle_metric_batch_puts",
616            batch_size = batch_size,
617        ));
618        let result = metric_engine
619            .put_regions_batch(put_requests)
620            .trace(span)
621            .await
622            .map_err(BoxedError::new)
623            .context(HandleRegionRequestSnafu {
624                region_id: first_region_id,
625            });
626
627        match result {
628            Ok(total_affected) => {
629                crate::metrics::REGION_CHANGED_ROW_COUNT
630                    .with_label_values(&[request_type])
631                    .inc_by(total_affected as u64);
632                Ok(Either::Left(RegionResponse::new(total_affected)))
633            }
634            Err(err) => {
635                crate::metrics::REGION_SERVER_INSERT_FAIL_COUNT
636                    .with_label_values(&[request_type])
637                    .inc_by(batch_size as u64);
638                Err(err)
639            }
640        }
641    }
642
643    async fn handle_sync_region_request(&self, request: &SyncRequest) -> Result<RegionResponse> {
644        let region_id = RegionId::from_u64(request.region_id);
645        let manifest_info = request
646            .manifest_info
647            .context(error::MissingRequiredFieldSnafu {
648                name: "manifest_info",
649            })?;
650
651        let manifest_info = match manifest_info {
652            ManifestInfo::MitoManifestInfo(info) => {
653                RegionManifestInfo::mito(info.data_manifest_version, 0, 0)
654            }
655            ManifestInfo::MetricManifestInfo(info) => RegionManifestInfo::metric(
656                info.data_manifest_version,
657                0,
658                info.metadata_manifest_version,
659                0,
660            ),
661        };
662
663        let tracing_context = TracingContext::from_current_span();
664        let span = tracing_context.attach(info_span!("RegionServer::handle_sync_region_request"));
665
666        self.sync_region(
667            region_id,
668            SyncRegionFromRequest::from_manifest(manifest_info),
669        )
670        .trace(span)
671        .await
672        .map(|_| RegionResponse::new(AffectedRows::default()))
673    }
674
675    /// Handles the ListMetadata request and retrieves metadata for specified regions.
676    ///
677    /// Returns the results as a JSON-serialized list in the [RegionResponse]. It serializes
678    /// non-existing regions as `null`.
679    #[tracing::instrument(skip_all)]
680    async fn handle_list_metadata_request(
681        &self,
682        request: &ListMetadataRequest,
683    ) -> Result<RegionResponse> {
684        let mut region_metadatas = Vec::new();
685        // Collect metadata for each region
686        for region_id in &request.region_ids {
687            let region_id = RegionId::from_u64(*region_id);
688            // Get the engine.
689            let Some(engine) = self.find_engine(region_id)? else {
690                region_metadatas.push(None);
691                continue;
692            };
693
694            match engine.get_metadata(region_id).await {
695                Ok(metadata) => region_metadatas.push(Some(metadata)),
696                Err(err) => {
697                    if err.status_code() == StatusCode::RegionNotFound {
698                        region_metadatas.push(None);
699                    } else {
700                        Err(err).with_context(|_| GetRegionMetadataSnafu {
701                            engine: engine.name(),
702                            region_id,
703                        })?;
704                    }
705                }
706            }
707        }
708
709        // Serialize metadata to JSON
710        let json_result = serde_json::to_vec(&region_metadatas).context(SerializeJsonSnafu)?;
711
712        let response = RegionResponse::from_metadata(json_result);
713
714        Ok(response)
715    }
716
717    async fn handle_remote_dyn_filter_request(
718        &self,
719        request: &RemoteDynFilterRequest,
720    ) -> Result<RegionResponse> {
721        if request.query_id.is_empty() {
722            return error::MissingRequiredFieldSnafu { name: "query_id" }.fail();
723        }
724
725        match request
726            .action
727            .as_ref()
728            .context(error::MissingRequiredFieldSnafu { name: "action" })?
729        {
730            Action::Update(update) => {
731                self.handle_remote_dyn_filter_update(&request.query_id, update)
732                    .await
733            }
734            Action::Unregister(unregister) => {
735                self.handle_remote_dyn_filter_unregister(&request.query_id, unregister)
736                    .await
737            }
738        }
739    }
740
741    async fn handle_remote_dyn_filter_update(
742        &self,
743        query_id: &str,
744        request: &api::v1::region::RemoteDynFilterUpdate,
745    ) -> Result<RegionResponse> {
746        if request.filter_id.is_empty() {
747            return error::MissingRequiredFieldSnafu { name: "filter_id" }.fail();
748        }
749
750        if request.payload.is_empty() {
751            return error::MissingRequiredFieldSnafu { name: "payload" }.fail();
752        }
753
754        NotYetImplementedSnafu {
755            what: format!(
756                "remote dyn filter update unary RPC placeholder for query_id {query_id}, filter_id {}",
757                request.filter_id
758            ),
759        }
760        .fail()
761    }
762
763    async fn handle_remote_dyn_filter_unregister(
764        &self,
765        query_id: &str,
766        request: &api::v1::region::RemoteDynFilterUnregister,
767    ) -> Result<RegionResponse> {
768        if request.filter_id.is_empty() {
769            return error::MissingRequiredFieldSnafu { name: "filter_id" }.fail();
770        }
771
772        NotYetImplementedSnafu {
773            what: format!(
774                "remote dyn filter unregister unary RPC placeholder for query_id {query_id}, filter_id {}",
775                request.filter_id
776            ),
777        }
778        .fail()
779    }
780
781    /// Sync region manifest and registers new opened logical regions.
782    pub async fn sync_region(
783        &self,
784        region_id: RegionId,
785        request: SyncRegionFromRequest,
786    ) -> Result<()> {
787        let engine = match self.inner.get_engine(region_id, &RegionChange::None)? {
788            CurrentEngine::Engine(engine) => engine,
789            _ => {
790                return UnexpectedSnafu {
791                    violated: "unexpected EarlyReturn engine status for a ready region",
792                }
793                .fail();
794            }
795        };
796
797        self.inner
798            .handle_sync_region(&engine, region_id, request)
799            .await
800    }
801
802    /// Remaps manifests from old regions to new regions.
803    pub async fn remap_manifests(
804        &self,
805        request: RemapManifestsRequest,
806    ) -> Result<RemapManifestsResponse> {
807        let region_id = request.region_id;
808        let engine = match self.inner.get_engine(region_id, &RegionChange::None)? {
809            CurrentEngine::Engine(engine) => engine,
810            _ => {
811                return UnexpectedSnafu {
812                    violated: "unexpected EarlyReturn engine status for a ready region",
813                }
814                .fail();
815            }
816        };
817
818        engine
819            .remap_manifests(request)
820            .await
821            .with_context(|_| HandleRegionRequestSnafu { region_id })
822    }
823
824    fn is_suspended(&self) -> bool {
825        self.suspend.load(Ordering::Relaxed)
826    }
827
828    pub(crate) fn suspend_state(&self) -> Arc<AtomicBool> {
829        self.suspend.clone()
830    }
831}
832
833fn wrap_flow_region_watermark_stream(
834    stream: SendableRecordBatchStream,
835    region_id: RegionId,
836    query_ctx: &QueryContextRef,
837) -> SendableRecordBatchStream {
838    if should_collect_region_watermark_from_extensions(&query_ctx.extensions())
839        && let Some(seq) = query_ctx.get_snapshot(region_id.as_u64())
840    {
841        Box::pin(RegionWatermarkStream::new(stream, region_id, seq)) as SendableRecordBatchStream
842    } else {
843        stream
844    }
845}
846
847/// Wraps a region read stream so terminal metrics can carry the scan-open watermark.
848struct RegionWatermarkStream {
849    stream: SendableRecordBatchStream,
850    region_id: u64,
851    snapshot_seq: u64,
852    finished: bool,
853}
854
855impl RegionWatermarkStream {
856    fn new(stream: SendableRecordBatchStream, region_id: RegionId, snapshot_seq: u64) -> Self {
857        Self {
858            stream,
859            region_id: region_id.as_u64(),
860            snapshot_seq,
861            finished: false,
862        }
863    }
864
865    fn merged_metrics(&self, mut metrics: RecordBatchMetrics) -> RecordBatchMetrics {
866        if metrics
867            .region_watermarks
868            .iter()
869            .any(|entry| entry.region_id == self.region_id)
870        {
871            return metrics;
872        }
873
874        metrics
875            .region_watermarks
876            .push(common_recordbatch::adapter::RegionWatermarkEntry {
877                region_id: self.region_id,
878                watermark: Some(self.snapshot_seq),
879            });
880        metrics
881    }
882}
883
884impl RecordBatchStream for RegionWatermarkStream {
885    fn name(&self) -> &str {
886        self.stream.name()
887    }
888
889    fn schema(&self) -> datatypes::schema::SchemaRef {
890        self.stream.schema()
891    }
892
893    fn output_ordering(&self) -> Option<&[OrderOption]> {
894        self.stream.output_ordering()
895    }
896
897    fn metrics(&self) -> Option<RecordBatchMetrics> {
898        let base = self.stream.metrics();
899        if !self.finished {
900            return base;
901        }
902
903        Some(self.merged_metrics(base.unwrap_or_default()))
904    }
905}
906
907impl Stream for RegionWatermarkStream {
908    type Item = common_recordbatch::error::Result<RecordBatch>;
909
910    fn size_hint(&self) -> (usize, Option<usize>) {
911        self.stream.size_hint()
912    }
913
914    fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
915        match Pin::new(&mut self.stream).poll_next(cx) {
916            Poll::Ready(None) => {
917                self.finished = true;
918                Poll::Ready(None)
919            }
920            other => other,
921        }
922    }
923}
924
925#[async_trait]
926impl RegionServerHandler for RegionServer {
927    async fn handle(&self, request: region_request::Body) -> ServerResult<RegionResponseV1> {
928        let failed_requests_cnt = crate::metrics::REGION_SERVER_REQUEST_FAILURE_COUNT
929            .with_label_values(&[request.as_ref()]);
930        let response = match &request {
931            region_request::Body::Creates(_)
932            | region_request::Body::Drops(_)
933            | region_request::Body::Alters(_) => self.handle_batch_ddl_requests(request).await,
934            region_request::Body::Inserts(_) | region_request::Body::Deletes(_) => {
935                self.handle_requests_in_parallel(request).await
936            }
937            region_request::Body::Sync(sync_request) => {
938                self.handle_sync_region_request(sync_request).await
939            }
940            region_request::Body::ListMetadata(list_metadata_request) => {
941                self.handle_list_metadata_request(list_metadata_request)
942                    .await
943            }
944            region_request::Body::RemoteDynFilter(remote_dyn_filter_request) => {
945                self.handle_remote_dyn_filter_request(remote_dyn_filter_request)
946                    .await
947            }
948            _ => self.handle_requests_in_serial(request).await,
949        }
950        .map_err(BoxedError::new)
951        .inspect_err(|_| {
952            failed_requests_cnt.inc();
953        })
954        .context(ExecuteGrpcRequestSnafu)?;
955
956        Ok(RegionResponseV1 {
957            header: Some(ResponseHeader {
958                status: Some(Status {
959                    status_code: StatusCode::Success as _,
960                    ..Default::default()
961                }),
962            }),
963            affected_rows: response.affected_rows as _,
964            extensions: response.extensions,
965            metadata: response.metadata,
966        })
967    }
968}
969
970#[async_trait]
971impl FlightCraft for RegionServer {
972    async fn do_get(
973        &self,
974        request: Request<Ticket>,
975    ) -> TonicResult<Response<TonicStream<FlightData>>> {
976        ensure!(!self.is_suspended(), SuspendedSnafu);
977
978        let ticket = request.into_inner().ticket;
979        let request = api::v1::region::QueryRequest::decode(ticket.as_ref())
980            .context(servers_error::InvalidFlightTicketSnafu)?;
981        let tracing_context = request
982            .header
983            .as_ref()
984            .map(|h| TracingContext::from_w3c(&h.tracing_context))
985            .unwrap_or_default();
986        let query_ctx = request
987            .header
988            .as_ref()
989            .map(|h| Arc::new(QueryContext::from(h)))
990            .unwrap_or(QueryContext::arc());
991
992        let result = self
993            .handle_remote_read(request, query_ctx.clone())
994            .trace(tracing_context.attach(info_span!("RegionServer::handle_read")))
995            .await?;
996
997        let stream = Box::pin(FlightRecordBatchStream::new(
998            result,
999            tracing_context,
1000            self.flight_compression,
1001            query_ctx,
1002        ));
1003        Ok(Response::new(stream))
1004    }
1005}
1006
1007#[derive(Clone)]
1008enum RegionEngineWithStatus {
1009    // An opening, or creating region.
1010    Registering(RegionEngineRef),
1011    // A closing, or dropping region.
1012    Deregistering(RegionEngineRef),
1013    // A ready region.
1014    Ready(RegionEngineRef),
1015}
1016
1017impl RegionEngineWithStatus {
1018    /// Returns [RegionEngineRef].
1019    pub fn into_engine(self) -> RegionEngineRef {
1020        match self {
1021            RegionEngineWithStatus::Registering(engine) => engine,
1022            RegionEngineWithStatus::Deregistering(engine) => engine,
1023            RegionEngineWithStatus::Ready(engine) => engine,
1024        }
1025    }
1026}
1027
1028impl Deref for RegionEngineWithStatus {
1029    type Target = RegionEngineRef;
1030
1031    fn deref(&self) -> &Self::Target {
1032        match self {
1033            RegionEngineWithStatus::Registering(engine) => engine,
1034            RegionEngineWithStatus::Deregistering(engine) => engine,
1035            RegionEngineWithStatus::Ready(engine) => engine,
1036        }
1037    }
1038}
1039
1040struct RegionServerInner {
1041    engines: RwLock<HashMap<String, RegionEngineRef>>,
1042    region_map: DashMap<RegionId, RegionEngineWithStatus>,
1043    query_engine: QueryEngineRef,
1044    runtime: Runtime,
1045    event_listener: RegionServerEventListenerRef,
1046    table_provider_factory: TableProviderFactoryRef,
1047    /// The number of queries allowed to be executed at the same time.
1048    /// Act as last line of defense on datanode to prevent query overloading.
1049    parallelism: Option<RegionServerParallelism>,
1050    /// The topic stats reporter.
1051    topic_stats_reporter: RwLock<Option<Box<dyn TopicStatsReporter>>>,
1052    /// HACK(zhongzc): Direct MitoEngine handle for diagnostics. This couples the
1053    /// server with a concrete engine; acceptable for now to fetch Mito-specific
1054    /// info (e.g., list SSTs). Consider a diagnostics trait later.
1055    mito_engine: RwLock<Option<MitoEngine>>,
1056}
1057
1058struct RegionServerParallelism {
1059    semaphore: Arc<Semaphore>,
1060    timeout: Duration,
1061}
1062
1063impl RegionServerParallelism {
1064    pub fn from_opts(
1065        max_concurrent_queries: usize,
1066        concurrent_query_limiter_timeout: Duration,
1067    ) -> Option<Self> {
1068        if max_concurrent_queries == 0 {
1069            return None;
1070        }
1071        Some(RegionServerParallelism {
1072            semaphore: Arc::new(Semaphore::new(max_concurrent_queries)),
1073            timeout: concurrent_query_limiter_timeout,
1074        })
1075    }
1076
1077    pub async fn acquire(&self) -> Result<OwnedSemaphorePermit> {
1078        timeout(self.timeout, self.semaphore.clone().acquire_owned())
1079            .await
1080            .context(ConcurrentQueryLimiterTimeoutSnafu)?
1081            .context(ConcurrentQueryLimiterClosedSnafu)
1082    }
1083}
1084
1085/// Wraps a record batch stream and holds a concurrency permit until the stream is
1086/// fully consumed (dropped), so `max_concurrent_queries` bounds the number of
1087/// in-flight read streams, not just query planning.
1088struct PermitGuardedStream {
1089    inner: SendableRecordBatchStream,
1090    _permit: OwnedSemaphorePermit,
1091}
1092
1093impl RecordBatchStream for PermitGuardedStream {
1094    fn name(&self) -> &str {
1095        self.inner.name()
1096    }
1097
1098    fn schema(&self) -> SchemaRef {
1099        self.inner.schema()
1100    }
1101
1102    fn output_ordering(&self) -> Option<&[OrderOption]> {
1103        self.inner.output_ordering()
1104    }
1105
1106    fn metrics(&self) -> Option<RecordBatchMetrics> {
1107        self.inner.metrics()
1108    }
1109}
1110
1111impl Stream for PermitGuardedStream {
1112    type Item = common_recordbatch::error::Result<RecordBatch>;
1113
1114    fn poll_next(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Option<Self::Item>> {
1115        self.inner.as_mut().poll_next(cx)
1116    }
1117}
1118
1119/// Wraps `stream` so it holds `permit` until fully consumed. Returns `stream`
1120/// unchanged when no permit was acquired (limiter disabled).
1121fn maybe_guard_stream(
1122    stream: SendableRecordBatchStream,
1123    permit: Option<OwnedSemaphorePermit>,
1124) -> SendableRecordBatchStream {
1125    match permit {
1126        Some(permit) => Box::pin(PermitGuardedStream {
1127            inner: stream,
1128            _permit: permit,
1129        }),
1130        None => stream,
1131    }
1132}
1133
1134enum CurrentEngine {
1135    Engine(RegionEngineRef),
1136    EarlyReturn(AffectedRows),
1137}
1138
1139impl Debug for CurrentEngine {
1140    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
1141        match self {
1142            CurrentEngine::Engine(engine) => f
1143                .debug_struct("CurrentEngine")
1144                .field("engine", &engine.name())
1145                .finish(),
1146            CurrentEngine::EarlyReturn(rows) => f
1147                .debug_struct("CurrentEngine")
1148                .field("return", rows)
1149                .finish(),
1150        }
1151    }
1152}
1153
1154impl RegionServerInner {
1155    pub fn new(
1156        query_engine: QueryEngineRef,
1157        runtime: Runtime,
1158        event_listener: RegionServerEventListenerRef,
1159        table_provider_factory: TableProviderFactoryRef,
1160        parallelism: Option<RegionServerParallelism>,
1161    ) -> Self {
1162        Self {
1163            engines: RwLock::new(HashMap::new()),
1164            region_map: DashMap::new(),
1165            query_engine,
1166            runtime,
1167            event_listener,
1168            table_provider_factory,
1169            parallelism,
1170            topic_stats_reporter: RwLock::new(None),
1171            mito_engine: RwLock::new(None),
1172        }
1173    }
1174
1175    pub fn register_engine(&self, engine: RegionEngineRef) {
1176        let engine_name = engine.name();
1177        if engine_name == MITO_ENGINE_NAME
1178            && let Some(mito_engine) = engine.as_any().downcast_ref::<MitoEngine>()
1179        {
1180            *self.mito_engine.write().unwrap() = Some(mito_engine.clone());
1181        }
1182
1183        info!("Region Engine {engine_name} is registered");
1184        self.engines
1185            .write()
1186            .unwrap()
1187            .insert(engine_name.to_string(), engine);
1188    }
1189
1190    pub fn set_topic_stats_reporter(&self, topic_stats_reporter: Box<dyn TopicStatsReporter>) {
1191        info!("Set topic stats reporter");
1192        *self.topic_stats_reporter.write().unwrap() = Some(topic_stats_reporter);
1193    }
1194
1195    fn get_engine(
1196        &self,
1197        region_id: RegionId,
1198        region_change: &RegionChange,
1199    ) -> Result<CurrentEngine> {
1200        let current_region_status = self.region_map.get(&region_id);
1201
1202        let engine = match region_change {
1203            RegionChange::Register(attribute) => match current_region_status {
1204                Some(status) => match status.clone() {
1205                    RegionEngineWithStatus::Registering(engine) => engine,
1206                    RegionEngineWithStatus::Deregistering(_) => {
1207                        return error::RegionBusySnafu { region_id }.fail();
1208                    }
1209                    RegionEngineWithStatus::Ready(_) => status.clone().into_engine(),
1210                },
1211                _ => self
1212                    .engines
1213                    .read()
1214                    .unwrap()
1215                    .get(attribute.engine())
1216                    .with_context(|| RegionEngineNotFoundSnafu {
1217                        name: attribute.engine(),
1218                    })?
1219                    .clone(),
1220            },
1221            RegionChange::Deregisters => match current_region_status {
1222                Some(status) => match status.clone() {
1223                    RegionEngineWithStatus::Registering(_) => {
1224                        return error::RegionBusySnafu { region_id }.fail();
1225                    }
1226                    RegionEngineWithStatus::Deregistering(_) => {
1227                        return Ok(CurrentEngine::EarlyReturn(0));
1228                    }
1229                    RegionEngineWithStatus::Ready(_) => status.clone().into_engine(),
1230                },
1231                None => return Ok(CurrentEngine::EarlyReturn(0)),
1232            },
1233            RegionChange::None | RegionChange::Catchup | RegionChange::Ingest => {
1234                match current_region_status {
1235                    Some(status) => match status.clone() {
1236                        RegionEngineWithStatus::Registering(_) => {
1237                            return error::RegionNotReadySnafu { region_id }.fail();
1238                        }
1239                        RegionEngineWithStatus::Deregistering(_) => {
1240                            return error::RegionNotFoundSnafu { region_id }.fail();
1241                        }
1242                        RegionEngineWithStatus::Ready(engine) => engine,
1243                    },
1244                    None => return error::RegionNotFoundSnafu { region_id }.fail(),
1245                }
1246            }
1247        };
1248
1249        Ok(CurrentEngine::Engine(engine))
1250    }
1251
1252    async fn handle_batch_open_requests_inner(
1253        &self,
1254        engine: RegionEngineRef,
1255        parallelism: usize,
1256        requests: Vec<(RegionId, RegionOpenRequest)>,
1257        ignore_nonexistent_region: bool,
1258    ) -> Result<Vec<RegionId>> {
1259        let region_changes = requests
1260            .iter()
1261            .map(|(region_id, open)| {
1262                let attribute = parse_region_attribute(&open.engine, &open.options)?;
1263                Ok((*region_id, RegionChange::Register(attribute)))
1264            })
1265            .collect::<Result<HashMap<_, _>>>()?;
1266
1267        for (&region_id, region_change) in &region_changes {
1268            self.set_region_status_not_ready(region_id, &engine, region_change)
1269        }
1270
1271        let mut open_regions = Vec::with_capacity(requests.len());
1272        let mut errors = vec![];
1273        match engine
1274            .handle_batch_open_requests(parallelism, requests)
1275            .await
1276            .with_context(|_| HandleBatchOpenRequestSnafu)
1277        {
1278            Ok(results) => {
1279                for (region_id, result) in results {
1280                    let region_change = &region_changes[&region_id];
1281                    match result {
1282                        Ok(_) => {
1283                            if let Err(e) = self
1284                                .set_region_status_ready(region_id, engine.clone(), *region_change)
1285                                .await
1286                            {
1287                                error!(e; "Failed to set region to ready: {}", region_id);
1288                                errors.push(BoxedError::new(e));
1289                            } else {
1290                                open_regions.push(region_id)
1291                            }
1292                        }
1293                        Err(e) => {
1294                            self.unset_region_status(region_id, &engine, *region_change);
1295                            if e.status_code() == StatusCode::RegionNotFound
1296                                && ignore_nonexistent_region
1297                            {
1298                                warn!("Region {} not found, ignore it, source: {:?}", region_id, e);
1299                            } else {
1300                                error!(e; "Failed to open region: {}", region_id);
1301                                errors.push(e);
1302                            }
1303                        }
1304                    }
1305                }
1306            }
1307            Err(e) => {
1308                for (&region_id, region_change) in &region_changes {
1309                    self.unset_region_status(region_id, &engine, *region_change);
1310                }
1311                error!(e; "Failed to open batch regions");
1312                errors.push(BoxedError::new(e));
1313            }
1314        }
1315
1316        if !errors.is_empty() {
1317            return error::UnexpectedSnafu {
1318                // Returns the first error.
1319                violated: format!("Failed to open batch regions: {:?}", errors[0]),
1320            }
1321            .fail();
1322        }
1323
1324        Ok(open_regions)
1325    }
1326
1327    pub async fn handle_batch_open_requests(
1328        &self,
1329        parallelism: usize,
1330        requests: Vec<(RegionId, RegionOpenRequest)>,
1331        ignore_nonexistent_region: bool,
1332    ) -> Result<Vec<RegionId>> {
1333        let mut engine_grouped_requests: HashMap<String, Vec<_>> =
1334            HashMap::with_capacity(requests.len());
1335        for (region_id, request) in requests {
1336            if let Some(requests) = engine_grouped_requests.get_mut(&request.engine) {
1337                requests.push((region_id, request));
1338            } else {
1339                engine_grouped_requests.insert(request.engine.clone(), vec![(region_id, request)]);
1340            }
1341        }
1342
1343        let mut results = Vec::with_capacity(engine_grouped_requests.keys().len());
1344        for (engine, requests) in engine_grouped_requests {
1345            let engine = self
1346                .engines
1347                .read()
1348                .unwrap()
1349                .get(&engine)
1350                .with_context(|| RegionEngineNotFoundSnafu { name: &engine })?
1351                .clone();
1352            results.push(
1353                self.handle_batch_open_requests_inner(
1354                    engine,
1355                    parallelism,
1356                    requests,
1357                    ignore_nonexistent_region,
1358                )
1359                .await,
1360            )
1361        }
1362
1363        Ok(results
1364            .into_iter()
1365            .collect::<Result<Vec<_>>>()?
1366            .into_iter()
1367            .flatten()
1368            .collect::<Vec<_>>())
1369    }
1370
1371    pub async fn handle_batch_catchup_requests_inner(
1372        &self,
1373        engine: RegionEngineRef,
1374        parallelism: usize,
1375        requests: Vec<(RegionId, RegionCatchupRequest)>,
1376    ) -> Result<Vec<(RegionId, std::result::Result<(), BoxedError>)>> {
1377        for (region_id, _) in &requests {
1378            self.set_region_status_not_ready(*region_id, &engine, &RegionChange::Catchup);
1379        }
1380        let region_ids = requests
1381            .iter()
1382            .map(|(region_id, _)| *region_id)
1383            .collect::<Vec<_>>();
1384        let mut responses = Vec::with_capacity(requests.len());
1385        match engine
1386            .handle_batch_catchup_requests(parallelism, requests)
1387            .await
1388        {
1389            Ok(results) => {
1390                for (region_id, result) in results {
1391                    match result {
1392                        Ok(_) => {
1393                            if let Err(e) = self
1394                                .set_region_status_ready(
1395                                    region_id,
1396                                    engine.clone(),
1397                                    RegionChange::Catchup,
1398                                )
1399                                .await
1400                            {
1401                                error!(e; "Failed to set region to ready: {}", region_id);
1402                                responses.push((region_id, Err(BoxedError::new(e))));
1403                            } else {
1404                                responses.push((region_id, Ok(())));
1405                            }
1406                        }
1407                        Err(e) => {
1408                            self.unset_region_status(region_id, &engine, RegionChange::Catchup);
1409                            error!(e; "Failed to catchup region: {}", region_id);
1410                            responses.push((region_id, Err(e)));
1411                        }
1412                    }
1413                }
1414            }
1415            Err(e) => {
1416                for region_id in region_ids {
1417                    self.unset_region_status(region_id, &engine, RegionChange::Catchup);
1418                }
1419                error!(e; "Failed to catchup batch regions");
1420                return error::UnexpectedSnafu {
1421                    violated: format!("Failed to catchup batch regions: {:?}", e),
1422                }
1423                .fail();
1424            }
1425        }
1426
1427        Ok(responses)
1428    }
1429
1430    pub async fn handle_batch_catchup_requests(
1431        &self,
1432        parallelism: usize,
1433        requests: Vec<(RegionId, RegionCatchupRequest)>,
1434    ) -> Result<Vec<(RegionId, std::result::Result<(), BoxedError>)>> {
1435        let mut engine_grouped_requests: HashMap<String, Vec<_>> = HashMap::new();
1436
1437        let mut responses = Vec::with_capacity(requests.len());
1438        for (region_id, request) in requests {
1439            if let Ok(engine) = self.get_engine(region_id, &RegionChange::Catchup) {
1440                match engine {
1441                    CurrentEngine::Engine(engine) => {
1442                        engine_grouped_requests
1443                            .entry(engine.name().to_string())
1444                            .or_default()
1445                            .push((region_id, request));
1446                    }
1447                    CurrentEngine::EarlyReturn(_) => {
1448                        return error::UnexpectedSnafu {
1449                            violated: format!("Unexpected engine type for region {}", region_id),
1450                        }
1451                        .fail();
1452                    }
1453                }
1454            } else {
1455                responses.push((
1456                    region_id,
1457                    Err(BoxedError::new(
1458                        error::RegionNotFoundSnafu { region_id }.build(),
1459                    )),
1460                ));
1461            }
1462        }
1463
1464        for (engine, requests) in engine_grouped_requests {
1465            let engine = self
1466                .engines
1467                .read()
1468                .unwrap()
1469                .get(&engine)
1470                .with_context(|| RegionEngineNotFoundSnafu { name: &engine })?
1471                .clone();
1472            responses.extend(
1473                self.handle_batch_catchup_requests_inner(engine, parallelism, requests)
1474                    .await?,
1475            );
1476        }
1477
1478        Ok(responses)
1479    }
1480
1481    // Handle requests in batch.
1482    //
1483    // limitation: all create requests must be in the same engine.
1484    pub async fn handle_batch_request(
1485        &self,
1486        batch_request: BatchRegionDdlRequest,
1487    ) -> Result<RegionResponse> {
1488        let region_changes = match &batch_request {
1489            BatchRegionDdlRequest::Create(requests) => requests
1490                .iter()
1491                .map(|(region_id, create)| {
1492                    let attribute = parse_region_attribute(&create.engine, &create.options)?;
1493                    Ok((*region_id, RegionChange::Register(attribute)))
1494                })
1495                .collect::<Result<Vec<_>>>()?,
1496            BatchRegionDdlRequest::Drop(requests) => requests
1497                .iter()
1498                .map(|(region_id, _)| (*region_id, RegionChange::Deregisters))
1499                .collect::<Vec<_>>(),
1500            BatchRegionDdlRequest::Alter(requests) => requests
1501                .iter()
1502                .map(|(region_id, _)| (*region_id, RegionChange::None))
1503                .collect::<Vec<_>>(),
1504        };
1505
1506        // The ddl procedure will ensure all requests are in the same engine.
1507        // Therefore, we can get the engine from the first request.
1508        let (first_region_id, first_region_change) = region_changes.first().unwrap();
1509        let engine = match self.get_engine(*first_region_id, first_region_change)? {
1510            CurrentEngine::Engine(engine) => engine,
1511            CurrentEngine::EarlyReturn(rows) => return Ok(RegionResponse::new(rows)),
1512        };
1513
1514        for (region_id, region_change) in region_changes.iter() {
1515            self.set_region_status_not_ready(*region_id, &engine, region_change);
1516        }
1517
1518        let ddl_type = batch_request.request_type();
1519        let result = engine
1520            .handle_batch_ddl_requests(batch_request)
1521            .await
1522            .context(HandleBatchDdlRequestSnafu { ddl_type });
1523
1524        match result {
1525            Ok(result) => {
1526                for (region_id, region_change) in &region_changes {
1527                    self.set_region_status_ready(*region_id, engine.clone(), *region_change)
1528                        .await?;
1529                }
1530
1531                Ok(RegionResponse {
1532                    affected_rows: result.affected_rows,
1533                    extensions: result.extensions,
1534                    metadata: Vec::new(),
1535                })
1536            }
1537            Err(err) => {
1538                for (region_id, region_change) in region_changes {
1539                    self.unset_region_status(region_id, &engine, region_change);
1540                }
1541
1542                Err(err)
1543            }
1544        }
1545    }
1546
1547    pub async fn handle_request(
1548        &self,
1549        region_id: RegionId,
1550        request: RegionRequest,
1551    ) -> Result<RegionResponse> {
1552        let request_type = request.request_type();
1553        let _timer = crate::metrics::HANDLE_REGION_REQUEST_ELAPSED
1554            .with_label_values(&[request_type])
1555            .start_timer();
1556
1557        let region_change = match &request {
1558            RegionRequest::Create(create) => {
1559                let attribute = parse_region_attribute(&create.engine, &create.options)?;
1560                RegionChange::Register(attribute)
1561            }
1562            RegionRequest::Open(open) => {
1563                let attribute = parse_region_attribute(&open.engine, &open.options)?;
1564                RegionChange::Register(attribute)
1565            }
1566            RegionRequest::Close(_) | RegionRequest::Drop(_) => RegionChange::Deregisters,
1567            RegionRequest::Put(_) | RegionRequest::Delete(_) | RegionRequest::BulkInserts(_) => {
1568                RegionChange::Ingest
1569            }
1570            RegionRequest::Alter(_)
1571            | RegionRequest::Flush(_)
1572            | RegionRequest::Compact(_)
1573            | RegionRequest::Truncate(_)
1574            | RegionRequest::BuildIndex(_)
1575            | RegionRequest::EnterStaging(_)
1576            | RegionRequest::ApplyStagingManifest(_) => RegionChange::None,
1577            RegionRequest::Catchup(_) => RegionChange::Catchup,
1578        };
1579
1580        let engine = match self.get_engine(region_id, &region_change)? {
1581            CurrentEngine::Engine(engine) => engine,
1582            CurrentEngine::EarlyReturn(rows) => return Ok(RegionResponse::new(rows)),
1583        };
1584
1585        // Sets corresponding region status to registering/deregistering before the operation.
1586        self.set_region_status_not_ready(region_id, &engine, &region_change);
1587
1588        match engine
1589            .handle_request(region_id, request)
1590            .await
1591            .with_context(|_| HandleRegionRequestSnafu { region_id })
1592        {
1593            Ok(result) => {
1594                // Update metrics
1595                if matches!(region_change, RegionChange::Ingest) {
1596                    crate::metrics::REGION_CHANGED_ROW_COUNT
1597                        .with_label_values(&[request_type])
1598                        .inc_by(result.affected_rows as u64);
1599                }
1600                // Sets corresponding region status to ready.
1601                self.set_region_status_ready(region_id, engine.clone(), region_change)
1602                    .await?;
1603
1604                Ok(RegionResponse {
1605                    affected_rows: result.affected_rows,
1606                    extensions: result.extensions,
1607                    metadata: Vec::new(),
1608                })
1609            }
1610            Err(err) => {
1611                if matches!(region_change, RegionChange::Ingest) {
1612                    crate::metrics::REGION_SERVER_INSERT_FAIL_COUNT
1613                        .with_label_values(&[request_type])
1614                        .inc();
1615                }
1616                // Removes the region status if the operation fails.
1617                self.unset_region_status(region_id, &engine, region_change);
1618                Err(err)
1619            }
1620        }
1621    }
1622
1623    /// Handles the sync region request.
1624    pub async fn handle_sync_region(
1625        &self,
1626        engine: &RegionEngineRef,
1627        region_id: RegionId,
1628        request: SyncRegionFromRequest,
1629    ) -> Result<()> {
1630        let Some(new_opened_regions) = engine
1631            .sync_region(region_id, request)
1632            .await
1633            .with_context(|_| HandleRegionRequestSnafu { region_id })?
1634            .new_opened_logical_region_ids()
1635        else {
1636            return Ok(());
1637        };
1638
1639        for region in &new_opened_regions {
1640            self.region_map
1641                .insert(*region, RegionEngineWithStatus::Ready(engine.clone()));
1642        }
1643        if !new_opened_regions.is_empty() {
1644            info!(
1645                region_id = %region_id,
1646                logical_region_count = new_opened_regions.len(),
1647                logical_regions = ?new_opened_regions,
1648                "Logical regions are registered"
1649            );
1650        }
1651
1652        Ok(())
1653    }
1654
1655    fn set_region_status_not_ready(
1656        &self,
1657        region_id: RegionId,
1658        engine: &RegionEngineRef,
1659        region_change: &RegionChange,
1660    ) {
1661        match region_change {
1662            RegionChange::Register(_) => {
1663                self.region_map.insert(
1664                    region_id,
1665                    RegionEngineWithStatus::Registering(engine.clone()),
1666                );
1667            }
1668            RegionChange::Deregisters => {
1669                self.region_map.insert(
1670                    region_id,
1671                    RegionEngineWithStatus::Deregistering(engine.clone()),
1672                );
1673            }
1674            _ => {}
1675        }
1676    }
1677
1678    fn unset_region_status(
1679        &self,
1680        region_id: RegionId,
1681        engine: &RegionEngineRef,
1682        region_change: RegionChange,
1683    ) {
1684        match region_change {
1685            RegionChange::None | RegionChange::Ingest => {}
1686            RegionChange::Register(_) => {
1687                self.region_map.remove(&region_id);
1688            }
1689            RegionChange::Deregisters => {
1690                self.region_map
1691                    .insert(region_id, RegionEngineWithStatus::Ready(engine.clone()));
1692            }
1693            RegionChange::Catchup => {}
1694        }
1695    }
1696
1697    async fn set_region_status_ready(
1698        &self,
1699        region_id: RegionId,
1700        engine: RegionEngineRef,
1701        region_change: RegionChange,
1702    ) -> Result<()> {
1703        let engine_type = engine.name();
1704        match region_change {
1705            RegionChange::None | RegionChange::Ingest => {}
1706            RegionChange::Register(attribute) => {
1707                info!(
1708                    "Region {region_id} is registered to engine {}",
1709                    attribute.engine()
1710                );
1711                self.region_map
1712                    .insert(region_id, RegionEngineWithStatus::Ready(engine.clone()));
1713
1714                match attribute {
1715                    RegionAttribute::Metric { physical } => {
1716                        if physical {
1717                            // Registers the logical regions belong to the physical region (`region_id`).
1718                            self.register_logical_regions(&engine, region_id).await?;
1719                            // We only send the `on_region_registered` event of the physical region.
1720                            self.event_listener.on_region_registered(region_id);
1721                        }
1722                    }
1723                    RegionAttribute::Mito => self.event_listener.on_region_registered(region_id),
1724                    RegionAttribute::File => {
1725                        // do nothing
1726                    }
1727                }
1728            }
1729            RegionChange::Deregisters => {
1730                info!("Region {region_id} is deregistered from engine {engine_type}");
1731                self.region_map
1732                    .remove(&region_id)
1733                    .map(|(id, engine)| engine.set_region_role(id, RegionRole::Follower));
1734                self.event_listener.on_region_deregistered(region_id);
1735            }
1736            RegionChange::Catchup => {
1737                if is_metric_engine(engine.name()) {
1738                    // Registers the logical regions belong to the physical region (`region_id`).
1739                    self.register_logical_regions(&engine, region_id).await?;
1740                }
1741            }
1742        }
1743        Ok(())
1744    }
1745
1746    async fn register_logical_regions(
1747        &self,
1748        engine: &RegionEngineRef,
1749        physical_region_id: RegionId,
1750    ) -> Result<()> {
1751        let metric_engine =
1752            engine
1753                .as_any()
1754                .downcast_ref::<MetricEngine>()
1755                .context(UnexpectedSnafu {
1756                    violated: format!(
1757                        "expecting engine type '{}', actual '{}'",
1758                        METRIC_ENGINE_NAME,
1759                        engine.name(),
1760                    ),
1761                })?;
1762
1763        let logical_regions = metric_engine
1764            .logical_regions(physical_region_id)
1765            .await
1766            .context(FindLogicalRegionsSnafu { physical_region_id })?;
1767
1768        for region in &logical_regions {
1769            self.region_map
1770                .insert(*region, RegionEngineWithStatus::Ready(engine.clone()));
1771        }
1772        if !logical_regions.is_empty() {
1773            info!(
1774                physical_region_id = %physical_region_id,
1775                logical_region_count = logical_regions.len(),
1776                logical_regions = ?logical_regions,
1777                "Logical regions are registered"
1778            );
1779        }
1780        Ok(())
1781    }
1782
1783    pub async fn handle_read(
1784        &self,
1785        request: QueryRequest,
1786        query_ctx: QueryContextRef,
1787    ) -> Result<SendableRecordBatchStream> {
1788        // TODO(ruihang): add metrics and set trace id
1789
1790        let result = self
1791            .query_engine
1792            .execute(request.plan, query_ctx)
1793            .await
1794            .context(ExecuteLogicalPlanSnafu)?;
1795
1796        match result.data {
1797            OutputData::AffectedRows(_) | OutputData::RecordBatches(_) => {
1798                UnsupportedOutputSnafu { expected: "stream" }.fail()
1799            }
1800            OutputData::Stream(stream) => Ok(stream),
1801        }
1802    }
1803
1804    async fn stop(&self) -> Result<()> {
1805        // Calling async functions while iterating inside the Dashmap could easily cause the Rust
1806        // complains "higher-ranked lifetime error". Rust can't prove some future is legit.
1807        // Possible related issue: https://github.com/rust-lang/rust/issues/102211
1808        //
1809        // The workaround is to put the async functions in the `common_runtime::spawn_global`. Or like
1810        // it here, collect the values first then use later separately.
1811
1812        let regions = self
1813            .region_map
1814            .iter()
1815            .map(|x| (*x.key(), x.value().clone()))
1816            .collect::<Vec<_>>();
1817        let num_regions = regions.len();
1818
1819        for (region_id, engine) in regions {
1820            let closed = engine
1821                .handle_request(region_id, RegionRequest::Close(RegionCloseRequest {}))
1822                .await;
1823            match closed {
1824                Ok(_) => debug!("Region {region_id} is closed"),
1825                Err(e) => warn!("Failed to close region {region_id}, err: {e}"),
1826            }
1827        }
1828        self.region_map.clear();
1829        info!("closed {num_regions} regions");
1830
1831        drop(self.mito_engine.write().unwrap().take());
1832        let engines = self.engines.write().unwrap().drain().collect::<Vec<_>>();
1833        for (engine_name, engine) in engines {
1834            engine
1835                .stop()
1836                .await
1837                .context(StopRegionEngineSnafu { name: &engine_name })?;
1838            info!("Region engine {engine_name} is stopped");
1839        }
1840
1841        Ok(())
1842    }
1843}
1844
1845#[derive(Debug, Clone, Copy)]
1846enum RegionChange {
1847    None,
1848    Register(RegionAttribute),
1849    Deregisters,
1850    Catchup,
1851    Ingest,
1852}
1853
1854fn is_metric_engine(engine: &str) -> bool {
1855    engine == METRIC_ENGINE_NAME
1856}
1857
1858fn parse_region_attribute(
1859    engine: &str,
1860    options: &HashMap<String, String>,
1861) -> Result<RegionAttribute> {
1862    match engine {
1863        MITO_ENGINE_NAME => Ok(RegionAttribute::Mito),
1864        METRIC_ENGINE_NAME => {
1865            let physical = !options.contains_key(LOGICAL_TABLE_METADATA_KEY);
1866
1867            Ok(RegionAttribute::Metric { physical })
1868        }
1869        FILE_ENGINE_NAME => Ok(RegionAttribute::File),
1870        _ => error::UnexpectedSnafu {
1871            violated: format!("Unknown engine: {}", engine),
1872        }
1873        .fail(),
1874    }
1875}
1876
1877#[derive(Debug, Clone, Copy)]
1878enum RegionAttribute {
1879    Mito,
1880    Metric { physical: bool },
1881    File,
1882}
1883
1884impl RegionAttribute {
1885    fn engine(&self) -> &'static str {
1886        match self {
1887            RegionAttribute::Mito => MITO_ENGINE_NAME,
1888            RegionAttribute::Metric { .. } => METRIC_ENGINE_NAME,
1889            RegionAttribute::File => FILE_ENGINE_NAME,
1890        }
1891    }
1892}
1893
1894#[cfg(test)]
1895mod tests {
1896
1897    use std::assert_matches;
1898    use std::collections::HashMap;
1899    use std::sync::Arc;
1900
1901    use api::v1::SemanticType;
1902    use api::v1::region::{
1903        RemoteDynFilterRequest, RemoteDynFilterUnregister, RemoteDynFilterUpdate,
1904        remote_dyn_filter_request,
1905    };
1906    use common_error::ext::ErrorExt;
1907    use common_recordbatch::RecordBatches;
1908    use common_recordbatch::adapter::{RecordBatchMetrics, RegionWatermarkEntry};
1909    use datatypes::prelude::{ConcreteDataType, VectorRef};
1910    use datatypes::schema::{ColumnSchema, Schema};
1911    use datatypes::vectors::Int32Vector;
1912    use futures_util::StreamExt;
1913    use mito2::test_util::CreateRequestBuilder;
1914    use query::options::FLOW_RETURN_REGION_SEQ;
1915    use store_api::metadata::{ColumnMetadata, RegionMetadata, RegionMetadataBuilder};
1916    use store_api::region_engine::RegionEngine;
1917    use store_api::region_request::{
1918        PathType, RegionDropRequest, RegionOpenRequest, RegionTruncateRequest,
1919    };
1920    use store_api::storage::RegionId;
1921
1922    use super::*;
1923    use crate::error::Result;
1924    use crate::tests::{MockRegionEngine, mock_region_server};
1925
1926    fn single_value_stream() -> SendableRecordBatchStream {
1927        let schema = Arc::new(Schema::new(vec![ColumnSchema::new(
1928            "v",
1929            ConcreteDataType::int32_datatype(),
1930            false,
1931        )]));
1932        let values: VectorRef = Arc::new(Int32Vector::from_slice([1]));
1933        let batch = RecordBatch::new(schema.clone(), vec![values]).unwrap();
1934        RecordBatches::try_new(schema, vec![batch])
1935            .unwrap()
1936            .as_stream()
1937    }
1938
1939    #[tokio::test]
1940    async fn test_region_watermark_stream_only_sets_terminal_metrics() {
1941        let schema = Arc::new(Schema::new(vec![ColumnSchema::new(
1942            "v",
1943            ConcreteDataType::int32_datatype(),
1944            false,
1945        )]));
1946        let values: VectorRef = Arc::new(Int32Vector::from_slice([1, 2]));
1947        let batch = RecordBatch::new(schema.clone(), vec![values]).unwrap();
1948        let stream = RecordBatches::try_new(schema, vec![batch])
1949            .unwrap()
1950            .as_stream();
1951
1952        let region_id = RegionId::new(42, 7);
1953        let wrapped = RegionWatermarkStream::new(stream, region_id, 99);
1954        let mut pinned = Box::pin(wrapped);
1955
1956        assert!(pinned.as_ref().get_ref().metrics().is_none());
1957        while pinned.next().await.is_some() {}
1958
1959        let metrics = pinned.as_ref().get_ref().metrics().unwrap();
1960        assert_eq!(
1961            metrics.region_watermarks,
1962            vec![RegionWatermarkEntry {
1963                region_id: region_id.as_u64(),
1964                watermark: Some(99),
1965            }]
1966        );
1967    }
1968
1969    #[test]
1970    fn test_region_watermark_stream_preserves_unproved_watermark() {
1971        let schema = Arc::new(Schema::new(vec![ColumnSchema::new(
1972            "v",
1973            ConcreteDataType::int32_datatype(),
1974            false,
1975        )]));
1976        let values: VectorRef = Arc::new(Int32Vector::from_slice([1]));
1977        let batch = RecordBatch::new(schema.clone(), vec![values]).unwrap();
1978        let stream = RecordBatches::try_new(schema, vec![batch])
1979            .unwrap()
1980            .as_stream();
1981
1982        let region_id = RegionId::new(42, 7);
1983        let wrapped = RegionWatermarkStream::new(stream, region_id, 99);
1984        let metrics = RecordBatchMetrics {
1985            region_watermarks: vec![RegionWatermarkEntry {
1986                region_id: region_id.as_u64(),
1987                watermark: None,
1988            }],
1989            ..Default::default()
1990        };
1991
1992        let merged = wrapped.merged_metrics(metrics);
1993        assert_eq!(
1994            merged.region_watermarks,
1995            vec![RegionWatermarkEntry {
1996                region_id: region_id.as_u64(),
1997                watermark: None,
1998            }]
1999        );
2000    }
2001
2002    #[tokio::test]
2003    async fn test_wrap_flow_region_watermark_stream_adds_terminal_metrics() {
2004        let region_id = RegionId::new(42, 7);
2005        let query_ctx = Arc::new(
2006            QueryContextBuilder::default()
2007                .extensions(HashMap::from([(
2008                    FLOW_RETURN_REGION_SEQ.to_string(),
2009                    "true".to_string(),
2010                )]))
2011                .build(),
2012        );
2013        query_ctx.set_snapshot(region_id.as_u64(), 99);
2014
2015        let wrapped =
2016            wrap_flow_region_watermark_stream(single_value_stream(), region_id, &query_ctx);
2017        let mut pinned = Box::pin(wrapped);
2018        while pinned.next().await.is_some() {}
2019
2020        let metrics = pinned.as_ref().get_ref().metrics().unwrap();
2021        assert_eq!(
2022            metrics.region_watermarks,
2023            vec![RegionWatermarkEntry {
2024                region_id: region_id.as_u64(),
2025                watermark: Some(99),
2026            }]
2027        );
2028    }
2029
2030    #[tokio::test]
2031    async fn test_wrap_flow_region_watermark_stream_skips_without_extension() {
2032        let region_id = RegionId::new(42, 7);
2033        let query_ctx = Arc::new(QueryContextBuilder::default().build());
2034        query_ctx.set_snapshot(region_id.as_u64(), 99);
2035
2036        let wrapped =
2037            wrap_flow_region_watermark_stream(single_value_stream(), region_id, &query_ctx);
2038        let mut pinned = Box::pin(wrapped);
2039        while pinned.next().await.is_some() {}
2040
2041        assert!(pinned.as_ref().get_ref().metrics().is_none());
2042    }
2043
2044    #[tokio::test]
2045    async fn test_wrap_flow_region_watermark_stream_skips_without_snapshot() {
2046        let region_id = RegionId::new(42, 7);
2047        let query_ctx = Arc::new(
2048            QueryContextBuilder::default()
2049                .extensions(HashMap::from([(
2050                    FLOW_RETURN_REGION_SEQ.to_string(),
2051                    "true".to_string(),
2052                )]))
2053                .build(),
2054        );
2055
2056        let wrapped =
2057            wrap_flow_region_watermark_stream(single_value_stream(), region_id, &query_ctx);
2058        let mut pinned = Box::pin(wrapped);
2059        while pinned.next().await.is_some() {}
2060
2061        assert!(pinned.as_ref().get_ref().metrics().is_none());
2062    }
2063
2064    #[tokio::test]
2065    async fn test_region_registering() {
2066        common_telemetry::init_default_ut_logging();
2067
2068        let mut mock_region_server = mock_region_server();
2069        let (engine, _receiver) = MockRegionEngine::new(MITO_ENGINE_NAME);
2070        let engine_name = engine.name();
2071        mock_region_server.register_engine(engine.clone());
2072        let region_id = RegionId::new(1, 1);
2073        let builder = CreateRequestBuilder::new();
2074        let create_req = builder.build();
2075        // Tries to create/open a registering region.
2076        mock_region_server.inner.region_map.insert(
2077            region_id,
2078            RegionEngineWithStatus::Registering(engine.clone()),
2079        );
2080        let response = mock_region_server
2081            .handle_request(region_id, RegionRequest::Create(create_req))
2082            .await
2083            .unwrap();
2084        assert_eq!(response.affected_rows, 0);
2085        let status = mock_region_server
2086            .inner
2087            .region_map
2088            .get(&region_id)
2089            .unwrap()
2090            .clone();
2091        assert!(matches!(status, RegionEngineWithStatus::Ready(_)));
2092
2093        mock_region_server.inner.region_map.insert(
2094            region_id,
2095            RegionEngineWithStatus::Registering(engine.clone()),
2096        );
2097        let response = mock_region_server
2098            .handle_request(
2099                region_id,
2100                RegionRequest::Open(RegionOpenRequest {
2101                    engine: engine_name.to_string(),
2102                    table_dir: String::new(),
2103                    path_type: PathType::Bare,
2104                    options: Default::default(),
2105                    skip_wal_replay: false,
2106                    checkpoint: None,
2107                    requirements: Default::default(),
2108                }),
2109            )
2110            .await
2111            .unwrap();
2112        assert_eq!(response.affected_rows, 0);
2113        let status = mock_region_server
2114            .inner
2115            .region_map
2116            .get(&region_id)
2117            .unwrap()
2118            .clone();
2119        assert!(matches!(status, RegionEngineWithStatus::Ready(_)));
2120    }
2121
2122    #[tokio::test]
2123    async fn test_region_deregistering() {
2124        common_telemetry::init_default_ut_logging();
2125
2126        let mut mock_region_server = mock_region_server();
2127        let (engine, _receiver) = MockRegionEngine::new(MITO_ENGINE_NAME);
2128
2129        mock_region_server.register_engine(engine.clone());
2130
2131        let region_id = RegionId::new(1, 1);
2132
2133        // Tries to drop/close a registering region.
2134        mock_region_server.inner.region_map.insert(
2135            region_id,
2136            RegionEngineWithStatus::Deregistering(engine.clone()),
2137        );
2138
2139        let response = mock_region_server
2140            .handle_request(
2141                region_id,
2142                RegionRequest::Drop(RegionDropRequest {
2143                    fast_path: false,
2144                    force: false,
2145                    partial_drop: false,
2146                }),
2147            )
2148            .await
2149            .unwrap();
2150        assert_eq!(response.affected_rows, 0);
2151
2152        let status = mock_region_server
2153            .inner
2154            .region_map
2155            .get(&region_id)
2156            .unwrap()
2157            .clone();
2158        assert!(matches!(status, RegionEngineWithStatus::Deregistering(_)));
2159
2160        mock_region_server.inner.region_map.insert(
2161            region_id,
2162            RegionEngineWithStatus::Deregistering(engine.clone()),
2163        );
2164
2165        let response = mock_region_server
2166            .handle_request(region_id, RegionRequest::Close(RegionCloseRequest {}))
2167            .await
2168            .unwrap();
2169        assert_eq!(response.affected_rows, 0);
2170
2171        let status = mock_region_server
2172            .inner
2173            .region_map
2174            .get(&region_id)
2175            .unwrap()
2176            .clone();
2177        assert!(matches!(status, RegionEngineWithStatus::Deregistering(_)));
2178    }
2179
2180    #[tokio::test]
2181    async fn test_region_not_ready() {
2182        common_telemetry::init_default_ut_logging();
2183
2184        let mut mock_region_server = mock_region_server();
2185        let (engine, _receiver) = MockRegionEngine::new(MITO_ENGINE_NAME);
2186
2187        mock_region_server.register_engine(engine.clone());
2188
2189        let region_id = RegionId::new(1, 1);
2190
2191        // Tries to drop/close a registering region.
2192        mock_region_server.inner.region_map.insert(
2193            region_id,
2194            RegionEngineWithStatus::Registering(engine.clone()),
2195        );
2196
2197        let err = mock_region_server
2198            .handle_request(
2199                region_id,
2200                RegionRequest::Truncate(RegionTruncateRequest::All),
2201            )
2202            .await
2203            .unwrap_err();
2204
2205        assert_eq!(err.status_code(), StatusCode::RegionNotReady);
2206    }
2207
2208    #[tokio::test]
2209    async fn test_region_request_failed() {
2210        common_telemetry::init_default_ut_logging();
2211
2212        let mut mock_region_server = mock_region_server();
2213        let (engine, _receiver) = MockRegionEngine::with_mock_fn(
2214            MITO_ENGINE_NAME,
2215            Box::new(|_region_id, _request| {
2216                error::UnexpectedSnafu {
2217                    violated: "test".to_string(),
2218                }
2219                .fail()
2220            }),
2221        );
2222
2223        mock_region_server.register_engine(engine.clone());
2224
2225        let region_id = RegionId::new(1, 1);
2226        let builder = CreateRequestBuilder::new();
2227        let create_req = builder.build();
2228        mock_region_server
2229            .handle_request(region_id, RegionRequest::Create(create_req))
2230            .await
2231            .unwrap_err();
2232
2233        let status = mock_region_server.inner.region_map.get(&region_id);
2234        assert!(status.is_none());
2235
2236        mock_region_server
2237            .inner
2238            .region_map
2239            .insert(region_id, RegionEngineWithStatus::Ready(engine.clone()));
2240
2241        mock_region_server
2242            .handle_request(
2243                region_id,
2244                RegionRequest::Drop(RegionDropRequest {
2245                    fast_path: false,
2246                    force: false,
2247                    partial_drop: false,
2248                }),
2249            )
2250            .await
2251            .unwrap_err();
2252
2253        let status = mock_region_server.inner.region_map.get(&region_id);
2254        assert!(status.is_some());
2255    }
2256
2257    #[tokio::test]
2258    async fn test_batch_open_region_ignore_nonexistent_regions() {
2259        common_telemetry::init_default_ut_logging();
2260        let mut mock_region_server = mock_region_server();
2261        let (engine, _receiver) = MockRegionEngine::with_mock_fn(
2262            MITO_ENGINE_NAME,
2263            Box::new(|region_id, _request| {
2264                if region_id == RegionId::new(1, 1) {
2265                    error::RegionNotFoundSnafu { region_id }.fail()
2266                } else {
2267                    Ok(0)
2268                }
2269            }),
2270        );
2271        mock_region_server.register_engine(engine.clone());
2272
2273        let region_ids = mock_region_server
2274            .handle_batch_open_requests(
2275                8,
2276                vec![
2277                    (
2278                        RegionId::new(1, 1),
2279                        RegionOpenRequest {
2280                            engine: MITO_ENGINE_NAME.to_string(),
2281                            table_dir: String::new(),
2282                            path_type: PathType::Bare,
2283                            options: Default::default(),
2284                            skip_wal_replay: false,
2285                            checkpoint: None,
2286                            requirements: Default::default(),
2287                        },
2288                    ),
2289                    (
2290                        RegionId::new(1, 2),
2291                        RegionOpenRequest {
2292                            engine: MITO_ENGINE_NAME.to_string(),
2293                            table_dir: String::new(),
2294                            path_type: PathType::Bare,
2295                            options: Default::default(),
2296                            skip_wal_replay: false,
2297                            checkpoint: None,
2298                            requirements: Default::default(),
2299                        },
2300                    ),
2301                ],
2302                true,
2303            )
2304            .await
2305            .unwrap();
2306        assert_eq!(region_ids, vec![RegionId::new(1, 2)]);
2307
2308        let err = mock_region_server
2309            .handle_batch_open_requests(
2310                8,
2311                vec![
2312                    (
2313                        RegionId::new(1, 1),
2314                        RegionOpenRequest {
2315                            engine: MITO_ENGINE_NAME.to_string(),
2316                            table_dir: String::new(),
2317                            path_type: PathType::Bare,
2318                            options: Default::default(),
2319                            skip_wal_replay: false,
2320                            checkpoint: None,
2321                            requirements: Default::default(),
2322                        },
2323                    ),
2324                    (
2325                        RegionId::new(1, 2),
2326                        RegionOpenRequest {
2327                            engine: MITO_ENGINE_NAME.to_string(),
2328                            table_dir: String::new(),
2329                            path_type: PathType::Bare,
2330                            options: Default::default(),
2331                            skip_wal_replay: false,
2332                            checkpoint: None,
2333                            requirements: Default::default(),
2334                        },
2335                    ),
2336                ],
2337                false,
2338            )
2339            .await
2340            .unwrap_err();
2341        assert_eq!(err.status_code(), StatusCode::Unexpected);
2342    }
2343
2344    struct CurrentEngineTest {
2345        region_id: RegionId,
2346        current_region_status: Option<RegionEngineWithStatus>,
2347        region_change: RegionChange,
2348        assert: Box<dyn FnOnce(Result<CurrentEngine>)>,
2349    }
2350
2351    #[tokio::test]
2352    async fn test_current_engine() {
2353        common_telemetry::init_default_ut_logging();
2354
2355        let mut mock_region_server = mock_region_server();
2356        let (engine, _) = MockRegionEngine::new(MITO_ENGINE_NAME);
2357        mock_region_server.register_engine(engine.clone());
2358
2359        let region_id = RegionId::new(1024, 1);
2360        let tests = vec![
2361            // RegionChange::None
2362            CurrentEngineTest {
2363                region_id,
2364                current_region_status: None,
2365                region_change: RegionChange::None,
2366                assert: Box::new(|result| {
2367                    let err = result.unwrap_err();
2368                    assert_eq!(err.status_code(), StatusCode::RegionNotFound);
2369                }),
2370            },
2371            CurrentEngineTest {
2372                region_id,
2373                current_region_status: Some(RegionEngineWithStatus::Ready(engine.clone())),
2374                region_change: RegionChange::None,
2375                assert: Box::new(|result| {
2376                    let current_engine = result.unwrap();
2377                    assert_matches!(current_engine, CurrentEngine::Engine(_));
2378                }),
2379            },
2380            CurrentEngineTest {
2381                region_id,
2382                current_region_status: Some(RegionEngineWithStatus::Registering(engine.clone())),
2383                region_change: RegionChange::None,
2384                assert: Box::new(|result| {
2385                    let err = result.unwrap_err();
2386                    assert_eq!(err.status_code(), StatusCode::RegionNotReady);
2387                }),
2388            },
2389            CurrentEngineTest {
2390                region_id,
2391                current_region_status: Some(RegionEngineWithStatus::Deregistering(engine.clone())),
2392                region_change: RegionChange::None,
2393                assert: Box::new(|result| {
2394                    let err = result.unwrap_err();
2395                    assert_eq!(err.status_code(), StatusCode::RegionNotFound);
2396                }),
2397            },
2398            // RegionChange::Register
2399            CurrentEngineTest {
2400                region_id,
2401                current_region_status: None,
2402                region_change: RegionChange::Register(RegionAttribute::Mito),
2403                assert: Box::new(|result| {
2404                    let current_engine = result.unwrap();
2405                    assert_matches!(current_engine, CurrentEngine::Engine(_));
2406                }),
2407            },
2408            CurrentEngineTest {
2409                region_id,
2410                current_region_status: Some(RegionEngineWithStatus::Registering(engine.clone())),
2411                region_change: RegionChange::Register(RegionAttribute::Mito),
2412                assert: Box::new(|result| {
2413                    let current_engine = result.unwrap();
2414                    assert_matches!(current_engine, CurrentEngine::Engine(_));
2415                }),
2416            },
2417            CurrentEngineTest {
2418                region_id,
2419                current_region_status: Some(RegionEngineWithStatus::Deregistering(engine.clone())),
2420                region_change: RegionChange::Register(RegionAttribute::Mito),
2421                assert: Box::new(|result| {
2422                    let err = result.unwrap_err();
2423                    assert_eq!(err.status_code(), StatusCode::RegionBusy);
2424                }),
2425            },
2426            CurrentEngineTest {
2427                region_id,
2428                current_region_status: Some(RegionEngineWithStatus::Ready(engine.clone())),
2429                region_change: RegionChange::Register(RegionAttribute::Mito),
2430                assert: Box::new(|result| {
2431                    let current_engine = result.unwrap();
2432                    assert_matches!(current_engine, CurrentEngine::Engine(_));
2433                }),
2434            },
2435            // RegionChange::Deregister
2436            CurrentEngineTest {
2437                region_id,
2438                current_region_status: None,
2439                region_change: RegionChange::Deregisters,
2440                assert: Box::new(|result| {
2441                    let current_engine = result.unwrap();
2442                    assert_matches!(current_engine, CurrentEngine::EarlyReturn(_));
2443                }),
2444            },
2445            CurrentEngineTest {
2446                region_id,
2447                current_region_status: Some(RegionEngineWithStatus::Registering(engine.clone())),
2448                region_change: RegionChange::Deregisters,
2449                assert: Box::new(|result| {
2450                    let err = result.unwrap_err();
2451                    assert_eq!(err.status_code(), StatusCode::RegionBusy);
2452                }),
2453            },
2454            CurrentEngineTest {
2455                region_id,
2456                current_region_status: Some(RegionEngineWithStatus::Deregistering(engine.clone())),
2457                region_change: RegionChange::Deregisters,
2458                assert: Box::new(|result| {
2459                    let current_engine = result.unwrap();
2460                    assert_matches!(current_engine, CurrentEngine::EarlyReturn(_));
2461                }),
2462            },
2463            CurrentEngineTest {
2464                region_id,
2465                current_region_status: Some(RegionEngineWithStatus::Ready(engine.clone())),
2466                region_change: RegionChange::Deregisters,
2467                assert: Box::new(|result| {
2468                    let current_engine = result.unwrap();
2469                    assert_matches!(current_engine, CurrentEngine::Engine(_));
2470                }),
2471            },
2472        ];
2473
2474        for test in tests {
2475            let CurrentEngineTest {
2476                region_id,
2477                current_region_status,
2478                region_change,
2479                assert,
2480            } = test;
2481
2482            // Sets up
2483            if let Some(status) = current_region_status {
2484                mock_region_server
2485                    .inner
2486                    .region_map
2487                    .insert(region_id, status);
2488            } else {
2489                mock_region_server.inner.region_map.remove(&region_id);
2490            }
2491
2492            let result = mock_region_server
2493                .inner
2494                .get_engine(region_id, &region_change);
2495
2496            assert(result);
2497        }
2498    }
2499
2500    #[tokio::test]
2501    async fn test_region_server_parallelism() {
2502        let p = RegionServerParallelism::from_opts(2, Duration::from_millis(1)).unwrap();
2503        let first_query = p.acquire().await;
2504        assert!(first_query.is_ok());
2505        let second_query = p.acquire().await;
2506        assert!(second_query.is_ok());
2507        let third_query = p.acquire().await;
2508        assert!(third_query.is_err());
2509        let err = third_query.unwrap_err();
2510        assert_eq!(
2511            err.output_msg(),
2512            "Failed to acquire permit under timeouts: deadline has elapsed".to_string()
2513        );
2514        drop(first_query);
2515        let forth_query = p.acquire().await;
2516        assert!(forth_query.is_ok());
2517    }
2518
2519    fn mock_region_metadata(region_id: RegionId) -> RegionMetadata {
2520        let mut metadata_builder = RegionMetadataBuilder::new(region_id);
2521        metadata_builder.push_column_metadata(ColumnMetadata {
2522            column_schema: datatypes::schema::ColumnSchema::new(
2523                "timestamp",
2524                ConcreteDataType::timestamp_nanosecond_datatype(),
2525                false,
2526            ),
2527            semantic_type: SemanticType::Timestamp,
2528            column_id: 0,
2529        });
2530        metadata_builder.push_column_metadata(ColumnMetadata {
2531            column_schema: datatypes::schema::ColumnSchema::new(
2532                "file",
2533                ConcreteDataType::string_datatype(),
2534                true,
2535            ),
2536            semantic_type: SemanticType::Tag,
2537            column_id: 1,
2538        });
2539        metadata_builder.push_column_metadata(ColumnMetadata {
2540            column_schema: datatypes::schema::ColumnSchema::new(
2541                "message",
2542                ConcreteDataType::string_datatype(),
2543                true,
2544            ),
2545            semantic_type: SemanticType::Field,
2546            column_id: 2,
2547        });
2548        metadata_builder.primary_key(vec![1]);
2549        metadata_builder.build().unwrap()
2550    }
2551
2552    #[tokio::test]
2553    async fn test_handle_list_metadata_request() {
2554        common_telemetry::init_default_ut_logging();
2555
2556        let mut mock_region_server = mock_region_server();
2557        let region_id_1 = RegionId::new(1, 0);
2558        let region_id_2 = RegionId::new(2, 0);
2559
2560        let metadata_1 = mock_region_metadata(region_id_1);
2561        let metadata_2 = mock_region_metadata(region_id_2);
2562        let metadatas = vec![Some(metadata_1.clone()), Some(metadata_2.clone())];
2563
2564        let metadata_1 = Arc::new(metadata_1);
2565        let metadata_2 = Arc::new(metadata_2);
2566        let (engine, _) = MockRegionEngine::with_metadata_mock_fn(
2567            MITO_ENGINE_NAME,
2568            Box::new(move |region_id| {
2569                if region_id == region_id_1 {
2570                    Ok(metadata_1.clone())
2571                } else if region_id == region_id_2 {
2572                    Ok(metadata_2.clone())
2573                } else {
2574                    error::RegionNotFoundSnafu { region_id }.fail()
2575                }
2576            }),
2577        );
2578
2579        mock_region_server.register_engine(engine.clone());
2580        mock_region_server
2581            .inner
2582            .region_map
2583            .insert(region_id_1, RegionEngineWithStatus::Ready(engine.clone()));
2584        mock_region_server
2585            .inner
2586            .region_map
2587            .insert(region_id_2, RegionEngineWithStatus::Ready(engine.clone()));
2588
2589        // All regions exist.
2590        let list_metadata_request = ListMetadataRequest {
2591            region_ids: vec![region_id_1.as_u64(), region_id_2.as_u64()],
2592        };
2593        let response = mock_region_server
2594            .handle_list_metadata_request(&list_metadata_request)
2595            .await
2596            .unwrap();
2597        let decoded_metadata: Vec<Option<RegionMetadata>> =
2598            serde_json::from_slice(&response.metadata).unwrap();
2599        assert_eq!(metadatas, decoded_metadata);
2600    }
2601
2602    #[tokio::test]
2603    async fn test_handle_list_metadata_not_found() {
2604        common_telemetry::init_default_ut_logging();
2605
2606        let mut mock_region_server = mock_region_server();
2607        let region_id_1 = RegionId::new(1, 0);
2608        let region_id_2 = RegionId::new(2, 0);
2609
2610        let metadata_1 = mock_region_metadata(region_id_1);
2611        let metadatas = vec![Some(metadata_1.clone()), None];
2612
2613        let metadata_1 = Arc::new(metadata_1);
2614        let (engine, _) = MockRegionEngine::with_metadata_mock_fn(
2615            MITO_ENGINE_NAME,
2616            Box::new(move |region_id| {
2617                if region_id == region_id_1 {
2618                    Ok(metadata_1.clone())
2619                } else {
2620                    error::RegionNotFoundSnafu { region_id }.fail()
2621                }
2622            }),
2623        );
2624
2625        mock_region_server.register_engine(engine.clone());
2626        mock_region_server
2627            .inner
2628            .region_map
2629            .insert(region_id_1, RegionEngineWithStatus::Ready(engine.clone()));
2630
2631        // Not in region map.
2632        let list_metadata_request = ListMetadataRequest {
2633            region_ids: vec![region_id_1.as_u64(), region_id_2.as_u64()],
2634        };
2635        let response = mock_region_server
2636            .handle_list_metadata_request(&list_metadata_request)
2637            .await
2638            .unwrap();
2639        let decoded_metadata: Vec<Option<RegionMetadata>> =
2640            serde_json::from_slice(&response.metadata).unwrap();
2641        assert_eq!(metadatas, decoded_metadata);
2642
2643        // Not in region engine.
2644        mock_region_server
2645            .inner
2646            .region_map
2647            .insert(region_id_2, RegionEngineWithStatus::Ready(engine.clone()));
2648        let response = mock_region_server
2649            .handle_list_metadata_request(&list_metadata_request)
2650            .await
2651            .unwrap();
2652        let decoded_metadata: Vec<Option<RegionMetadata>> =
2653            serde_json::from_slice(&response.metadata).unwrap();
2654        assert_eq!(metadatas, decoded_metadata);
2655    }
2656
2657    #[tokio::test]
2658    async fn test_handle_list_metadata_failed() {
2659        common_telemetry::init_default_ut_logging();
2660
2661        let mut mock_region_server = mock_region_server();
2662        let region_id_1 = RegionId::new(1, 0);
2663
2664        let (engine, _) = MockRegionEngine::with_metadata_mock_fn(
2665            MITO_ENGINE_NAME,
2666            Box::new(move |region_id| {
2667                error::UnexpectedSnafu {
2668                    violated: format!("Failed to get region {region_id}"),
2669                }
2670                .fail()
2671            }),
2672        );
2673
2674        mock_region_server.register_engine(engine.clone());
2675        mock_region_server
2676            .inner
2677            .region_map
2678            .insert(region_id_1, RegionEngineWithStatus::Ready(engine.clone()));
2679
2680        // Failed to get.
2681        let list_metadata_request = ListMetadataRequest {
2682            region_ids: vec![region_id_1.as_u64()],
2683        };
2684        mock_region_server
2685            .handle_list_metadata_request(&list_metadata_request)
2686            .await
2687            .unwrap_err();
2688    }
2689
2690    #[tokio::test]
2691    async fn test_handle_remote_dyn_filter_request_requires_query_id() {
2692        let mock_region_server = mock_region_server();
2693
2694        let err = mock_region_server
2695            .handle_remote_dyn_filter_request(&RemoteDynFilterRequest {
2696                query_id: String::new(),
2697                action: Some(remote_dyn_filter_request::Action::Unregister(
2698                    RemoteDynFilterUnregister {
2699                        filter_id: "filter-1".to_string(),
2700                    },
2701                )),
2702            })
2703            .await
2704            .unwrap_err();
2705
2706        assert_matches!(
2707            err,
2708            crate::error::Error::MissingRequiredField { ref name, .. } if name == "query_id"
2709        );
2710    }
2711
2712    #[tokio::test]
2713    async fn test_handle_remote_dyn_filter_request_requires_action() {
2714        let mock_region_server = mock_region_server();
2715
2716        let err = mock_region_server
2717            .handle_remote_dyn_filter_request(&RemoteDynFilterRequest {
2718                query_id: "query-1".to_string(),
2719                action: None,
2720            })
2721            .await
2722            .unwrap_err();
2723
2724        assert_matches!(
2725            err,
2726            crate::error::Error::MissingRequiredField { ref name, .. } if name == "action"
2727        );
2728    }
2729
2730    #[tokio::test]
2731    async fn test_handle_remote_dyn_filter_update_requires_filter_id() {
2732        let mock_region_server = mock_region_server();
2733
2734        let err = mock_region_server
2735            .handle_remote_dyn_filter_request(&RemoteDynFilterRequest {
2736                query_id: "query-1".to_string(),
2737                action: Some(remote_dyn_filter_request::Action::Update(
2738                    RemoteDynFilterUpdate {
2739                        filter_id: String::new(),
2740                        payload: vec![1],
2741                        generation: 1,
2742                        is_complete: false,
2743                    },
2744                )),
2745            })
2746            .await
2747            .unwrap_err();
2748
2749        assert_matches!(
2750            err,
2751            crate::error::Error::MissingRequiredField { ref name, .. } if name == "filter_id"
2752        );
2753    }
2754
2755    #[tokio::test]
2756    async fn test_handle_remote_dyn_filter_update_requires_payload() {
2757        let mock_region_server = mock_region_server();
2758
2759        let err = mock_region_server
2760            .handle_remote_dyn_filter_request(&RemoteDynFilterRequest {
2761                query_id: "query-1".to_string(),
2762                action: Some(remote_dyn_filter_request::Action::Update(
2763                    RemoteDynFilterUpdate {
2764                        filter_id: "filter-1".to_string(),
2765                        payload: Vec::new(),
2766                        generation: 1,
2767                        is_complete: false,
2768                    },
2769                )),
2770            })
2771            .await
2772            .unwrap_err();
2773
2774        assert_matches!(
2775            err,
2776            crate::error::Error::MissingRequiredField { ref name, .. } if name == "payload"
2777        );
2778    }
2779
2780    #[tokio::test]
2781    async fn test_handle_remote_dyn_filter_update_placeholder() {
2782        let mock_region_server = mock_region_server();
2783
2784        let err = mock_region_server
2785            .handle_remote_dyn_filter_request(&RemoteDynFilterRequest {
2786                query_id: "query-1".to_string(),
2787                action: Some(remote_dyn_filter_request::Action::Update(
2788                    RemoteDynFilterUpdate {
2789                        filter_id: "filter-1".to_string(),
2790                        payload: vec![1],
2791                        generation: 1,
2792                        is_complete: false,
2793                    },
2794                )),
2795            })
2796            .await
2797            .unwrap_err();
2798
2799        assert_matches!(err, crate::error::Error::NotYetImplemented { .. });
2800    }
2801
2802    #[tokio::test]
2803    async fn test_handle_remote_dyn_filter_unregister_placeholder() {
2804        let mock_region_server = mock_region_server();
2805
2806        let err = mock_region_server
2807            .handle_remote_dyn_filter_request(&RemoteDynFilterRequest {
2808                query_id: "query-1".to_string(),
2809                action: Some(remote_dyn_filter_request::Action::Unregister(
2810                    RemoteDynFilterUnregister {
2811                        filter_id: "filter-1".to_string(),
2812                    },
2813                )),
2814            })
2815            .await
2816            .unwrap_err();
2817
2818        assert_matches!(err, crate::error::Error::NotYetImplemented { .. });
2819    }
2820}