Skip to content

Commit b6bc941

Browse files
authored
fix: gracefullly shutdown runners (#47)
* fix: gracefullly shutdown runners Use `Store::shutdown_runners()` to gracefully shutdown runners.` Signed-off-by: MrCroxx <[email protected]> * log warn if send failed instead of panic Signed-off-by: MrCroxx <[email protected]> --------- Signed-off-by: MrCroxx <[email protected]>
1 parent f1a3bbf commit b6bc941

File tree

6 files changed

+221
-137
lines changed

6 files changed

+221
-137
lines changed

foyer-storage-bench/src/main.rs

+2
Original file line numberDiff line numberDiff line change
@@ -260,6 +260,8 @@ async fn main() {
260260
&metrics_dump_end,
261261
);
262262
println!("\nTotal:\n{}", analysis);
263+
264+
store.shutdown_runners().await.unwrap();
263265
}
264266

265267
async fn bench(args: Args, store: Arc<TStore>, metrics: Metrics, stop: oneshot::Receiver<()>) {

foyer-storage/src/flusher.rs

+79-62
Original file line numberDiff line numberDiff line change
@@ -17,9 +17,9 @@ use std::sync::Arc;
1717
use foyer_common::queue::AsyncQueue;
1818
use foyer_intrusive::{core::adapter::Link, eviction::EvictionPolicy};
1919
use itertools::Itertools;
20-
use tokio::sync::{
21-
mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender},
22-
Mutex,
20+
use tokio::{
21+
sync::{broadcast, mpsc, Mutex},
22+
task::JoinHandle,
2323
};
2424

2525
use crate::{
@@ -38,7 +38,7 @@ pub struct FlushTask {
3838
struct FlusherInner {
3939
sequence: usize,
4040

41-
task_txs: Vec<UnboundedSender<FlushTask>>,
41+
task_txs: Vec<mpsc::UnboundedSender<FlushTask>>,
4242
}
4343

4444
pub struct Flusher {
@@ -63,7 +63,9 @@ impl Flusher {
6363
&self,
6464
buffers: Arc<AsyncQueue<Vec<u8, A>>>,
6565
region_manager: Arc<RegionManager<A, D, E, EL>>,
66-
) where
66+
stop_rxs: Vec<broadcast::Receiver<()>>,
67+
) -> Vec<JoinHandle<()>>
68+
where
6769
A: BufferAllocator,
6870
D: Device<IoBufferAllocator = A>,
6971
E: EvictionPolicy<RegionEpItemAdapter<EL>, Link = EL>,
@@ -73,25 +75,30 @@ impl Flusher {
7375

7476
#[allow(clippy::type_complexity)]
7577
let (mut txs, rxs): (
76-
Vec<UnboundedSender<FlushTask>>,
77-
Vec<UnboundedReceiver<FlushTask>>,
78-
) = (0..self.runners).map(|_| unbounded_channel()).unzip();
78+
Vec<mpsc::UnboundedSender<FlushTask>>,
79+
Vec<mpsc::UnboundedReceiver<FlushTask>>,
80+
) = (0..self.runners).map(|_| mpsc::unbounded_channel()).unzip();
7981
inner.task_txs.append(&mut txs);
8082

8183
let runners = rxs
8284
.into_iter()
83-
.map(|rx| Runner {
84-
task_rx: rx,
85+
.zip_eq(stop_rxs.into_iter())
86+
.map(|(task_rx, stop_rx)| Runner {
87+
task_rx,
8588
buffers: buffers.clone(),
8689
region_manager: region_manager.clone(),
90+
stop_rx,
8791
})
8892
.collect_vec();
8993

94+
let mut handles = vec![];
9095
for runner in runners {
91-
tokio::spawn(async move {
96+
let handle = tokio::spawn(async move {
9297
runner.run().await.unwrap();
9398
});
99+
handles.push(handle);
94100
}
101+
handles
95102
}
96103

97104
pub fn runners(&self) -> usize {
@@ -113,10 +120,12 @@ where
113120
E: EvictionPolicy<RegionEpItemAdapter<EL>, Link = EL>,
114121
EL: Link,
115122
{
116-
task_rx: UnboundedReceiver<FlushTask>,
123+
task_rx: mpsc::UnboundedReceiver<FlushTask>,
117124
buffers: Arc<AsyncQueue<Vec<u8, A>>>,
118125

119126
region_manager: Arc<RegionManager<A, D, E, EL>>,
127+
128+
stop_rx: broadcast::Receiver<()>,
120129
}
121130

122131
impl<A, D, E, EL> Runner<A, D, E, EL>
@@ -128,71 +137,79 @@ where
128137
{
129138
async fn run(mut self) -> Result<()> {
130139
loop {
131-
if let Some(task) = self.task_rx.recv().await {
132-
// TODO(MrCroxx): seal buffer
140+
tokio::select! {
141+
Some(task) = self.task_rx.recv() => {
142+
self.handle(task).await?;
143+
}
144+
_ = self.stop_rx.recv() => {
145+
tracing::info!("[flusher] exit");
146+
return Ok(())
147+
}
148+
}
149+
}
150+
}
133151

134-
tracing::info!("[flusher] receive flush task, region: {}", task.region_id);
152+
async fn handle(&self, task: FlushTask) -> Result<()> {
153+
tracing::info!("[flusher] receive flush task, region: {}", task.region_id);
135154

136-
let region = self.region_manager.region(&task.region_id);
155+
let region = self.region_manager.region(&task.region_id);
137156

138-
tracing::trace!("[flusher] step 1");
157+
tracing::trace!("[flusher] step 1");
139158

140-
{
141-
// step 1: write buffer back to device
142-
let slice = region.load(.., 0).await?.unwrap();
159+
{
160+
// step 1: write buffer back to device
161+
let slice = region.load(.., 0).await?.unwrap();
143162

144-
// wait all physical readers (from previous version) and writers done
145-
let guard = region.exclusive(false, true, false).await;
163+
// wait all physical readers (from previous version) and writers done
164+
let guard = region.exclusive(false, true, false).await;
146165

147-
tracing::trace!("[flusher] write region {} back to device", task.region_id);
166+
tracing::trace!("[flusher] write region {} back to device", task.region_id);
148167

149-
let mut offset = 0;
150-
let len = region.device().io_size();
151-
while offset < region.device().region_size() {
152-
let start = offset;
153-
let end = std::cmp::min(offset + len, region.device().region_size());
168+
let mut offset = 0;
169+
let len = region.device().io_size();
170+
while offset < region.device().region_size() {
171+
let start = offset;
172+
let end = std::cmp::min(offset + len, region.device().region_size());
154173

155-
let s = unsafe { Slice::new(&slice.as_ref()[start..end]) };
156-
region
157-
.device()
158-
.write(s, region.id(), offset as u64, len)
159-
.await?;
160-
offset += len;
161-
}
162-
drop(guard);
163-
slice.destroy().await;
164-
}
174+
let s = unsafe { Slice::new(&slice.as_ref()[start..end]) };
175+
region
176+
.device()
177+
.write(s, region.id(), offset as u64, len)
178+
.await?;
179+
offset += len;
180+
}
181+
drop(guard);
182+
slice.destroy().await;
183+
}
184+
185+
tracing::trace!("[flusher] step 2");
165186

166-
tracing::trace!("[flusher] step 2");
187+
let buffer = {
188+
// step 2: detach buffer
189+
let mut guard = region.exclusive(false, false, true).await;
167190

168-
let buffer = {
169-
// step 2: detach buffer
170-
let mut guard = region.exclusive(false, false, true).await;
191+
let buffer = guard.detach_buffer();
171192

172-
let buffer = guard.detach_buffer();
193+
tracing::trace!(
194+
"[flusher] region {}, writers: {}, buffered readers: {}, physical readers: {}",
195+
region.id(),
196+
guard.writers(),
197+
guard.buffered_readers(),
198+
guard.physical_readers()
199+
);
173200

174-
tracing::trace!(
175-
"[flusher] region {}, writers: {}, buffered readers: {}, physical readers: {}",
176-
region.id(),
177-
guard.writers(),
178-
guard.buffered_readers(),
179-
guard.physical_readers()
180-
);
201+
drop(guard);
202+
buffer
203+
};
181204

182-
drop(guard);
183-
buffer
184-
};
205+
tracing::trace!("[flusher] step 3");
185206

186-
tracing::trace!("[flusher] step 3");
207+
// step 3: release buffer
208+
self.buffers.release(buffer);
209+
self.region_manager.set_region_evictable(&region.id()).await;
187210

188-
// step 3: release buffer
189-
self.buffers.release(buffer);
190-
self.region_manager.set_region_evictable(&region.id()).await;
211+
tracing::info!("[flusher] finish flush task, region: {}", task.region_id);
191212

192-
tracing::info!("[flusher] finish flush task, region: {}", task.region_id);
193-
} else {
194-
return Ok(());
195-
}
196-
}
213+
Ok(())
197214
}
198215
}

foyer-storage/src/lib.rs

+1
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@
1616
#![feature(strict_provenance)]
1717
#![feature(trait_alias)]
1818
#![feature(get_mut_unchecked)]
19+
#![feature(let_chains)]
1920
#![allow(clippy::type_complexity)]
2021

2122
use device::io_buffer::AlignedAllocator;

0 commit comments

Comments
 (0)