feat: introduce high_watermark for remote wal logstore (#5877)

* feat: introduce high_watermark_since_flush

* test: add unit test for high watermark

* refactor: submit a request instead

* fix: send reply before submit request

* fix: no need to update twice

* feat: update high watermark in background periodically

* test: update unit tests

* fix: update high watermark periodically

* test: update unit tests

* chore: apply review comments

* chore: rename

* chore: apply review comments

* chore: clean up

* chore: apply review comments
This commit is contained in:
Yuhan Wang
2025-04-18 20:10:47 +08:00
committed by GitHub
parent 1e394af583
commit 41814bb49f
20 changed files with 589 additions and 128 deletions

View File

@@ -25,7 +25,7 @@ use common_wal::options::WAL_OPTIONS_KEY;
use rstest::rstest;
use rstest_reuse::{self, apply};
use store_api::region_engine::RegionEngine;
use store_api::region_request::RegionRequest;
use store_api::region_request::{RegionFlushRequest, RegionRequest};
use store_api::storage::{RegionId, ScanRequest};
use crate::config::MitoConfig;
@@ -33,8 +33,8 @@ use crate::engine::listener::{FlushListener, StallListener};
use crate::test_util::{
build_rows, build_rows_for_key, flush_region, kafka_log_store_factory,
multiple_log_store_factories, prepare_test_for_kafka_log_store, put_rows,
raft_engine_log_store_factory, reopen_region, rows_schema, CreateRequestBuilder,
LogStoreFactory, MockWriteBufferManager, TestEnv,
raft_engine_log_store_factory, reopen_region, rows_schema, single_kafka_log_store_factory,
CreateRequestBuilder, LogStoreFactory, MockWriteBufferManager, TestEnv,
};
use crate::time_provider::TimeProvider;
use crate::worker::MAX_INITIAL_CHECK_DELAY_SECS;
@@ -544,3 +544,67 @@ async fn test_flush_workers() {
+-------+---------+---------------------+";
assert_eq!(expected, batches.pretty_print().unwrap());
}
#[apply(single_kafka_log_store_factory)]
async fn test_update_topic_latest_entry_id(factory: Option<LogStoreFactory>) {
common_telemetry::init_default_ut_logging();
let Some(factory) = factory else {
return;
};
let write_buffer_manager = Arc::new(MockWriteBufferManager::default());
let listener = Arc::new(FlushListener::default());
let mut env = TestEnv::new().with_log_store_factory(factory.clone());
let engine = env
.create_engine_with(
MitoConfig::default(),
Some(write_buffer_manager.clone()),
Some(listener.clone()),
)
.await;
let region_id = RegionId::new(1, 1);
env.get_schema_metadata_manager()
.register_region_table_info(
region_id.table_id(),
"test_table",
"test_catalog",
"test_schema",
None,
env.get_kv_backend(),
)
.await;
let topic = prepare_test_for_kafka_log_store(&factory).await;
let request = CreateRequestBuilder::new()
.kafka_topic(topic.clone())
.build();
let column_schemas = rows_schema(&request);
engine
.handle_request(region_id, RegionRequest::Create(request.clone()))
.await
.unwrap();
let region = engine.get_region(region_id).unwrap();
assert_eq!(region.topic_latest_entry_id.load(Ordering::Relaxed), 0);
let rows = Rows {
schema: column_schemas.clone(),
rows: build_rows_for_key("a", 0, 2, 0),
};
put_rows(&engine, region_id, rows.clone()).await;
let request = RegionFlushRequest::default();
engine
.handle_request(region_id, RegionRequest::Flush(request.clone()))
.await
.unwrap();
// Wait until flush is finished.
listener.wait().await;
assert_eq!(region.topic_latest_entry_id.load(Ordering::Relaxed), 0);
engine
.handle_request(region_id, RegionRequest::Flush(request.clone()))
.await
.unwrap();
assert_eq!(region.topic_latest_entry_id.load(Ordering::Relaxed), 1);
}

View File

@@ -119,6 +119,16 @@ pub(crate) struct MitoRegion {
last_compaction_millis: AtomicI64,
/// Provider to get current time.
time_provider: TimeProviderRef,
/// The topic's latest entry id since the region's last flushing.
/// **Only used for remote WAL pruning.**
///
/// The value will be updated to the high watermark of the topic
/// if region receives a flush request or schedules a periodic flush task
/// and the region's memtable is empty.
///
/// There are no WAL entries in range [flushed_entry_id, topic_latest_entry_id] for current region,
/// which means these WAL entries maybe able to be pruned up to `topic_latest_entry_id`.
pub(crate) topic_latest_entry_id: AtomicU64,
/// Memtable builder for the region.
pub(crate) memtable_builder: MemtableBuilderRef,
/// manifest stats
@@ -287,12 +297,14 @@ impl MitoRegion {
let sst_usage = version.ssts.sst_usage();
let index_usage = version.ssts.index_usage();
let flushed_entry_id = version.flushed_entry_id;
let wal_usage = self.estimated_wal_usage(memtable_usage);
let manifest_usage = self.stats.total_manifest_size();
let num_rows = version.ssts.num_rows() + version.memtables.num_rows();
let manifest_version = self.stats.manifest_version();
let flushed_entry_id = version.flushed_entry_id;
let topic_latest_entry_id = self.topic_latest_entry_id.load(Ordering::Relaxed);
RegionStatistic {
num_rows,
@@ -305,6 +317,8 @@ impl MitoRegion {
manifest_version,
flushed_entry_id,
},
data_topic_latest_entry_id: topic_latest_entry_id,
metadata_topic_latest_entry_id: topic_latest_entry_id,
}
}

View File

@@ -274,6 +274,7 @@ impl RegionOpener {
last_flush_millis: AtomicI64::new(now),
last_compaction_millis: AtomicI64::new(now),
time_provider: self.time_provider.clone(),
topic_latest_entry_id: AtomicU64::new(0),
memtable_builder,
stats: self.stats,
})
@@ -452,6 +453,7 @@ impl RegionOpener {
last_flush_millis: AtomicI64::new(now),
last_compaction_millis: AtomicI64::new(now),
time_provider: self.time_provider.clone(),
topic_latest_entry_id: AtomicU64::new(0),
memtable_builder,
stats: self.stats.clone(),
};

View File

@@ -196,6 +196,10 @@ mod tests {
) -> Result<Entry, Self::Error> {
unreachable!()
}
fn high_watermark(&self, _provider: &Provider) -> Result<EntryId, Self::Error> {
unreachable!()
}
}
#[tokio::test]

View File

@@ -14,6 +14,7 @@
//! Handling flush related requests.
use std::sync::atomic::Ordering;
use std::sync::Arc;
use common_telemetry::{error, info};
@@ -29,34 +30,6 @@ use crate::request::{FlushFailed, FlushFinished, OnFailure, OptionOutputTx};
use crate::worker::RegionWorkerLoop;
impl<S> RegionWorkerLoop<S> {
/// Handles manual flush request.
pub(crate) async fn handle_flush_request(
&mut self,
region_id: RegionId,
request: RegionFlushRequest,
mut sender: OptionOutputTx,
) {
let Some(region) = self.regions.flushable_region_or(region_id, &mut sender) else {
return;
};
let reason = if region.is_downgrading() {
FlushReason::Downgrading
} else {
FlushReason::Manual
};
let mut task =
self.new_flush_task(&region, reason, request.row_group_size, self.config.clone());
task.push_sender(sender);
if let Err(e) =
self.flush_scheduler
.schedule_flush(region.region_id, &region.version_control, task)
{
error!(e; "Failed to schedule flush task for region {}", region.region_id);
}
}
/// On region flush job failed.
pub(crate) async fn handle_flush_failed(&mut self, region_id: RegionId, request: FlushFailed) {
self.flush_scheduler.on_flush_failed(region_id, request.err);
@@ -129,37 +102,6 @@ impl<S> RegionWorkerLoop<S> {
Ok(())
}
/// Flushes regions periodically.
pub(crate) fn flush_periodically(&mut self) -> Result<()> {
let regions = self.regions.list_regions();
let now = self.time_provider.current_time_millis();
let min_last_flush_time = now - self.config.auto_flush_interval.as_millis() as i64;
for region in &regions {
if self.flush_scheduler.is_flush_requested(region.region_id) || !region.is_writable() {
// Already flushing or not writable.
continue;
}
if region.last_flush_millis() < min_last_flush_time {
// If flush time of this region is earlier than `min_last_flush_time`, we can flush this region.
let task = self.new_flush_task(
region,
FlushReason::Periodically,
None,
self.config.clone(),
);
self.flush_scheduler.schedule_flush(
region.region_id,
&region.version_control,
task,
)?;
}
}
Ok(())
}
/// Creates a flush task with specific `reason` for the `region`.
pub(crate) fn new_flush_task(
&self,
@@ -185,6 +127,75 @@ impl<S> RegionWorkerLoop<S> {
}
impl<S: LogStore> RegionWorkerLoop<S> {
/// Handles manual flush request.
pub(crate) async fn handle_flush_request(
&mut self,
region_id: RegionId,
request: RegionFlushRequest,
mut sender: OptionOutputTx,
) {
let Some(region) = self.regions.flushable_region_or(region_id, &mut sender) else {
return;
};
// `update_topic_latest_entry_id` updates `topic_latest_entry_id` when memtables are empty.
// But the flush is skipped if memtables are empty. Thus should update the `topic_latest_entry_id`
// when handling flush request instead of in `schedule_flush` or `flush_finished`.
self.update_topic_latest_entry_id(&region);
info!(
"Region {} flush request, high watermark: {}",
region_id,
region.topic_latest_entry_id.load(Ordering::Relaxed)
);
let reason = if region.is_downgrading() {
FlushReason::Downgrading
} else {
FlushReason::Manual
};
let mut task =
self.new_flush_task(&region, reason, request.row_group_size, self.config.clone());
task.push_sender(sender);
if let Err(e) =
self.flush_scheduler
.schedule_flush(region.region_id, &region.version_control, task)
{
error!(e; "Failed to schedule flush task for region {}", region.region_id);
}
}
/// Flushes regions periodically.
pub(crate) fn flush_periodically(&mut self) -> Result<()> {
let regions = self.regions.list_regions();
let now = self.time_provider.current_time_millis();
let min_last_flush_time = now - self.config.auto_flush_interval.as_millis() as i64;
for region in &regions {
if self.flush_scheduler.is_flush_requested(region.region_id) || !region.is_writable() {
// Already flushing or not writable.
continue;
}
self.update_topic_latest_entry_id(region);
if region.last_flush_millis() < min_last_flush_time {
// If flush time of this region is earlier than `min_last_flush_time`, we can flush this region.
let task = self.new_flush_task(
region,
FlushReason::Periodically,
None,
self.config.clone(),
);
self.flush_scheduler.schedule_flush(
region.region_id,
&region.version_control,
task,
)?;
}
}
Ok(())
}
/// On region flush job finished.
pub(crate) async fn handle_flush_finished(
&mut self,
@@ -247,4 +258,25 @@ impl<S: LogStore> RegionWorkerLoop<S> {
self.listener.on_flush_success(region_id);
}
/// Updates the latest entry id since flush of the region.
/// **This is only used for remote WAL pruning.**
pub(crate) fn update_topic_latest_entry_id(&mut self, region: &MitoRegionRef) {
if region.provider.is_remote_wal() && region.version().memtables.is_empty() {
let high_watermark = self
.wal
.store()
.high_watermark(&region.provider)
.unwrap_or(0);
if high_watermark != 0 {
region
.topic_latest_entry_id
.store(high_watermark, Ordering::Relaxed);
}
info!(
"Region {} high watermark updated to {}",
region.region_id, high_watermark
);
}
}
}