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