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: write memtable in parallel #5456

Open
wants to merge 6 commits into
base: main
Choose a base branch
from
Open
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
2 changes: 1 addition & 1 deletion src/mito2/src/region/opener.rs
Original file line number Diff line number Diff line change
Expand Up @@ -554,7 +554,7 @@ where

// set next_entry_id and write to memtable.
region_write_ctx.set_next_entry_id(last_entry_id + 1);
region_write_ctx.write_memtable();
region_write_ctx.write_memtable().await;
}

// TODO(weny): We need to update `flushed_entry_id` in the region manifest
Expand Down
38 changes: 27 additions & 11 deletions src/mito2/src/region_write_ctx.rs
Original file line number Diff line number Diff line change
Expand Up @@ -197,23 +197,39 @@ impl RegionWriteCtx {
}

/// Consumes mutations and writes them into mutable memtable.
pub(crate) fn write_memtable(&mut self) {
pub(crate) async fn write_memtable(&mut self) {
debug_assert_eq!(self.notifiers.len(), self.wal_entry.mutations.len());

if self.failed {
return;
}

let mutable = &self.version.memtables.mutable;
// Takes mutations from the wal entry.
let mutations = mem::take(&mut self.wal_entry.mutations);
for (mutation, notify) in mutations.into_iter().zip(&mut self.notifiers) {
// Write mutation to the memtable.
let Some(kvs) = KeyValues::new(&self.version.metadata, mutation) else {
continue;
};
if let Err(e) = mutable.write(&kvs) {
notify.err = Some(Arc::new(e));
let mutable = self.version.memtables.mutable.clone();
let mutations = mem::take(&mut self.wal_entry.mutations)
.into_iter()
.enumerate()
.filter_map(|(i, mutation)| {
let kvs = KeyValues::new(&self.version.metadata, mutation)?;
Some((i, kvs))
})
.collect::<Vec<_>>();

if mutations.len() == 1 {
if let Err(err) = mutable.write(&mutations[0].1) {
self.notifiers[mutations[0].0].err = Some(Arc::new(err));
}
} else {
let write_tasks = mutations.into_iter().map(|(i, kvs)| {
let mutable = mutable.clone();
// use tokio runtime to schedule tasks.
common_runtime::spawn_blocking_global(move || (i, mutable.write(&kvs)))
});
Comment on lines +217 to +226
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we use something like rayon instead of spawn futures that inherently not async?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

rayon can't reuse existing tokio managed threads, and I'm hesitant to create another group of runtimes... all those tasks go into spawn_blocking which isn't async, I think that's fine.

BTW I do find a combination of rayon + tokio https://github.com/andybarron/tokio-rayon, but the last update time is 4 years ago 🤣

for result in futures::future::join_all(write_tasks).await {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

join_all is ordered, maybe using FuturesUnordered makes more sense since already record the order?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

from its document the ordered is more performant, let me have a try https://docs.rs/futures/latest/futures/future/fn.join_all.html#see-also

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

from its document the ordered is more performant, let me have a try https://docs.rs/futures/latest/futures/future/fn.join_all.html#see-also

by performant it mean compare to itself, JoinAll is like this:

pub struct JoinAll<F>
where
    F: Future,
{
    kind: JoinAllKind<F>,
}

#[cfg_attr(target_os = "none", cfg(target_has_atomic = "ptr"))]
pub(crate) const SMALL: usize = 30;

enum JoinAllKind<F>
where
    F: Future,
{
    Small {
        elems: Pin<Box<[MaybeDone<F>]>>,
    },
    #[cfg_attr(target_os = "none", cfg(target_has_atomic = "ptr"))]
    Big {
        fut: Collect<FuturesOrdered<F>, Vec<F::Output>>,
    },
}

So it's ordered even <30 futures, but do we need ordered here or is unordered good enough

// first unwrap the result from `spawn` above
let (i, result) = result.unwrap();
waynexia marked this conversation as resolved.
Show resolved Hide resolved
if let Err(err) = result {
self.notifiers[i].err = Some(Arc::new(err));
}
}
}

Expand Down
31 changes: 28 additions & 3 deletions src/mito2/src/worker/handle_write.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ use std::collections::{hash_map, HashMap};
use std::sync::Arc;

use api::v1::OpType;
use common_telemetry::debug;
use common_telemetry::{debug, error};
use snafu::ensure;
use store_api::codec::PrimaryKeyEncoding;
use store_api::logstore::LogStore;
Expand Down Expand Up @@ -105,10 +105,35 @@ impl<S: LogStore> RegionWorkerLoop<S> {
let _timer = WRITE_STAGE_ELAPSED
.with_label_values(&["write_memtable"])
.start_timer();
for mut region_ctx in region_ctxs.into_values() {
region_ctx.write_memtable();
if region_ctxs.len() == 1 {
// fast path for single region.
let mut region_ctx = region_ctxs.into_values().next().unwrap();
region_ctx.write_memtable().await;
put_rows += region_ctx.put_num;
delete_rows += region_ctx.delete_num;
} else {
let region_write_task = region_ctxs
.into_values()
.map(|mut region_ctx| {
// use tokio runtime to schedule tasks.
common_runtime::spawn_global(async move {
region_ctx.write_memtable().await;
(region_ctx.put_num, region_ctx.delete_num)
})
})
.collect::<Vec<_>>();

for result in futures::future::join_all(region_write_task).await {
match result {
Ok((put, delete)) => {
put_rows += put;
delete_rows += delete;
}
Err(e) => {
error!(e; "unexpected error when joining region write tasks");
}
}
}
}
}
WRITE_ROWS_TOTAL
Expand Down
Loading