Skip to main content

datanode/
region_server.rs

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