Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: serialize table operations by lock rather than queue #814

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
10 changes: 2 additions & 8 deletions analytic_engine/src/compaction/mod.rs
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
// Copyright 2022 CeresDB Project Authors. Licensed under Apache-2.0.
// Copyright 2022-2023 CeresDB Project Authors. Licensed under Apache-2.0.

//! Compaction.

Expand All @@ -11,7 +11,6 @@ use tokio::sync::oneshot;

use crate::{
compaction::picker::{CommonCompactionPicker, CompactionPickerRef},
instance::write_worker::CompactionNotifier,
sst::file::{FileHandle, Level},
table::data::TableDataRef,
table_options::COMPACTION_STRATEGY,
Expand Down Expand Up @@ -429,18 +428,13 @@ impl Drop for WaiterNotifier {
/// Request to compact single table.
pub struct TableCompactionRequest {
pub table_data: TableDataRef,
pub compaction_notifier: Option<CompactionNotifier>,
pub waiter: Option<oneshot::Sender<WaitResult<()>>>,
}

impl TableCompactionRequest {
pub fn no_waiter(
table_data: TableDataRef,
compaction_notifier: Option<CompactionNotifier>,
) -> Self {
pub fn no_waiter(table_data: TableDataRef) -> Self {
TableCompactionRequest {
table_data,
compaction_notifier,
waiter: None,
}
}
Expand Down
42 changes: 16 additions & 26 deletions analytic_engine/src/compaction/scheduler.rs
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,8 @@ use crate::{
PickerManager, TableCompactionRequest, WaitError, WaiterNotifier,
},
instance::{
flush_compaction::TableFlushOptions, write_worker::CompactionNotifier, Instance, SpaceStore,
flush_compaction::{Flusher, TableFlushOptions},
SpaceStore,
},
sst::factory::{ScanOptions, SstWriteOptions},
table::data::TableDataRef,
Expand Down Expand Up @@ -458,7 +459,6 @@ impl ScheduleWorker {
&self,
table_data: TableDataRef,
compaction_task: CompactionTask,
compaction_notifier: Option<CompactionNotifier>,
waiter_notifier: WaiterNotifier,
token: MemoryUsageToken,
) {
Expand Down Expand Up @@ -518,27 +518,18 @@ impl ScheduleWorker {
// Notify the background compact table result.
match res {
Ok(()) => {
if let Some(notifier) = compaction_notifier.clone() {
notifier.notify_ok();
}
waiter_notifier.notify_wait_result(Ok(()));

if keep_scheduling_compaction {
schedule_table_compaction(
sender,
TableCompactionRequest::no_waiter(
table_data.clone(),
compaction_notifier.clone(),
),
TableCompactionRequest::no_waiter(table_data.clone()),
)
.await;
}
}
Err(e) => {
let e = Arc::new(e);
if let Some(notifier) = compaction_notifier {
notifier.notify_err(e.clone());
}

let wait_err = WaitError::Compaction { source: e };
waiter_notifier.notify_wait_result(Err(wait_err));
Expand Down Expand Up @@ -612,16 +603,9 @@ impl ScheduleWorker {
}
};

let compaction_notifier = compact_req.compaction_notifier;
let waiter_notifier = WaiterNotifier::new(compact_req.waiter);

self.do_table_compaction_task(
table_data,
compaction_task,
compaction_notifier,
waiter_notifier,
token,
);
self.do_table_compaction_task(table_data, compaction_task, waiter_notifier, token);
}

async fn schedule(&mut self) {
Expand All @@ -642,25 +626,31 @@ impl ScheduleWorker {

// This will spawn a background job to purge ssts and avoid schedule thread
// blocked.
self.handle_table_compaction_request(TableCompactionRequest::no_waiter(
table_data, None,
))
.await;
self.handle_table_compaction_request(TableCompactionRequest::no_waiter(table_data))
.await;
}
}

async fn flush_tables(&self) {
let mut tables_buf = Vec::new();
self.space_store.list_all_tables(&mut tables_buf);
let flusher = Flusher {
space_store: self.space_store.clone(),
runtime: self.runtime.clone(),
write_sst_max_buffer_size: self.write_sst_max_buffer_size,
};

for table_data in &tables_buf {
let last_flush_time = table_data.last_flush_time();
if last_flush_time + self.max_unflushed_duration.as_millis_u64()
> common_util::time::current_time_millis()
{
let mut serial_exec = table_data.serial_exec.lock().await;
let flush_scheduler = serial_exec.flush_scheduler();
// Instance flush the table asynchronously.
if let Err(e) =
Instance::flush_table(table_data.clone(), TableFlushOptions::default()).await
if let Err(e) = flusher
.schedule_flush(flush_scheduler, table_data, TableFlushOptions::default())
.await
{
error!("Failed to flush table, err:{}", e);
}
Expand Down
Loading