mito2/
flush.rs

1// Copyright 2023 Greptime Team
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15//! Flush related utilities and structs.
16
17use std::collections::HashMap;
18use std::num::NonZeroU64;
19use std::sync::atomic::{AtomicUsize, Ordering};
20use std::sync::Arc;
21
22use common_telemetry::{debug, error, info, trace};
23use snafu::ResultExt;
24use store_api::storage::RegionId;
25use strum::IntoStaticStr;
26use tokio::sync::{mpsc, watch};
27
28use crate::access_layer::{AccessLayerRef, OperationType, SstWriteRequest};
29use crate::cache::CacheManagerRef;
30use crate::config::MitoConfig;
31use crate::error::{
32    Error, FlushRegionSnafu, RegionClosedSnafu, RegionDroppedSnafu, RegionTruncatedSnafu, Result,
33};
34use crate::manifest::action::{RegionEdit, RegionMetaAction, RegionMetaActionList};
35use crate::metrics::{
36    FLUSH_BYTES_TOTAL, FLUSH_ELAPSED, FLUSH_FAILURE_TOTAL, FLUSH_REQUESTS_TOTAL,
37    INFLIGHT_FLUSH_COUNT,
38};
39use crate::read::Source;
40use crate::region::options::IndexOptions;
41use crate::region::version::{VersionControlData, VersionControlRef};
42use crate::region::{ManifestContextRef, RegionLeaderState};
43use crate::request::{
44    BackgroundNotify, FlushFailed, FlushFinished, OptionOutputTx, OutputTx, SenderBulkRequest,
45    SenderDdlRequest, SenderWriteRequest, WorkerRequest,
46};
47use crate::schedule::scheduler::{Job, SchedulerRef};
48use crate::sst::file::FileMeta;
49use crate::sst::parquet::WriteOptions;
50use crate::worker::WorkerListener;
51
52/// Global write buffer (memtable) manager.
53///
54/// Tracks write buffer (memtable) usages and decide whether the engine needs to flush.
55pub trait WriteBufferManager: Send + Sync + std::fmt::Debug {
56    /// Returns whether to trigger the engine.
57    fn should_flush_engine(&self) -> bool;
58
59    /// Returns whether to stall write requests.
60    fn should_stall(&self) -> bool;
61
62    /// Reserves `mem` bytes.
63    fn reserve_mem(&self, mem: usize);
64
65    /// Tells the manager we are freeing `mem` bytes.
66    ///
67    /// We are in the process of freeing `mem` bytes, so it is not considered
68    /// when checking the soft limit.
69    fn schedule_free_mem(&self, mem: usize);
70
71    /// We have freed `mem` bytes.
72    fn free_mem(&self, mem: usize);
73
74    /// Returns the total memory used by memtables.
75    fn memory_usage(&self) -> usize;
76}
77
78pub type WriteBufferManagerRef = Arc<dyn WriteBufferManager>;
79
80/// Default [WriteBufferManager] implementation.
81///
82/// Inspired by RocksDB's WriteBufferManager.
83/// <https://github.com/facebook/rocksdb/blob/main/include/rocksdb/write_buffer_manager.h>
84#[derive(Debug)]
85pub struct WriteBufferManagerImpl {
86    /// Write buffer size for the engine.
87    global_write_buffer_size: usize,
88    /// Mutable memtable memory size limit.
89    mutable_limit: usize,
90    /// Memory in used (e.g. used by mutable and immutable memtables).
91    memory_used: AtomicUsize,
92    /// Memory that hasn't been scheduled to free (e.g. used by mutable memtables).
93    memory_active: AtomicUsize,
94    /// Optional notifier.
95    /// The manager can wake up the worker once we free the write buffer.
96    notifier: Option<watch::Sender<()>>,
97}
98
99impl WriteBufferManagerImpl {
100    /// Returns a new manager with specific `global_write_buffer_size`.
101    pub fn new(global_write_buffer_size: usize) -> Self {
102        Self {
103            global_write_buffer_size,
104            mutable_limit: Self::get_mutable_limit(global_write_buffer_size),
105            memory_used: AtomicUsize::new(0),
106            memory_active: AtomicUsize::new(0),
107            notifier: None,
108        }
109    }
110
111    /// Attaches a notifier to the manager.
112    pub fn with_notifier(mut self, notifier: watch::Sender<()>) -> Self {
113        self.notifier = Some(notifier);
114        self
115    }
116
117    /// Returns memory usage of mutable memtables.
118    pub fn mutable_usage(&self) -> usize {
119        self.memory_active.load(Ordering::Relaxed)
120    }
121
122    /// Returns the size limit for mutable memtables.
123    fn get_mutable_limit(global_write_buffer_size: usize) -> usize {
124        // Reserves half of the write buffer for mutable memtable.
125        global_write_buffer_size / 2
126    }
127}
128
129impl WriteBufferManager for WriteBufferManagerImpl {
130    fn should_flush_engine(&self) -> bool {
131        let mutable_memtable_memory_usage = self.memory_active.load(Ordering::Relaxed);
132        if mutable_memtable_memory_usage > self.mutable_limit {
133            debug!(
134                "Engine should flush (over mutable limit), mutable_usage: {}, memory_usage: {}, mutable_limit: {}, global_limit: {}",
135                mutable_memtable_memory_usage, self.memory_usage(), self.mutable_limit, self.global_write_buffer_size,
136            );
137            return true;
138        }
139
140        let memory_usage = self.memory_used.load(Ordering::Relaxed);
141        // If the memory exceeds the buffer size, we trigger more aggressive
142        // flush. But if already more than half memory is being flushed,
143        // triggering more flush may not help. We will hold it instead.
144        if memory_usage >= self.global_write_buffer_size {
145            if mutable_memtable_memory_usage >= self.global_write_buffer_size / 2 {
146                debug!(
147                "Engine should flush (over total limit), memory_usage: {}, global_write_buffer_size: {}, \
148                 mutable_usage: {}.",
149                memory_usage,
150                self.global_write_buffer_size,
151                mutable_memtable_memory_usage);
152                return true;
153            } else {
154                trace!(
155                    "Engine won't flush, memory_usage: {}, global_write_buffer_size: {}, mutable_usage: {}.",
156                    memory_usage,
157                    self.global_write_buffer_size,
158                    mutable_memtable_memory_usage);
159            }
160        }
161
162        false
163    }
164
165    fn should_stall(&self) -> bool {
166        self.memory_usage() >= self.global_write_buffer_size
167    }
168
169    fn reserve_mem(&self, mem: usize) {
170        self.memory_used.fetch_add(mem, Ordering::Relaxed);
171        self.memory_active.fetch_add(mem, Ordering::Relaxed);
172    }
173
174    fn schedule_free_mem(&self, mem: usize) {
175        self.memory_active.fetch_sub(mem, Ordering::Relaxed);
176    }
177
178    fn free_mem(&self, mem: usize) {
179        self.memory_used.fetch_sub(mem, Ordering::Relaxed);
180        if let Some(notifier) = &self.notifier {
181            // Notifies the worker after the memory usage is decreased. When we drop the memtable
182            // outside of the worker, the worker may still stall requests because the memory usage
183            // is not updated. So we need to notify the worker to handle stalled requests again.
184            let _ = notifier.send(());
185        }
186    }
187
188    fn memory_usage(&self) -> usize {
189        self.memory_used.load(Ordering::Relaxed)
190    }
191}
192
193/// Reason of a flush task.
194#[derive(Debug, IntoStaticStr)]
195pub enum FlushReason {
196    /// Other reasons.
197    Others,
198    /// Engine reaches flush threshold.
199    EngineFull,
200    /// Manual flush.
201    Manual,
202    /// Flush to alter table.
203    Alter,
204    /// Flush periodically.
205    Periodically,
206    /// Flush memtable during downgrading state.
207    Downgrading,
208}
209
210impl FlushReason {
211    /// Get flush reason as static str.
212    fn as_str(&self) -> &'static str {
213        self.into()
214    }
215}
216
217/// Task to flush a region.
218pub(crate) struct RegionFlushTask {
219    /// Region to flush.
220    pub(crate) region_id: RegionId,
221    /// Reason to flush.
222    pub(crate) reason: FlushReason,
223    /// Flush result senders.
224    pub(crate) senders: Vec<OutputTx>,
225    /// Request sender to notify the worker.
226    pub(crate) request_sender: mpsc::Sender<WorkerRequest>,
227
228    pub(crate) access_layer: AccessLayerRef,
229    pub(crate) listener: WorkerListener,
230    pub(crate) engine_config: Arc<MitoConfig>,
231    pub(crate) row_group_size: Option<usize>,
232    pub(crate) cache_manager: CacheManagerRef,
233    pub(crate) manifest_ctx: ManifestContextRef,
234
235    /// Index options for the region.
236    pub(crate) index_options: IndexOptions,
237}
238
239impl RegionFlushTask {
240    /// Push the sender if it is not none.
241    pub(crate) fn push_sender(&mut self, mut sender: OptionOutputTx) {
242        if let Some(sender) = sender.take_inner() {
243            self.senders.push(sender);
244        }
245    }
246
247    /// Consumes the task and notify the sender the job is success.
248    fn on_success(self) {
249        for sender in self.senders {
250            sender.send(Ok(0));
251        }
252    }
253
254    /// Send flush error to waiter.
255    fn on_failure(&mut self, err: Arc<Error>) {
256        for sender in self.senders.drain(..) {
257            sender.send(Err(err.clone()).context(FlushRegionSnafu {
258                region_id: self.region_id,
259            }));
260        }
261    }
262
263    /// Converts the flush task into a background job.
264    ///
265    /// We must call this in the region worker.
266    fn into_flush_job(mut self, version_control: &VersionControlRef) -> Job {
267        // Get a version of this region before creating a job to get current
268        // wal entry id, sequence and immutable memtables.
269        let version_data = version_control.current();
270
271        Box::pin(async move {
272            INFLIGHT_FLUSH_COUNT.inc();
273            self.do_flush(version_data).await;
274            INFLIGHT_FLUSH_COUNT.dec();
275        })
276    }
277
278    /// Runs the flush task.
279    async fn do_flush(&mut self, version_data: VersionControlData) {
280        let timer = FLUSH_ELAPSED.with_label_values(&["total"]).start_timer();
281        self.listener.on_flush_begin(self.region_id).await;
282
283        let worker_request = match self.flush_memtables(&version_data).await {
284            Ok(edit) => {
285                let memtables_to_remove = version_data
286                    .version
287                    .memtables
288                    .immutables()
289                    .iter()
290                    .map(|m| m.id())
291                    .collect();
292                let flush_finished = FlushFinished {
293                    region_id: self.region_id,
294                    // The last entry has been flushed.
295                    flushed_entry_id: version_data.last_entry_id,
296                    senders: std::mem::take(&mut self.senders),
297                    _timer: timer,
298                    edit,
299                    memtables_to_remove,
300                };
301                WorkerRequest::Background {
302                    region_id: self.region_id,
303                    notify: BackgroundNotify::FlushFinished(flush_finished),
304                }
305            }
306            Err(e) => {
307                error!(e; "Failed to flush region {}", self.region_id);
308                // Discard the timer.
309                timer.stop_and_discard();
310
311                let err = Arc::new(e);
312                self.on_failure(err.clone());
313                WorkerRequest::Background {
314                    region_id: self.region_id,
315                    notify: BackgroundNotify::FlushFailed(FlushFailed { err }),
316                }
317            }
318        };
319        self.send_worker_request(worker_request).await;
320    }
321
322    /// Flushes memtables to level 0 SSTs and updates the manifest.
323    /// Returns the [RegionEdit] to apply.
324    async fn flush_memtables(&self, version_data: &VersionControlData) -> Result<RegionEdit> {
325        // We must use the immutable memtables list and entry ids from the `version_data`
326        // for consistency as others might already modify the version in the `version_control`.
327        let version = &version_data.version;
328        let timer = FLUSH_ELAPSED
329            .with_label_values(&["flush_memtables"])
330            .start_timer();
331
332        let mut write_opts = WriteOptions {
333            write_buffer_size: self.engine_config.sst_write_buffer_size,
334            ..Default::default()
335        };
336        if let Some(row_group_size) = self.row_group_size {
337            write_opts.row_group_size = row_group_size;
338        }
339
340        let memtables = version.memtables.immutables();
341        let mut file_metas = Vec::with_capacity(memtables.len());
342        let mut flushed_bytes = 0;
343        for mem in memtables {
344            if mem.is_empty() {
345                // Skip empty memtables.
346                continue;
347            }
348
349            let max_sequence = mem.stats().max_sequence();
350            let iter = mem.iter(None, None, None)?;
351            let source = Source::Iter(iter);
352
353            // Flush to level 0.
354            let write_request = SstWriteRequest {
355                op_type: OperationType::Flush,
356                metadata: version.metadata.clone(),
357                source,
358                cache_manager: self.cache_manager.clone(),
359                storage: version.options.storage.clone(),
360                max_sequence: Some(max_sequence),
361                index_options: self.index_options.clone(),
362                inverted_index_config: self.engine_config.inverted_index.clone(),
363                fulltext_index_config: self.engine_config.fulltext_index.clone(),
364                bloom_filter_index_config: self.engine_config.bloom_filter_index.clone(),
365            };
366
367            let ssts_written = self
368                .access_layer
369                .write_sst(write_request, &write_opts)
370                .await?;
371            if ssts_written.is_empty() {
372                // No data written.
373                continue;
374            }
375
376            file_metas.extend(ssts_written.into_iter().map(|sst_info| {
377                flushed_bytes += sst_info.file_size;
378                FileMeta {
379                    region_id: self.region_id,
380                    file_id: sst_info.file_id,
381                    time_range: sst_info.time_range,
382                    level: 0,
383                    file_size: sst_info.file_size,
384                    available_indexes: sst_info.index_metadata.build_available_indexes(),
385                    index_file_size: sst_info.index_metadata.file_size,
386                    num_rows: sst_info.num_rows as u64,
387                    num_row_groups: sst_info.num_row_groups,
388                    sequence: NonZeroU64::new(max_sequence),
389                }
390            }));
391        }
392
393        if !file_metas.is_empty() {
394            FLUSH_BYTES_TOTAL.inc_by(flushed_bytes);
395        }
396
397        let file_ids: Vec<_> = file_metas.iter().map(|f| f.file_id).collect();
398        info!(
399            "Successfully flush memtables, region: {}, reason: {}, files: {:?}, cost: {:?}s",
400            self.region_id,
401            self.reason.as_str(),
402            file_ids,
403            timer.stop_and_record(),
404        );
405
406        let edit = RegionEdit {
407            files_to_add: file_metas,
408            files_to_remove: Vec::new(),
409            compaction_time_window: None,
410            // The last entry has been flushed.
411            flushed_entry_id: Some(version_data.last_entry_id),
412            flushed_sequence: Some(version_data.committed_sequence),
413        };
414        info!("Applying {edit:?} to region {}", self.region_id);
415
416        let action_list = RegionMetaActionList::with_action(RegionMetaAction::Edit(edit.clone()));
417
418        let expected_state = if matches!(self.reason, FlushReason::Downgrading) {
419            RegionLeaderState::Downgrading
420        } else {
421            RegionLeaderState::Writable
422        };
423        // We will leak files if the manifest update fails, but we ignore them for simplicity. We can
424        // add a cleanup job to remove them later.
425        let version = self
426            .manifest_ctx
427            .update_manifest(expected_state, action_list)
428            .await?;
429        info!(
430            "Successfully update manifest version to {version}, region: {}, reason: {}",
431            self.region_id,
432            self.reason.as_str()
433        );
434
435        Ok(edit)
436    }
437
438    /// Notify flush job status.
439    async fn send_worker_request(&self, request: WorkerRequest) {
440        if let Err(e) = self.request_sender.send(request).await {
441            error!(
442                "Failed to notify flush job status for region {}, request: {:?}",
443                self.region_id, e.0
444            );
445        }
446    }
447
448    /// Merge two flush tasks.
449    fn merge(&mut self, mut other: RegionFlushTask) {
450        assert_eq!(self.region_id, other.region_id);
451        // Now we only merge senders. They share the same flush reason.
452        self.senders.append(&mut other.senders);
453    }
454}
455
456/// Manages background flushes of a worker.
457pub(crate) struct FlushScheduler {
458    /// Tracks regions need to flush.
459    region_status: HashMap<RegionId, FlushStatus>,
460    /// Background job scheduler.
461    scheduler: SchedulerRef,
462}
463
464impl FlushScheduler {
465    /// Creates a new flush scheduler.
466    pub(crate) fn new(scheduler: SchedulerRef) -> FlushScheduler {
467        FlushScheduler {
468            region_status: HashMap::new(),
469            scheduler,
470        }
471    }
472
473    /// Returns true if the region already requested flush.
474    pub(crate) fn is_flush_requested(&self, region_id: RegionId) -> bool {
475        self.region_status.contains_key(&region_id)
476    }
477
478    /// Schedules a flush `task` for specific `region`.
479    pub(crate) fn schedule_flush(
480        &mut self,
481        region_id: RegionId,
482        version_control: &VersionControlRef,
483        task: RegionFlushTask,
484    ) -> Result<()> {
485        debug_assert_eq!(region_id, task.region_id);
486
487        let version = version_control.current().version;
488        if version.memtables.is_empty() {
489            debug_assert!(!self.region_status.contains_key(&region_id));
490            // The region has nothing to flush.
491            task.on_success();
492            return Ok(());
493        }
494
495        // Don't increase the counter if a region has nothing to flush.
496        FLUSH_REQUESTS_TOTAL
497            .with_label_values(&[task.reason.as_str()])
498            .inc();
499
500        // Add this region to status map.
501        let flush_status = self
502            .region_status
503            .entry(region_id)
504            .or_insert_with(|| FlushStatus::new(region_id, version_control.clone()));
505        // Checks whether we can flush the region now.
506        if flush_status.flushing {
507            // There is already a flush job running.
508            flush_status.merge_task(task);
509            return Ok(());
510        }
511
512        // TODO(yingwen): We can merge with pending and execute directly.
513        // If there are pending tasks, then we should push it to pending list.
514        if flush_status.pending_task.is_some() {
515            flush_status.merge_task(task);
516            return Ok(());
517        }
518
519        // Now we can flush the region directly.
520        if let Err(e) = version_control.freeze_mutable() {
521            error!(e; "Failed to freeze the mutable memtable for region {}", region_id);
522
523            // Remove from region status if we can't freeze the mutable memtable.
524            self.region_status.remove(&region_id);
525            return Err(e);
526        }
527        // Submit a flush job.
528        let job = task.into_flush_job(version_control);
529        if let Err(e) = self.scheduler.schedule(job) {
530            // If scheduler returns error, senders in the job will be dropped and waiters
531            // can get recv errors.
532            error!(e; "Failed to schedule flush job for region {}", region_id);
533
534            // Remove from region status if we can't submit the task.
535            self.region_status.remove(&region_id);
536            return Err(e);
537        }
538
539        flush_status.flushing = true;
540
541        Ok(())
542    }
543
544    /// Notifies the scheduler that the flush job is finished.
545    ///
546    /// Returns all pending requests if the region doesn't need to flush again.
547    pub(crate) fn on_flush_success(
548        &mut self,
549        region_id: RegionId,
550    ) -> Option<(
551        Vec<SenderDdlRequest>,
552        Vec<SenderWriteRequest>,
553        Vec<SenderBulkRequest>,
554    )> {
555        let flush_status = self.region_status.get_mut(&region_id)?;
556
557        // This region doesn't have running flush job.
558        flush_status.flushing = false;
559
560        let pending_requests = if flush_status.pending_task.is_none() {
561            // The region doesn't have any pending flush task.
562            // Safety: The flush status must exist.
563            let flush_status = self.region_status.remove(&region_id).unwrap();
564            Some((
565                flush_status.pending_ddls,
566                flush_status.pending_writes,
567                flush_status.pending_bulk_writes,
568            ))
569        } else {
570            let version_data = flush_status.version_control.current();
571            if version_data.version.memtables.is_empty() {
572                // The region has nothing to flush, we also need to remove it from the status.
573                // Safety: The pending task is not None.
574                let task = flush_status.pending_task.take().unwrap();
575                // The region has nothing to flush. We can notify pending task.
576                task.on_success();
577                // `schedule_next_flush()` may pick up the same region to flush, so we must remove
578                // it from the status to avoid leaking pending requests.
579                // Safety: The flush status must exist.
580                let flush_status = self.region_status.remove(&region_id).unwrap();
581                Some((
582                    flush_status.pending_ddls,
583                    flush_status.pending_writes,
584                    flush_status.pending_bulk_writes,
585                ))
586            } else {
587                // We can flush the region again, keep it in the region status.
588                None
589            }
590        };
591
592        // Schedule next flush job.
593        if let Err(e) = self.schedule_next_flush() {
594            error!(e; "Flush of region {} is successful, but failed to schedule next flush", region_id);
595        }
596
597        pending_requests
598    }
599
600    /// Notifies the scheduler that the flush job is failed.
601    pub(crate) fn on_flush_failed(&mut self, region_id: RegionId, err: Arc<Error>) {
602        error!(err; "Region {} failed to flush, cancel all pending tasks", region_id);
603
604        FLUSH_FAILURE_TOTAL.inc();
605
606        // Remove this region.
607        let Some(flush_status) = self.region_status.remove(&region_id) else {
608            return;
609        };
610
611        // Fast fail: cancels all pending tasks and sends error to their waiters.
612        flush_status.on_failure(err);
613
614        // Still tries to schedule a new flush.
615        if let Err(e) = self.schedule_next_flush() {
616            error!(e; "Failed to schedule next flush after region {} flush is failed", region_id);
617        }
618    }
619
620    /// Notifies the scheduler that the region is dropped.
621    pub(crate) fn on_region_dropped(&mut self, region_id: RegionId) {
622        self.remove_region_on_failure(
623            region_id,
624            Arc::new(RegionDroppedSnafu { region_id }.build()),
625        );
626    }
627
628    /// Notifies the scheduler that the region is closed.
629    pub(crate) fn on_region_closed(&mut self, region_id: RegionId) {
630        self.remove_region_on_failure(region_id, Arc::new(RegionClosedSnafu { region_id }.build()));
631    }
632
633    /// Notifies the scheduler that the region is truncated.
634    pub(crate) fn on_region_truncated(&mut self, region_id: RegionId) {
635        self.remove_region_on_failure(
636            region_id,
637            Arc::new(RegionTruncatedSnafu { region_id }.build()),
638        );
639    }
640
641    fn remove_region_on_failure(&mut self, region_id: RegionId, err: Arc<Error>) {
642        // Remove this region.
643        let Some(flush_status) = self.region_status.remove(&region_id) else {
644            return;
645        };
646
647        // Notifies all pending tasks.
648        flush_status.on_failure(err);
649    }
650
651    /// Add ddl request to pending queue.
652    ///
653    /// # Panics
654    /// Panics if region didn't request flush.
655    pub(crate) fn add_ddl_request_to_pending(&mut self, request: SenderDdlRequest) {
656        let status = self.region_status.get_mut(&request.region_id).unwrap();
657        status.pending_ddls.push(request);
658    }
659
660    /// Add write request to pending queue.
661    ///
662    /// # Panics
663    /// Panics if region didn't request flush.
664    pub(crate) fn add_write_request_to_pending(&mut self, request: SenderWriteRequest) {
665        let status = self
666            .region_status
667            .get_mut(&request.request.region_id)
668            .unwrap();
669        status.pending_writes.push(request);
670    }
671
672    /// Add bulk write request to pending queue.
673    ///
674    /// # Panics
675    /// Panics if region didn't request flush.
676    pub(crate) fn add_bulk_request_to_pending(&mut self, request: SenderBulkRequest) {
677        let status = self.region_status.get_mut(&request.region_id).unwrap();
678        status.pending_bulk_writes.push(request);
679    }
680
681    /// Returns true if the region has pending DDLs.
682    pub(crate) fn has_pending_ddls(&self, region_id: RegionId) -> bool {
683        self.region_status
684            .get(&region_id)
685            .map(|status| !status.pending_ddls.is_empty())
686            .unwrap_or(false)
687    }
688
689    /// Schedules a new flush task when the scheduler can submit next task.
690    pub(crate) fn schedule_next_flush(&mut self) -> Result<()> {
691        debug_assert!(self
692            .region_status
693            .values()
694            .all(|status| status.flushing || status.pending_task.is_some()));
695
696        // Get the first region from status map.
697        let Some(flush_status) = self
698            .region_status
699            .values_mut()
700            .find(|status| status.pending_task.is_some())
701        else {
702            return Ok(());
703        };
704        debug_assert!(!flush_status.flushing);
705        let task = flush_status.pending_task.take().unwrap();
706        let region_id = flush_status.region_id;
707        let version_control = flush_status.version_control.clone();
708
709        self.schedule_flush(region_id, &version_control, task)
710    }
711}
712
713impl Drop for FlushScheduler {
714    fn drop(&mut self) {
715        for (region_id, flush_status) in self.region_status.drain() {
716            // We are shutting down so notify all pending tasks.
717            flush_status.on_failure(Arc::new(RegionClosedSnafu { region_id }.build()));
718        }
719    }
720}
721
722/// Flush status of a region scheduled by the [FlushScheduler].
723///
724/// Tracks running and pending flush tasks and all pending requests of a region.
725struct FlushStatus {
726    /// Current region.
727    region_id: RegionId,
728    /// Version control of the region.
729    version_control: VersionControlRef,
730    /// There is a flush task running.
731    ///
732    /// It is possible that a region is not flushing but has pending task if the scheduler
733    /// doesn't schedules this region.
734    flushing: bool,
735    /// Task waiting for next flush.
736    pending_task: Option<RegionFlushTask>,
737    /// Pending ddl requests.
738    pending_ddls: Vec<SenderDdlRequest>,
739    /// Requests waiting to write after altering the region.
740    pending_writes: Vec<SenderWriteRequest>,
741    /// Bulk requests waiting to write after altering the region.
742    pending_bulk_writes: Vec<SenderBulkRequest>,
743}
744
745impl FlushStatus {
746    fn new(region_id: RegionId, version_control: VersionControlRef) -> FlushStatus {
747        FlushStatus {
748            region_id,
749            version_control,
750            flushing: false,
751            pending_task: None,
752            pending_ddls: Vec::new(),
753            pending_writes: Vec::new(),
754            pending_bulk_writes: Vec::new(),
755        }
756    }
757
758    /// Merges the task to pending task.
759    fn merge_task(&mut self, task: RegionFlushTask) {
760        if let Some(pending) = &mut self.pending_task {
761            pending.merge(task);
762        } else {
763            self.pending_task = Some(task);
764        }
765    }
766
767    fn on_failure(self, err: Arc<Error>) {
768        if let Some(mut task) = self.pending_task {
769            task.on_failure(err.clone());
770        }
771        for ddl in self.pending_ddls {
772            ddl.sender.send(Err(err.clone()).context(FlushRegionSnafu {
773                region_id: self.region_id,
774            }));
775        }
776        for write_req in self.pending_writes {
777            write_req
778                .sender
779                .send(Err(err.clone()).context(FlushRegionSnafu {
780                    region_id: self.region_id,
781                }));
782        }
783    }
784}
785
786#[cfg(test)]
787mod tests {
788    use tokio::sync::oneshot;
789
790    use super::*;
791    use crate::cache::CacheManager;
792    use crate::memtable::time_series::TimeSeriesMemtableBuilder;
793    use crate::test_util::scheduler_util::{SchedulerEnv, VecScheduler};
794    use crate::test_util::version_util::{write_rows_to_version, VersionControlBuilder};
795
796    #[test]
797    fn test_get_mutable_limit() {
798        assert_eq!(4, WriteBufferManagerImpl::get_mutable_limit(8));
799        assert_eq!(5, WriteBufferManagerImpl::get_mutable_limit(10));
800        assert_eq!(32, WriteBufferManagerImpl::get_mutable_limit(64));
801        assert_eq!(0, WriteBufferManagerImpl::get_mutable_limit(0));
802    }
803
804    #[test]
805    fn test_over_mutable_limit() {
806        // Mutable limit is 500.
807        let manager = WriteBufferManagerImpl::new(1000);
808        manager.reserve_mem(400);
809        assert!(!manager.should_flush_engine());
810        assert!(!manager.should_stall());
811
812        // More than mutable limit.
813        manager.reserve_mem(400);
814        assert!(manager.should_flush_engine());
815
816        // Freezes mutable.
817        manager.schedule_free_mem(400);
818        assert!(!manager.should_flush_engine());
819        assert_eq!(800, manager.memory_used.load(Ordering::Relaxed));
820        assert_eq!(400, manager.memory_active.load(Ordering::Relaxed));
821
822        // Releases immutable.
823        manager.free_mem(400);
824        assert_eq!(400, manager.memory_used.load(Ordering::Relaxed));
825        assert_eq!(400, manager.memory_active.load(Ordering::Relaxed));
826    }
827
828    #[test]
829    fn test_over_global() {
830        // Mutable limit is 500.
831        let manager = WriteBufferManagerImpl::new(1000);
832        manager.reserve_mem(1100);
833        assert!(manager.should_stall());
834        // Global usage is still 1100.
835        manager.schedule_free_mem(200);
836        assert!(manager.should_flush_engine());
837
838        // More than global limit, but mutable (1100-200-450=450) is not enough (< 500).
839        manager.schedule_free_mem(450);
840        assert!(!manager.should_flush_engine());
841
842        // Now mutable is enough.
843        manager.reserve_mem(50);
844        assert!(manager.should_flush_engine());
845        manager.reserve_mem(100);
846        assert!(manager.should_flush_engine());
847    }
848
849    #[test]
850    fn test_manager_notify() {
851        let (sender, receiver) = watch::channel(());
852        let manager = WriteBufferManagerImpl::new(1000).with_notifier(sender);
853        manager.reserve_mem(500);
854        assert!(!receiver.has_changed().unwrap());
855        manager.schedule_free_mem(500);
856        assert!(!receiver.has_changed().unwrap());
857        manager.free_mem(500);
858        assert!(receiver.has_changed().unwrap());
859    }
860
861    #[tokio::test]
862    async fn test_schedule_empty() {
863        let env = SchedulerEnv::new().await;
864        let (tx, _rx) = mpsc::channel(4);
865        let mut scheduler = env.mock_flush_scheduler();
866        let builder = VersionControlBuilder::new();
867
868        let version_control = Arc::new(builder.build());
869        let (output_tx, output_rx) = oneshot::channel();
870        let mut task = RegionFlushTask {
871            region_id: builder.region_id(),
872            reason: FlushReason::Others,
873            senders: Vec::new(),
874            request_sender: tx,
875            access_layer: env.access_layer.clone(),
876            listener: WorkerListener::default(),
877            engine_config: Arc::new(MitoConfig::default()),
878            row_group_size: None,
879            cache_manager: Arc::new(CacheManager::default()),
880            manifest_ctx: env
881                .mock_manifest_context(version_control.current().version.metadata.clone())
882                .await,
883            index_options: IndexOptions::default(),
884        };
885        task.push_sender(OptionOutputTx::from(output_tx));
886        scheduler
887            .schedule_flush(builder.region_id(), &version_control, task)
888            .unwrap();
889        assert!(scheduler.region_status.is_empty());
890        let output = output_rx.await.unwrap().unwrap();
891        assert_eq!(output, 0);
892        assert!(scheduler.region_status.is_empty());
893    }
894
895    #[tokio::test]
896    async fn test_schedule_pending_request() {
897        let job_scheduler = Arc::new(VecScheduler::default());
898        let env = SchedulerEnv::new().await.scheduler(job_scheduler.clone());
899        let (tx, _rx) = mpsc::channel(4);
900        let mut scheduler = env.mock_flush_scheduler();
901        let mut builder = VersionControlBuilder::new();
902        // Overwrites the empty memtable builder.
903        builder.set_memtable_builder(Arc::new(TimeSeriesMemtableBuilder::default()));
904        let version_control = Arc::new(builder.build());
905        // Writes data to the memtable so it is not empty.
906        let version_data = version_control.current();
907        write_rows_to_version(&version_data.version, "host0", 0, 10);
908        let manifest_ctx = env
909            .mock_manifest_context(version_data.version.metadata.clone())
910            .await;
911        // Creates 3 tasks.
912        let mut tasks: Vec<_> = (0..3)
913            .map(|_| RegionFlushTask {
914                region_id: builder.region_id(),
915                reason: FlushReason::Others,
916                senders: Vec::new(),
917                request_sender: tx.clone(),
918                access_layer: env.access_layer.clone(),
919                listener: WorkerListener::default(),
920                engine_config: Arc::new(MitoConfig::default()),
921                row_group_size: None,
922                cache_manager: Arc::new(CacheManager::default()),
923                manifest_ctx: manifest_ctx.clone(),
924                index_options: IndexOptions::default(),
925            })
926            .collect();
927        // Schedule first task.
928        let task = tasks.pop().unwrap();
929        scheduler
930            .schedule_flush(builder.region_id(), &version_control, task)
931            .unwrap();
932        // Should schedule 1 flush.
933        assert_eq!(1, scheduler.region_status.len());
934        assert_eq!(1, job_scheduler.num_jobs());
935        // Check the new version.
936        let version_data = version_control.current();
937        assert_eq!(0, version_data.version.memtables.immutables()[0].id());
938        // Schedule remaining tasks.
939        let output_rxs: Vec<_> = tasks
940            .into_iter()
941            .map(|mut task| {
942                let (output_tx, output_rx) = oneshot::channel();
943                task.push_sender(OptionOutputTx::from(output_tx));
944                scheduler
945                    .schedule_flush(builder.region_id(), &version_control, task)
946                    .unwrap();
947                output_rx
948            })
949            .collect();
950        // Assumes the flush job is finished.
951        version_control.apply_edit(
952            RegionEdit {
953                files_to_add: Vec::new(),
954                files_to_remove: Vec::new(),
955                compaction_time_window: None,
956                flushed_entry_id: None,
957                flushed_sequence: None,
958            },
959            &[0],
960            builder.file_purger(),
961        );
962        scheduler.on_flush_success(builder.region_id());
963        // No new flush task.
964        assert_eq!(1, job_scheduler.num_jobs());
965        // The flush status is cleared.
966        assert!(scheduler.region_status.is_empty());
967        for output_rx in output_rxs {
968            let output = output_rx.await.unwrap().unwrap();
969            assert_eq!(output, 0);
970        }
971    }
972}