1mod handle_alter;
18mod handle_bulk_insert;
19mod handle_catchup;
20mod handle_close;
21mod handle_compaction;
22mod handle_create;
23mod handle_drop;
24mod handle_flush;
25mod handle_manifest;
26mod handle_open;
27mod handle_rebuild_index;
28mod handle_truncate;
29mod handle_write;
30
31use std::collections::HashMap;
32use std::path::Path;
33use std::sync::Arc;
34use std::sync::atomic::{AtomicBool, Ordering};
35use std::time::Duration;
36
37use common_base::Plugins;
38use common_error::ext::BoxedError;
39use common_meta::key::SchemaMetadataManagerRef;
40use common_runtime::JoinHandle;
41use common_telemetry::{error, info, warn};
42use futures::future::try_join_all;
43use object_store::manager::ObjectStoreManagerRef;
44use prometheus::{Histogram, IntGauge};
45use rand::{Rng, rng};
46use snafu::{ResultExt, ensure};
47use store_api::logstore::LogStore;
48use store_api::region_engine::{
49 SetRegionRoleStateResponse, SetRegionRoleStateSuccess, SettableRegionRoleState,
50};
51use store_api::storage::{FileId, RegionId};
52use tokio::sync::mpsc::{Receiver, Sender};
53use tokio::sync::{Mutex, Semaphore, mpsc, oneshot, watch};
54
55use crate::cache::write_cache::{WriteCache, WriteCacheRef};
56use crate::cache::{CacheManager, CacheManagerRef};
57use crate::compaction::CompactionScheduler;
58use crate::config::MitoConfig;
59use crate::error::{self, CreateDirSnafu, JoinSnafu, Result, WorkerStoppedSnafu};
60use crate::flush::{FlushScheduler, WriteBufferManagerImpl, WriteBufferManagerRef};
61use crate::gc::{GcLimiter, GcLimiterRef};
62use crate::memtable::MemtableBuilderProvider;
63use crate::metrics::{REGION_COUNT, REQUEST_WAIT_TIME, WRITE_STALLING};
64use crate::region::opener::PartitionExprFetcherRef;
65use crate::region::{
66 CatchupRegions, CatchupRegionsRef, MitoRegionRef, OpeningRegions, OpeningRegionsRef, RegionMap,
67 RegionMapRef,
68};
69use crate::request::{
70 BackgroundNotify, DdlRequest, SenderBulkRequest, SenderDdlRequest, SenderWriteRequest,
71 WorkerRequest, WorkerRequestWithTime,
72};
73use crate::schedule::scheduler::{LocalScheduler, SchedulerRef};
74use crate::sst::file::RegionFileId;
75use crate::sst::file_ref::FileReferenceManagerRef;
76use crate::sst::index::IndexBuildScheduler;
77use crate::sst::index::intermediate::IntermediateManager;
78use crate::sst::index::puffin_manager::PuffinManagerFactory;
79use crate::time_provider::{StdTimeProvider, TimeProviderRef};
80use crate::wal::Wal;
81use crate::worker::handle_manifest::RegionEditQueues;
82
83pub(crate) type WorkerId = u32;
85
86pub(crate) const DROPPING_MARKER_FILE: &str = ".dropping";
87
88pub(crate) const CHECK_REGION_INTERVAL: Duration = Duration::from_secs(60);
90pub(crate) const MAX_INITIAL_CHECK_DELAY_SECS: u64 = 60 * 3;
92
93#[cfg_attr(doc, aquamarine::aquamarine)]
94pub(crate) struct WorkerGroup {
131 workers: Vec<RegionWorker>,
133 flush_job_pool: SchedulerRef,
135 compact_job_pool: SchedulerRef,
137 index_build_job_pool: SchedulerRef,
139 purge_scheduler: SchedulerRef,
141 cache_manager: CacheManagerRef,
143 file_ref_manager: FileReferenceManagerRef,
145 gc_limiter: GcLimiterRef,
147}
148
149impl WorkerGroup {
150 pub(crate) async fn start<S: LogStore>(
154 config: Arc<MitoConfig>,
155 log_store: Arc<S>,
156 object_store_manager: ObjectStoreManagerRef,
157 schema_metadata_manager: SchemaMetadataManagerRef,
158 file_ref_manager: FileReferenceManagerRef,
159 partition_expr_fetcher: PartitionExprFetcherRef,
160 plugins: Plugins,
161 ) -> Result<WorkerGroup> {
162 let (flush_sender, flush_receiver) = watch::channel(());
163 let write_buffer_manager = Arc::new(
164 WriteBufferManagerImpl::new(config.global_write_buffer_size.as_bytes() as usize)
165 .with_notifier(flush_sender.clone()),
166 );
167 let puffin_manager_factory = PuffinManagerFactory::new(
168 &config.index.aux_path,
169 config.index.staging_size.as_bytes(),
170 Some(config.index.write_buffer_size.as_bytes() as _),
171 config.index.staging_ttl,
172 )
173 .await?;
174 let intermediate_manager = IntermediateManager::init_fs(&config.index.aux_path)
175 .await?
176 .with_buffer_size(Some(config.index.write_buffer_size.as_bytes() as _));
177 let index_build_job_pool =
178 Arc::new(LocalScheduler::new(config.max_background_index_builds));
179 let flush_job_pool = Arc::new(LocalScheduler::new(config.max_background_flushes));
180 let compact_job_pool = Arc::new(LocalScheduler::new(config.max_background_compactions));
181 let flush_semaphore = Arc::new(Semaphore::new(config.max_background_flushes));
182 let purge_scheduler = Arc::new(LocalScheduler::new(config.max_background_purges));
184 let write_cache = write_cache_from_config(
185 &config,
186 puffin_manager_factory.clone(),
187 intermediate_manager.clone(),
188 )
189 .await?;
190 let cache_manager = Arc::new(
191 CacheManager::builder()
192 .sst_meta_cache_size(config.sst_meta_cache_size.as_bytes())
193 .vector_cache_size(config.vector_cache_size.as_bytes())
194 .page_cache_size(config.page_cache_size.as_bytes())
195 .selector_result_cache_size(config.selector_result_cache_size.as_bytes())
196 .index_metadata_size(config.index.metadata_cache_size.as_bytes())
197 .index_content_size(config.index.content_cache_size.as_bytes())
198 .index_content_page_size(config.index.content_cache_page_size.as_bytes())
199 .index_result_cache_size(config.index.result_cache_size.as_bytes())
200 .puffin_metadata_size(config.index.metadata_cache_size.as_bytes())
201 .write_cache(write_cache)
202 .build(),
203 );
204 let time_provider = Arc::new(StdTimeProvider);
205 let gc_limiter = Arc::new(GcLimiter::new(config.gc.max_concurrent_gc_job));
206
207 let workers = (0..config.num_workers)
208 .map(|id| {
209 WorkerStarter {
210 id: id as WorkerId,
211 config: config.clone(),
212 log_store: log_store.clone(),
213 object_store_manager: object_store_manager.clone(),
214 write_buffer_manager: write_buffer_manager.clone(),
215 index_build_job_pool: index_build_job_pool.clone(),
216 flush_job_pool: flush_job_pool.clone(),
217 compact_job_pool: compact_job_pool.clone(),
218 purge_scheduler: purge_scheduler.clone(),
219 listener: WorkerListener::default(),
220 cache_manager: cache_manager.clone(),
221 puffin_manager_factory: puffin_manager_factory.clone(),
222 intermediate_manager: intermediate_manager.clone(),
223 time_provider: time_provider.clone(),
224 flush_sender: flush_sender.clone(),
225 flush_receiver: flush_receiver.clone(),
226 plugins: plugins.clone(),
227 schema_metadata_manager: schema_metadata_manager.clone(),
228 file_ref_manager: file_ref_manager.clone(),
229 partition_expr_fetcher: partition_expr_fetcher.clone(),
230 flush_semaphore: flush_semaphore.clone(),
231 }
232 .start()
233 })
234 .collect::<Result<Vec<_>>>()?;
235
236 Ok(WorkerGroup {
237 workers,
238 flush_job_pool,
239 compact_job_pool,
240 index_build_job_pool,
241 purge_scheduler,
242 cache_manager,
243 file_ref_manager,
244 gc_limiter,
245 })
246 }
247
248 pub(crate) async fn stop(&self) -> Result<()> {
250 info!("Stop region worker group");
251
252 self.compact_job_pool.stop(true).await?;
255 self.flush_job_pool.stop(true).await?;
257 self.purge_scheduler.stop(true).await?;
259 self.index_build_job_pool.stop(true).await?;
261
262 try_join_all(self.workers.iter().map(|worker| worker.stop())).await?;
263
264 Ok(())
265 }
266
267 pub(crate) async fn submit_to_worker(
269 &self,
270 region_id: RegionId,
271 request: WorkerRequest,
272 ) -> Result<()> {
273 self.worker(region_id).submit_request(request).await
274 }
275
276 pub(crate) fn is_region_exists(&self, region_id: RegionId) -> bool {
278 self.worker(region_id).is_region_exists(region_id)
279 }
280
281 pub(crate) fn is_region_opening(&self, region_id: RegionId) -> bool {
283 self.worker(region_id).is_region_opening(region_id)
284 }
285
286 pub(crate) fn is_region_catching_up(&self, region_id: RegionId) -> bool {
288 self.worker(region_id).is_region_catching_up(region_id)
289 }
290
291 pub(crate) fn get_region(&self, region_id: RegionId) -> Option<MitoRegionRef> {
295 self.worker(region_id).get_region(region_id)
296 }
297
298 pub(crate) fn cache_manager(&self) -> CacheManagerRef {
300 self.cache_manager.clone()
301 }
302
303 pub(crate) fn file_ref_manager(&self) -> FileReferenceManagerRef {
304 self.file_ref_manager.clone()
305 }
306
307 pub(crate) fn gc_limiter(&self) -> GcLimiterRef {
308 self.gc_limiter.clone()
309 }
310
311 pub(crate) fn worker(&self, region_id: RegionId) -> &RegionWorker {
313 let index = region_id_to_index(region_id, self.workers.len());
314
315 &self.workers[index]
316 }
317
318 pub(crate) fn all_regions(&self) -> impl Iterator<Item = MitoRegionRef> + use<'_> {
319 self.workers
320 .iter()
321 .flat_map(|worker| worker.regions.list_regions())
322 }
323}
324
325#[cfg(any(test, feature = "test"))]
327impl WorkerGroup {
328 #[allow(clippy::too_many_arguments)]
332 pub(crate) async fn start_for_test<S: LogStore>(
333 config: Arc<MitoConfig>,
334 log_store: Arc<S>,
335 object_store_manager: ObjectStoreManagerRef,
336 write_buffer_manager: Option<WriteBufferManagerRef>,
337 listener: Option<crate::engine::listener::EventListenerRef>,
338 schema_metadata_manager: SchemaMetadataManagerRef,
339 file_ref_manager: FileReferenceManagerRef,
340 time_provider: TimeProviderRef,
341 partition_expr_fetcher: PartitionExprFetcherRef,
342 ) -> Result<WorkerGroup> {
343 let (flush_sender, flush_receiver) = watch::channel(());
344 let write_buffer_manager = write_buffer_manager.unwrap_or_else(|| {
345 Arc::new(
346 WriteBufferManagerImpl::new(config.global_write_buffer_size.as_bytes() as usize)
347 .with_notifier(flush_sender.clone()),
348 )
349 });
350 let index_build_job_pool =
351 Arc::new(LocalScheduler::new(config.max_background_index_builds));
352 let flush_job_pool = Arc::new(LocalScheduler::new(config.max_background_flushes));
353 let compact_job_pool = Arc::new(LocalScheduler::new(config.max_background_compactions));
354 let flush_semaphore = Arc::new(Semaphore::new(config.max_background_flushes));
355 let purge_scheduler = Arc::new(LocalScheduler::new(config.max_background_flushes));
356 let puffin_manager_factory = PuffinManagerFactory::new(
357 &config.index.aux_path,
358 config.index.staging_size.as_bytes(),
359 Some(config.index.write_buffer_size.as_bytes() as _),
360 config.index.staging_ttl,
361 )
362 .await?;
363 let intermediate_manager = IntermediateManager::init_fs(&config.index.aux_path)
364 .await?
365 .with_buffer_size(Some(config.index.write_buffer_size.as_bytes() as _));
366 let write_cache = write_cache_from_config(
367 &config,
368 puffin_manager_factory.clone(),
369 intermediate_manager.clone(),
370 )
371 .await?;
372 let cache_manager = Arc::new(
373 CacheManager::builder()
374 .sst_meta_cache_size(config.sst_meta_cache_size.as_bytes())
375 .vector_cache_size(config.vector_cache_size.as_bytes())
376 .page_cache_size(config.page_cache_size.as_bytes())
377 .selector_result_cache_size(config.selector_result_cache_size.as_bytes())
378 .write_cache(write_cache)
379 .build(),
380 );
381 let gc_limiter = Arc::new(GcLimiter::new(config.gc.max_concurrent_gc_job));
382 let workers = (0..config.num_workers)
383 .map(|id| {
384 WorkerStarter {
385 id: id as WorkerId,
386 config: config.clone(),
387 log_store: log_store.clone(),
388 object_store_manager: object_store_manager.clone(),
389 write_buffer_manager: write_buffer_manager.clone(),
390 index_build_job_pool: index_build_job_pool.clone(),
391 flush_job_pool: flush_job_pool.clone(),
392 compact_job_pool: compact_job_pool.clone(),
393 purge_scheduler: purge_scheduler.clone(),
394 listener: WorkerListener::new(listener.clone()),
395 cache_manager: cache_manager.clone(),
396 puffin_manager_factory: puffin_manager_factory.clone(),
397 intermediate_manager: intermediate_manager.clone(),
398 time_provider: time_provider.clone(),
399 flush_sender: flush_sender.clone(),
400 flush_receiver: flush_receiver.clone(),
401 plugins: Plugins::new(),
402 schema_metadata_manager: schema_metadata_manager.clone(),
403 file_ref_manager: file_ref_manager.clone(),
404 partition_expr_fetcher: partition_expr_fetcher.clone(),
405 flush_semaphore: flush_semaphore.clone(),
406 }
407 .start()
408 })
409 .collect::<Result<Vec<_>>>()?;
410
411 Ok(WorkerGroup {
412 workers,
413 flush_job_pool,
414 compact_job_pool,
415 index_build_job_pool,
416 purge_scheduler,
417 cache_manager,
418 file_ref_manager,
419 gc_limiter,
420 })
421 }
422
423 pub(crate) fn purge_scheduler(&self) -> &SchedulerRef {
425 &self.purge_scheduler
426 }
427}
428
429fn region_id_to_index(id: RegionId, num_workers: usize) -> usize {
430 ((id.table_id() as usize % num_workers) + (id.region_number() as usize % num_workers))
431 % num_workers
432}
433
434pub async fn write_cache_from_config(
435 config: &MitoConfig,
436 puffin_manager_factory: PuffinManagerFactory,
437 intermediate_manager: IntermediateManager,
438) -> Result<Option<WriteCacheRef>> {
439 if !config.enable_write_cache {
440 return Ok(None);
441 }
442
443 tokio::fs::create_dir_all(Path::new(&config.write_cache_path))
444 .await
445 .context(CreateDirSnafu {
446 dir: &config.write_cache_path,
447 })?;
448
449 let cache = WriteCache::new_fs(
450 &config.write_cache_path,
451 config.write_cache_size,
452 config.write_cache_ttl,
453 puffin_manager_factory,
454 intermediate_manager,
455 )
456 .await?;
457 Ok(Some(Arc::new(cache)))
458}
459
460pub(crate) fn worker_init_check_delay() -> Duration {
462 let init_check_delay = rng().random_range(0..MAX_INITIAL_CHECK_DELAY_SECS);
463 Duration::from_secs(init_check_delay)
464}
465
466struct WorkerStarter<S> {
468 id: WorkerId,
469 config: Arc<MitoConfig>,
470 log_store: Arc<S>,
471 object_store_manager: ObjectStoreManagerRef,
472 write_buffer_manager: WriteBufferManagerRef,
473 compact_job_pool: SchedulerRef,
474 index_build_job_pool: SchedulerRef,
475 flush_job_pool: SchedulerRef,
476 purge_scheduler: SchedulerRef,
477 listener: WorkerListener,
478 cache_manager: CacheManagerRef,
479 puffin_manager_factory: PuffinManagerFactory,
480 intermediate_manager: IntermediateManager,
481 time_provider: TimeProviderRef,
482 flush_sender: watch::Sender<()>,
484 flush_receiver: watch::Receiver<()>,
486 plugins: Plugins,
487 schema_metadata_manager: SchemaMetadataManagerRef,
488 file_ref_manager: FileReferenceManagerRef,
489 partition_expr_fetcher: PartitionExprFetcherRef,
490 flush_semaphore: Arc<Semaphore>,
491}
492
493impl<S: LogStore> WorkerStarter<S> {
494 fn start(self) -> Result<RegionWorker> {
496 let regions = Arc::new(RegionMap::default());
497 let opening_regions = Arc::new(OpeningRegions::default());
498 let catchup_regions = Arc::new(CatchupRegions::default());
499 let (sender, receiver) = mpsc::channel(self.config.worker_channel_size);
500
501 let running = Arc::new(AtomicBool::new(true));
502 let now = self.time_provider.current_time_millis();
503 let id_string = self.id.to_string();
504 let mut worker_thread = RegionWorkerLoop {
505 id: self.id,
506 config: self.config.clone(),
507 regions: regions.clone(),
508 catchup_regions: catchup_regions.clone(),
509 dropping_regions: Arc::new(RegionMap::default()),
510 opening_regions: opening_regions.clone(),
511 sender: sender.clone(),
512 receiver,
513 wal: Wal::new(self.log_store),
514 object_store_manager: self.object_store_manager.clone(),
515 running: running.clone(),
516 memtable_builder_provider: MemtableBuilderProvider::new(
517 Some(self.write_buffer_manager.clone()),
518 self.config.clone(),
519 ),
520 purge_scheduler: self.purge_scheduler.clone(),
521 write_buffer_manager: self.write_buffer_manager,
522 index_build_scheduler: IndexBuildScheduler::new(
523 self.index_build_job_pool,
524 self.config.max_background_index_builds,
525 ),
526 flush_scheduler: FlushScheduler::new(self.flush_job_pool),
527 compaction_scheduler: CompactionScheduler::new(
528 self.compact_job_pool,
529 sender.clone(),
530 self.cache_manager.clone(),
531 self.config,
532 self.listener.clone(),
533 self.plugins.clone(),
534 ),
535 stalled_requests: StalledRequests::default(),
536 listener: self.listener,
537 cache_manager: self.cache_manager,
538 puffin_manager_factory: self.puffin_manager_factory,
539 intermediate_manager: self.intermediate_manager,
540 time_provider: self.time_provider,
541 last_periodical_check_millis: now,
542 flush_sender: self.flush_sender,
543 flush_receiver: self.flush_receiver,
544 stalling_count: WRITE_STALLING.with_label_values(&[&id_string]),
545 region_count: REGION_COUNT.with_label_values(&[&id_string]),
546 request_wait_time: REQUEST_WAIT_TIME.with_label_values(&[&id_string]),
547 region_edit_queues: RegionEditQueues::default(),
548 schema_metadata_manager: self.schema_metadata_manager,
549 file_ref_manager: self.file_ref_manager.clone(),
550 partition_expr_fetcher: self.partition_expr_fetcher,
551 flush_semaphore: self.flush_semaphore,
552 };
553 let handle = common_runtime::spawn_global(async move {
554 worker_thread.run().await;
555 });
556
557 Ok(RegionWorker {
558 id: self.id,
559 regions,
560 opening_regions,
561 catchup_regions,
562 sender,
563 handle: Mutex::new(Some(handle)),
564 running,
565 })
566 }
567}
568
569pub(crate) struct RegionWorker {
571 id: WorkerId,
573 regions: RegionMapRef,
575 opening_regions: OpeningRegionsRef,
577 catchup_regions: CatchupRegionsRef,
579 sender: Sender<WorkerRequestWithTime>,
581 handle: Mutex<Option<JoinHandle<()>>>,
583 running: Arc<AtomicBool>,
585}
586
587impl RegionWorker {
588 async fn submit_request(&self, request: WorkerRequest) -> Result<()> {
590 ensure!(self.is_running(), WorkerStoppedSnafu { id: self.id });
591 let request_with_time = WorkerRequestWithTime::new(request);
592 if self.sender.send(request_with_time).await.is_err() {
593 warn!(
594 "Worker {} is already exited but the running flag is still true",
595 self.id
596 );
597 self.set_running(false);
599 return WorkerStoppedSnafu { id: self.id }.fail();
600 }
601
602 Ok(())
603 }
604
605 async fn stop(&self) -> Result<()> {
609 let handle = self.handle.lock().await.take();
610 if let Some(handle) = handle {
611 info!("Stop region worker {}", self.id);
612
613 self.set_running(false);
614 if self
615 .sender
616 .send(WorkerRequestWithTime::new(WorkerRequest::Stop))
617 .await
618 .is_err()
619 {
620 warn!("Worker {} is already exited before stop", self.id);
621 }
622
623 handle.await.context(JoinSnafu)?;
624 }
625
626 Ok(())
627 }
628
629 fn is_running(&self) -> bool {
631 self.running.load(Ordering::Relaxed)
632 }
633
634 fn set_running(&self, value: bool) {
636 self.running.store(value, Ordering::Relaxed)
637 }
638
639 fn is_region_exists(&self, region_id: RegionId) -> bool {
641 self.regions.is_region_exists(region_id)
642 }
643
644 fn is_region_opening(&self, region_id: RegionId) -> bool {
646 self.opening_regions.is_region_exists(region_id)
647 }
648
649 fn is_region_catching_up(&self, region_id: RegionId) -> bool {
651 self.catchup_regions.is_region_exists(region_id)
652 }
653
654 fn get_region(&self, region_id: RegionId) -> Option<MitoRegionRef> {
656 self.regions.get_region(region_id)
657 }
658
659 #[cfg(test)]
660 pub(crate) fn opening_regions(&self) -> &OpeningRegionsRef {
662 &self.opening_regions
663 }
664
665 #[cfg(test)]
666 pub(crate) fn catchup_regions(&self) -> &CatchupRegionsRef {
668 &self.catchup_regions
669 }
670}
671
672impl Drop for RegionWorker {
673 fn drop(&mut self) {
674 if self.is_running() {
675 self.set_running(false);
676 }
678 }
679}
680
681type RequestBuffer = Vec<WorkerRequest>;
682
683#[derive(Default)]
687pub(crate) struct StalledRequests {
688 pub(crate) requests:
695 HashMap<RegionId, (usize, Vec<SenderWriteRequest>, Vec<SenderBulkRequest>)>,
696 pub(crate) estimated_size: usize,
698}
699
700impl StalledRequests {
701 pub(crate) fn append(
703 &mut self,
704 requests: &mut Vec<SenderWriteRequest>,
705 bulk_requests: &mut Vec<SenderBulkRequest>,
706 ) {
707 for req in requests.drain(..) {
708 self.push(req);
709 }
710 for req in bulk_requests.drain(..) {
711 self.push_bulk(req);
712 }
713 }
714
715 pub(crate) fn push(&mut self, req: SenderWriteRequest) {
717 let (size, requests, _) = self.requests.entry(req.request.region_id).or_default();
718 let req_size = req.request.estimated_size();
719 *size += req_size;
720 self.estimated_size += req_size;
721 requests.push(req);
722 }
723
724 pub(crate) fn push_bulk(&mut self, req: SenderBulkRequest) {
725 let region_id = req.region_id;
726 let (size, _, requests) = self.requests.entry(region_id).or_default();
727 let req_size = req.request.estimated_size();
728 *size += req_size;
729 self.estimated_size += req_size;
730 requests.push(req);
731 }
732
733 pub(crate) fn remove(
735 &mut self,
736 region_id: &RegionId,
737 ) -> (Vec<SenderWriteRequest>, Vec<SenderBulkRequest>) {
738 if let Some((size, write_reqs, bulk_reqs)) = self.requests.remove(region_id) {
739 self.estimated_size -= size;
740 (write_reqs, bulk_reqs)
741 } else {
742 (vec![], vec![])
743 }
744 }
745
746 pub(crate) fn stalled_count(&self) -> usize {
748 self.requests
749 .values()
750 .map(|(_, reqs, bulk_reqs)| reqs.len() + bulk_reqs.len())
751 .sum()
752 }
753}
754
755struct RegionWorkerLoop<S> {
757 id: WorkerId,
759 config: Arc<MitoConfig>,
761 regions: RegionMapRef,
763 dropping_regions: RegionMapRef,
765 opening_regions: OpeningRegionsRef,
767 catchup_regions: CatchupRegionsRef,
769 sender: Sender<WorkerRequestWithTime>,
771 receiver: Receiver<WorkerRequestWithTime>,
773 wal: Wal<S>,
775 object_store_manager: ObjectStoreManagerRef,
777 running: Arc<AtomicBool>,
779 memtable_builder_provider: MemtableBuilderProvider,
781 purge_scheduler: SchedulerRef,
783 write_buffer_manager: WriteBufferManagerRef,
785 index_build_scheduler: IndexBuildScheduler,
787 flush_scheduler: FlushScheduler,
789 compaction_scheduler: CompactionScheduler,
791 stalled_requests: StalledRequests,
793 listener: WorkerListener,
795 cache_manager: CacheManagerRef,
797 puffin_manager_factory: PuffinManagerFactory,
799 intermediate_manager: IntermediateManager,
801 time_provider: TimeProviderRef,
803 last_periodical_check_millis: i64,
805 flush_sender: watch::Sender<()>,
807 flush_receiver: watch::Receiver<()>,
809 stalling_count: IntGauge,
811 region_count: IntGauge,
813 request_wait_time: Histogram,
815 region_edit_queues: RegionEditQueues,
817 schema_metadata_manager: SchemaMetadataManagerRef,
819 file_ref_manager: FileReferenceManagerRef,
821 partition_expr_fetcher: PartitionExprFetcherRef,
823 flush_semaphore: Arc<Semaphore>,
825}
826
827impl<S: LogStore> RegionWorkerLoop<S> {
828 async fn run(&mut self) {
830 let init_check_delay = worker_init_check_delay();
831 info!(
832 "Start region worker thread {}, init_check_delay: {:?}",
833 self.id, init_check_delay
834 );
835 self.last_periodical_check_millis += init_check_delay.as_millis() as i64;
836
837 let mut write_req_buffer: Vec<SenderWriteRequest> =
839 Vec::with_capacity(self.config.worker_request_batch_size);
840 let mut bulk_req_buffer: Vec<SenderBulkRequest> =
841 Vec::with_capacity(self.config.worker_request_batch_size);
842 let mut ddl_req_buffer: Vec<SenderDdlRequest> =
843 Vec::with_capacity(self.config.worker_request_batch_size);
844 let mut general_req_buffer: Vec<WorkerRequest> =
845 RequestBuffer::with_capacity(self.config.worker_request_batch_size);
846
847 while self.running.load(Ordering::Relaxed) {
848 write_req_buffer.clear();
850 ddl_req_buffer.clear();
851 general_req_buffer.clear();
852
853 let max_wait_time = self.time_provider.wait_duration(CHECK_REGION_INTERVAL);
854 let sleep = tokio::time::sleep(max_wait_time);
855 tokio::pin!(sleep);
856
857 tokio::select! {
858 request_opt = self.receiver.recv() => {
859 match request_opt {
860 Some(request_with_time) => {
861 let wait_time = request_with_time.created_at.elapsed();
863 self.request_wait_time.observe(wait_time.as_secs_f64());
864
865 match request_with_time.request {
866 WorkerRequest::Write(sender_req) => write_req_buffer.push(sender_req),
867 WorkerRequest::Ddl(sender_req) => ddl_req_buffer.push(sender_req),
868 req => general_req_buffer.push(req),
869 }
870 },
871 None => break,
873 }
874 }
875 recv_res = self.flush_receiver.changed() => {
876 if recv_res.is_err() {
877 break;
879 } else {
880 self.maybe_flush_worker();
885 self.handle_stalled_requests().await;
887 continue;
888 }
889 }
890 _ = &mut sleep => {
891 self.handle_periodical_tasks();
893 continue;
894 }
895 }
896
897 if self.flush_receiver.has_changed().unwrap_or(false) {
898 self.handle_stalled_requests().await;
902 }
903
904 for _ in 1..self.config.worker_request_batch_size {
906 match self.receiver.try_recv() {
908 Ok(request_with_time) => {
909 let wait_time = request_with_time.created_at.elapsed();
911 self.request_wait_time.observe(wait_time.as_secs_f64());
912
913 match request_with_time.request {
914 WorkerRequest::Write(sender_req) => write_req_buffer.push(sender_req),
915 WorkerRequest::Ddl(sender_req) => ddl_req_buffer.push(sender_req),
916 req => general_req_buffer.push(req),
917 }
918 }
919 Err(_) => break,
921 }
922 }
923
924 self.listener.on_recv_requests(
925 write_req_buffer.len() + ddl_req_buffer.len() + general_req_buffer.len(),
926 );
927
928 self.handle_requests(
929 &mut write_req_buffer,
930 &mut ddl_req_buffer,
931 &mut general_req_buffer,
932 &mut bulk_req_buffer,
933 )
934 .await;
935
936 self.handle_periodical_tasks();
937 }
938
939 self.clean().await;
940
941 info!("Exit region worker thread {}", self.id);
942 }
943
944 async fn handle_requests(
948 &mut self,
949 write_requests: &mut Vec<SenderWriteRequest>,
950 ddl_requests: &mut Vec<SenderDdlRequest>,
951 general_requests: &mut Vec<WorkerRequest>,
952 bulk_requests: &mut Vec<SenderBulkRequest>,
953 ) {
954 for worker_req in general_requests.drain(..) {
955 match worker_req {
956 WorkerRequest::Write(_) | WorkerRequest::Ddl(_) => {
957 continue;
959 }
960 WorkerRequest::Background { region_id, notify } => {
961 self.handle_background_notify(region_id, notify).await;
963 }
964 WorkerRequest::SetRegionRoleStateGracefully {
965 region_id,
966 region_role_state,
967 sender,
968 } => {
969 self.set_role_state_gracefully(region_id, region_role_state, sender)
970 .await;
971 }
972 WorkerRequest::EditRegion(request) => {
973 self.handle_region_edit(request).await;
974 }
975 WorkerRequest::Stop => {
976 debug_assert!(!self.running.load(Ordering::Relaxed));
977 }
978 WorkerRequest::SyncRegion(req) => {
979 self.handle_region_sync(req).await;
980 }
981 WorkerRequest::BulkInserts {
982 metadata,
983 request,
984 sender,
985 } => {
986 if let Some(region_metadata) = metadata {
987 self.handle_bulk_insert_batch(
988 region_metadata,
989 request,
990 bulk_requests,
991 sender,
992 )
993 .await;
994 } else {
995 error!("Cannot find region metadata for {}", request.region_id);
996 sender.send(
997 error::RegionNotFoundSnafu {
998 region_id: request.region_id,
999 }
1000 .fail(),
1001 );
1002 }
1003 }
1004 }
1005 }
1006
1007 self.handle_write_requests(write_requests, bulk_requests, true)
1010 .await;
1011
1012 self.handle_ddl_requests(ddl_requests).await;
1013 }
1014
1015 async fn handle_ddl_requests(&mut self, ddl_requests: &mut Vec<SenderDdlRequest>) {
1017 if ddl_requests.is_empty() {
1018 return;
1019 }
1020
1021 for ddl in ddl_requests.drain(..) {
1022 let res = match ddl.request {
1023 DdlRequest::Create(req) => self.handle_create_request(ddl.region_id, req).await,
1024 DdlRequest::Drop => self.handle_drop_request(ddl.region_id).await,
1025 DdlRequest::Open((req, wal_entry_receiver)) => {
1026 self.handle_open_request(ddl.region_id, req, wal_entry_receiver, ddl.sender)
1027 .await;
1028 continue;
1029 }
1030 DdlRequest::Close(_) => self.handle_close_request(ddl.region_id).await,
1031 DdlRequest::Alter(req) => {
1032 self.handle_alter_request(ddl.region_id, req, ddl.sender)
1033 .await;
1034 continue;
1035 }
1036 DdlRequest::Flush(req) => {
1037 self.handle_flush_request(ddl.region_id, req, ddl.sender)
1038 .await;
1039 continue;
1040 }
1041 DdlRequest::Compact(req) => {
1042 self.handle_compaction_request(ddl.region_id, req, ddl.sender)
1043 .await;
1044 continue;
1045 }
1046 DdlRequest::BuildIndex(req) => {
1047 self.handle_build_index_request(ddl.region_id, req, ddl.sender)
1048 .await;
1049 continue;
1050 }
1051 DdlRequest::Truncate(req) => {
1052 self.handle_truncate_request(ddl.region_id, req, ddl.sender)
1053 .await;
1054 continue;
1055 }
1056 DdlRequest::Catchup((req, wal_entry_receiver)) => {
1057 self.handle_catchup_request(ddl.region_id, req, wal_entry_receiver, ddl.sender)
1058 .await;
1059 continue;
1060 }
1061 };
1062
1063 ddl.sender.send(res);
1064 }
1065 }
1066
1067 fn handle_periodical_tasks(&mut self) {
1069 let interval = CHECK_REGION_INTERVAL.as_millis() as i64;
1070 if self
1071 .time_provider
1072 .elapsed_since(self.last_periodical_check_millis)
1073 < interval
1074 {
1075 return;
1076 }
1077
1078 self.last_periodical_check_millis = self.time_provider.current_time_millis();
1079
1080 if let Err(e) = self.flush_periodically() {
1081 error!(e; "Failed to flush regions periodically");
1082 }
1083 }
1084
1085 async fn handle_background_notify(&mut self, region_id: RegionId, notify: BackgroundNotify) {
1087 match notify {
1088 BackgroundNotify::FlushFinished(req) => {
1089 self.handle_flush_finished(region_id, req).await
1090 }
1091 BackgroundNotify::FlushFailed(req) => self.handle_flush_failed(region_id, req).await,
1092 BackgroundNotify::IndexBuildFinished(req) => {
1093 self.handle_index_build_finished(region_id, req).await
1094 }
1095 BackgroundNotify::IndexBuildStopped(req) => {
1096 self.handle_index_build_stopped(region_id, req).await
1097 }
1098 BackgroundNotify::IndexBuildFailed(req) => {
1099 self.handle_index_build_failed(region_id, req).await
1100 }
1101 BackgroundNotify::CompactionFinished(req) => {
1102 self.handle_compaction_finished(region_id, req).await
1103 }
1104 BackgroundNotify::CompactionFailed(req) => self.handle_compaction_failure(req).await,
1105 BackgroundNotify::Truncate(req) => self.handle_truncate_result(req).await,
1106 BackgroundNotify::RegionChange(req) => {
1107 self.handle_manifest_region_change_result(req).await
1108 }
1109 BackgroundNotify::RegionEdit(req) => self.handle_region_edit_result(req).await,
1110 }
1111 }
1112
1113 async fn set_role_state_gracefully(
1115 &mut self,
1116 region_id: RegionId,
1117 region_role_state: SettableRegionRoleState,
1118 sender: oneshot::Sender<SetRegionRoleStateResponse>,
1119 ) {
1120 if let Some(region) = self.regions.get_region(region_id) {
1121 common_runtime::spawn_global(async move {
1123 match region.set_role_state_gracefully(region_role_state).await {
1124 Ok(()) => {
1125 let last_entry_id = region.version_control.current().last_entry_id;
1126 let _ = sender.send(SetRegionRoleStateResponse::success(
1127 SetRegionRoleStateSuccess::mito(last_entry_id),
1128 ));
1129 }
1130 Err(e) => {
1131 error!(e; "Failed to set region {} role state to {:?}", region_id, region_role_state);
1132 let _ = sender.send(SetRegionRoleStateResponse::invalid_transition(
1133 BoxedError::new(e),
1134 ));
1135 }
1136 }
1137 });
1138 } else {
1139 let _ = sender.send(SetRegionRoleStateResponse::NotFound);
1140 }
1141 }
1142}
1143
1144impl<S> RegionWorkerLoop<S> {
1145 async fn clean(&self) {
1147 let regions = self.regions.list_regions();
1149 for region in regions {
1150 region.stop().await;
1151 }
1152
1153 self.regions.clear();
1154 }
1155
1156 fn notify_group(&mut self) {
1159 let _ = self.flush_sender.send(());
1161 self.flush_receiver.borrow_and_update();
1163 }
1164}
1165
1166#[derive(Default, Clone)]
1168pub(crate) struct WorkerListener {
1169 #[cfg(any(test, feature = "test"))]
1170 listener: Option<crate::engine::listener::EventListenerRef>,
1171}
1172
1173impl WorkerListener {
1174 #[cfg(any(test, feature = "test"))]
1175 pub(crate) fn new(
1176 listener: Option<crate::engine::listener::EventListenerRef>,
1177 ) -> WorkerListener {
1178 WorkerListener { listener }
1179 }
1180
1181 pub(crate) fn on_flush_success(&self, region_id: RegionId) {
1183 #[cfg(any(test, feature = "test"))]
1184 if let Some(listener) = &self.listener {
1185 listener.on_flush_success(region_id);
1186 }
1187 let _ = region_id;
1189 }
1190
1191 pub(crate) fn on_write_stall(&self) {
1193 #[cfg(any(test, feature = "test"))]
1194 if let Some(listener) = &self.listener {
1195 listener.on_write_stall();
1196 }
1197 }
1198
1199 pub(crate) async fn on_flush_begin(&self, region_id: RegionId) {
1200 #[cfg(any(test, feature = "test"))]
1201 if let Some(listener) = &self.listener {
1202 listener.on_flush_begin(region_id).await;
1203 }
1204 let _ = region_id;
1206 }
1207
1208 pub(crate) fn on_later_drop_begin(&self, region_id: RegionId) -> Option<Duration> {
1209 #[cfg(any(test, feature = "test"))]
1210 if let Some(listener) = &self.listener {
1211 return listener.on_later_drop_begin(region_id);
1212 }
1213 let _ = region_id;
1215 None
1216 }
1217
1218 pub(crate) fn on_later_drop_end(&self, region_id: RegionId, removed: bool) {
1220 #[cfg(any(test, feature = "test"))]
1221 if let Some(listener) = &self.listener {
1222 listener.on_later_drop_end(region_id, removed);
1223 }
1224 let _ = region_id;
1226 let _ = removed;
1227 }
1228
1229 pub(crate) async fn on_merge_ssts_finished(&self, region_id: RegionId) {
1230 #[cfg(any(test, feature = "test"))]
1231 if let Some(listener) = &self.listener {
1232 listener.on_merge_ssts_finished(region_id).await;
1233 }
1234 let _ = region_id;
1236 }
1237
1238 pub(crate) fn on_recv_requests(&self, request_num: usize) {
1239 #[cfg(any(test, feature = "test"))]
1240 if let Some(listener) = &self.listener {
1241 listener.on_recv_requests(request_num);
1242 }
1243 let _ = request_num;
1245 }
1246
1247 pub(crate) fn on_file_cache_filled(&self, _file_id: FileId) {
1248 #[cfg(any(test, feature = "test"))]
1249 if let Some(listener) = &self.listener {
1250 listener.on_file_cache_filled(_file_id);
1251 }
1252 }
1253
1254 pub(crate) fn on_compaction_scheduled(&self, _region_id: RegionId) {
1255 #[cfg(any(test, feature = "test"))]
1256 if let Some(listener) = &self.listener {
1257 listener.on_compaction_scheduled(_region_id);
1258 }
1259 }
1260
1261 pub(crate) async fn on_notify_region_change_result_begin(&self, _region_id: RegionId) {
1262 #[cfg(any(test, feature = "test"))]
1263 if let Some(listener) = &self.listener {
1264 listener
1265 .on_notify_region_change_result_begin(_region_id)
1266 .await;
1267 }
1268 }
1269
1270 pub(crate) async fn on_index_build_finish(&self, _region_file_id: RegionFileId) {
1271 #[cfg(any(test, feature = "test"))]
1272 if let Some(listener) = &self.listener {
1273 listener.on_index_build_finish(_region_file_id).await;
1274 }
1275 }
1276
1277 pub(crate) async fn on_index_build_begin(&self, _region_file_id: RegionFileId) {
1278 #[cfg(any(test, feature = "test"))]
1279 if let Some(listener) = &self.listener {
1280 listener.on_index_build_begin(_region_file_id).await;
1281 }
1282 }
1283
1284 pub(crate) async fn on_index_build_abort(&self, _region_file_id: RegionFileId) {
1285 #[cfg(any(test, feature = "test"))]
1286 if let Some(listener) = &self.listener {
1287 listener.on_index_build_abort(_region_file_id).await;
1288 }
1289 }
1290}
1291
1292#[cfg(test)]
1293mod tests {
1294 use super::*;
1295 use crate::test_util::TestEnv;
1296
1297 #[test]
1298 fn test_region_id_to_index() {
1299 let num_workers = 4;
1300
1301 let region_id = RegionId::new(1, 2);
1302 let index = region_id_to_index(region_id, num_workers);
1303 assert_eq!(index, 3);
1304
1305 let region_id = RegionId::new(2, 3);
1306 let index = region_id_to_index(region_id, num_workers);
1307 assert_eq!(index, 1);
1308 }
1309
1310 #[tokio::test]
1311 async fn test_worker_group_start_stop() {
1312 let env = TestEnv::with_prefix("group-stop").await;
1313 let group = env
1314 .create_worker_group(MitoConfig {
1315 num_workers: 4,
1316 ..Default::default()
1317 })
1318 .await;
1319
1320 group.stop().await.unwrap();
1321 }
1322}