Skip to content

Commit e31ed97

Browse files
committed
feat: imple simple compaction scheduler
1 parent 47bb357 commit e31ed97

File tree

6 files changed

+190
-29
lines changed

6 files changed

+190
-29
lines changed

src/storage/src/compaction.rs

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@
1313
// limitations under the License.
1414

1515
mod dedup_deque;
16+
mod picker;
1617
mod rate_limit;
1718
mod scheduler;
1819
mod task;

src/storage/src/compaction/dedup_deque.rs

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -58,8 +58,7 @@ impl<K: Eq + Hash + Clone, V> DedupDeque<K, V> {
5858
Some((key, value))
5959
}
6060

61-
#[cfg(test)]
62-
fn len(&self) -> usize {
61+
pub fn len(&self) -> usize {
6362
debug_assert_eq!(self.deque.len(), self.existing.len());
6463
self.deque.len()
6564
}
Lines changed: 53 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,53 @@
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+
use crate::compaction::scheduler::CompactionRequest;
16+
use crate::compaction::task::{CompactionTask, CompactionTaskImpl};
17+
18+
/// Picker picks input SST files and build the compaction task.
19+
/// Different compaction strategy may implement different pickers.
20+
pub(crate) trait Picker<R, T: CompactionTask>: 'static {
21+
fn pick(&self, req: &R) -> crate::error::Result<T>;
22+
}
23+
24+
/// L0 -> L1 all-to-all compaction based on time windows.
25+
pub(crate) struct SimplePicker {}
26+
27+
impl SimplePicker {
28+
pub fn new() -> Self {
29+
Self {}
30+
}
31+
}
32+
33+
impl Picker<CompactionRequest, CompactionTaskImpl> for SimplePicker {
34+
fn pick(&self, _req: &CompactionRequest) -> crate::error::Result<CompactionTaskImpl> {
35+
todo!()
36+
}
37+
}
38+
39+
#[cfg(test)]
40+
pub mod tests {
41+
use super::*;
42+
use crate::compaction::task::tests::{CallbackRef, NoopCompactionTask};
43+
44+
pub(crate) struct MockPicker {
45+
pub cbs: Vec<CallbackRef>,
46+
}
47+
48+
impl Picker<CompactionRequest, NoopCompactionTask> for MockPicker {
49+
fn pick(&self, _req: &CompactionRequest) -> crate::error::Result<NoopCompactionTask> {
50+
Ok(NoopCompactionTask::new(self.cbs.clone()))
51+
}
52+
}
53+
}

src/storage/src/compaction/rate_limit.rs

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -77,9 +77,9 @@ impl<R> RateLimiter for MaxInflightTaskLimiter<R> {
7777
.fail();
7878
}
7979

80-
return Ok(Box::new(MaxInflightLimiterToken {
80+
Ok(Box::new(MaxInflightLimiterToken {
8181
counter: self.inflight_task.clone(),
82-
}));
82+
}))
8383
}
8484
}
8585

src/storage/src/compaction/scheduler.rs

Lines changed: 87 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -12,20 +12,22 @@
1212
// See the License for the specific language governing permissions and
1313
// limitations under the License.
1414

15+
use std::marker::PhantomData;
1516
use std::sync::Arc;
1617

1718
use async_trait::async_trait;
18-
use common_telemetry::info;
19+
use common_telemetry::{debug, info};
1920
use table::metadata::TableId;
2021
use tokio::sync::{Notify, RwLock};
2122
use tokio::task::JoinHandle;
2223
use tokio_util::sync::CancellationToken;
2324

2425
use crate::compaction::dedup_deque::DedupDeque;
26+
use crate::compaction::picker::{Picker, SimplePicker};
2527
use crate::compaction::rate_limit::{
2628
CascadeRateLimiter, RateLimitToken, RateLimitTokenPtr, RateLimiter,
2729
};
28-
use crate::compaction::task::CompactionTask;
30+
use crate::compaction::task::{CompactionTask, CompactionTaskImpl};
2931
use crate::error::Result;
3032

3133
/// Table compaction request.
@@ -87,9 +89,13 @@ impl LocalCompactionScheduler {
8789
req_queue: request_queue.clone(),
8890
cancel_token: cancel_token.child_token(),
8991
limiter: Arc::new(CascadeRateLimiter::new(vec![])),
92+
picker: SimplePicker::new(),
93+
_phantom_data: PhantomData::<CompactionTaskImpl>::default(),
9094
};
91-
let join_handle: JoinHandle<()> =
92-
common_runtime::spawn_bg(async move { handler.run().await });
95+
let join_handle = common_runtime::spawn_bg(async move {
96+
debug!("Compaction handler loop spawned");
97+
handler.run().await;
98+
});
9399
Self {
94100
join_handle,
95101
request_queue,
@@ -100,24 +106,27 @@ impl LocalCompactionScheduler {
100106
}
101107

102108
#[allow(unused)]
103-
struct CompactionHandler {
104-
req_queue: Arc<RwLock<DedupDeque<TableId, CompactionRequest>>>,
109+
struct CompactionHandler<R, T: CompactionTask, P: Picker<R, T>> {
110+
req_queue: Arc<RwLock<DedupDeque<TableId, R>>>,
105111
cancel_token: CancellationToken,
106112
task_notifier: Arc<Notify>,
107-
limiter: Arc<CascadeRateLimiter<CompactionRequest>>,
113+
limiter: Arc<CascadeRateLimiter<R>>,
114+
picker: P,
115+
_phantom_data: PhantomData<T>,
108116
}
109117

110118
#[allow(unused)]
111-
impl CompactionHandler {
119+
impl<R, T: CompactionTask, P: Picker<R, T>> CompactionHandler<R, T, P> {
112120
/// Runs table compaction requests dispatch loop.
113-
pub async fn run(self) {
121+
pub async fn run(&self) {
114122
let task_notifier = self.task_notifier.clone();
115123
let limiter = self.limiter.clone();
116124
loop {
117125
tokio::select! {
118126
_ = task_notifier.notified() => {
119127
// poll requests as many as possible until rate limited, and then wait for
120128
// notification (some task's finished).
129+
debug!("Notified, task size: {:?}", self.req_queue.read().await.len());
121130
while let Some((table_id, req)) = self.poll_task().await {
122131
if let Ok(token) = limiter.acquire_token(&req) {
123132
self.handle_compaction_request(req, token).await;
@@ -138,38 +147,95 @@ impl CompactionHandler {
138147
}
139148

140149
#[inline]
141-
async fn poll_task(&self) -> Option<(TableId, CompactionRequest)> {
150+
async fn poll_task(&self) -> Option<(TableId, R)> {
142151
let mut queue = self.req_queue.write().await;
143152
queue.pop_front()
144153
}
145154

146155
/// Puts request back to the front of request queue.
147156
#[inline]
148-
async fn put_back_req(&self, table_id: TableId, req: CompactionRequest) {
157+
async fn put_back_req(&self, table_id: TableId, req: R) {
149158
let mut queue = self.req_queue.write().await;
150159
queue.push_front(table_id, req);
151160
}
152161

153162
// Handles compaction request, submit task to bg runtime.
154-
async fn handle_compaction_request(
155-
&self,
156-
mut req: CompactionRequest,
157-
token: RateLimitTokenPtr,
158-
) {
163+
async fn handle_compaction_request(&self, mut req: R, token: RateLimitTokenPtr) -> Result<()> {
159164
let cloned_notify = self.task_notifier.clone();
160-
let task = self.build_compaction_task(req).await;
165+
let task = self.build_compaction_task(req).await?;
161166

162167
common_runtime::spawn_bg(async move {
163-
task.run();
164-
// releases rate limit token
168+
task.run().await; // TODO(hl): handle errors
169+
// releases rate limit token
165170
token.try_release();
166171
// notify scheduler to schedule next task when current task finishes.
167172
cloned_notify.notify_one();
168173
});
174+
175+
Ok(())
169176
}
170177

171178
// TODO(hl): generate compaction task(find SSTs to compact along with the output of compaction)
172-
async fn build_compaction_task(&self, req: CompactionRequest) -> CompactionTask {
173-
todo!()
179+
async fn build_compaction_task(&self, req: R) -> crate::error::Result<T> {
180+
self.picker.pick(&req)
181+
}
182+
}
183+
184+
#[cfg(test)]
185+
mod tests {
186+
use std::sync::atomic::{AtomicUsize, Ordering};
187+
188+
use tokio::sync::Barrier;
189+
190+
use super::*;
191+
use crate::compaction::picker::tests::MockPicker;
192+
use crate::compaction::rate_limit::MaxInflightTaskLimiter;
193+
194+
#[tokio::test]
195+
async fn test_schedule_handler() {
196+
common_telemetry::init_default_ut_logging();
197+
let queue = Arc::new(RwLock::new(DedupDeque::default()));
198+
let task_finished = Arc::new(AtomicUsize::new(0));
199+
200+
let task_finished_clone = task_finished.clone();
201+
let barrier = Arc::new(Barrier::new(3));
202+
let barrier_clone = barrier.clone();
203+
let picker = MockPicker {
204+
cbs: vec![Arc::new(move || {
205+
debug!("Running callback");
206+
task_finished_clone.fetch_add(1, Ordering::Relaxed);
207+
let barrier_clone_2 = barrier_clone.clone();
208+
Box::pin(async move {
209+
barrier_clone_2.wait().await;
210+
})
211+
})],
212+
};
213+
let handler = Arc::new(CompactionHandler {
214+
req_queue: queue.clone(),
215+
cancel_token: Default::default(),
216+
task_notifier: Arc::new(Default::default()),
217+
limiter: Arc::new(CascadeRateLimiter::new(vec![Box::new(
218+
MaxInflightTaskLimiter::new(3),
219+
)])),
220+
picker,
221+
_phantom_data: Default::default(),
222+
});
223+
224+
let handler_cloned = handler.clone();
225+
common_runtime::spawn_bg(async move { handler_cloned.run().await });
226+
227+
queue
228+
.write()
229+
.await
230+
.push_back(1, CompactionRequest::default());
231+
handler.task_notifier.notify_one();
232+
queue
233+
.write()
234+
.await
235+
.push_back(2, CompactionRequest::default());
236+
handler.task_notifier.notify_one();
237+
238+
barrier.wait().await;
239+
assert_eq!(2, task_finished.load(Ordering::Relaxed));
174240
}
175241
}

src/storage/src/compaction/task.rs

Lines changed: 46 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -15,15 +15,20 @@
1515
use crate::error::Result;
1616
use crate::sst::FileHandle;
1717

18+
#[async_trait::async_trait]
19+
pub(crate) trait CompactionTask: Send + Sync + 'static {
20+
async fn run(&self) -> Result<()>;
21+
}
22+
1823
#[allow(unused)]
19-
pub(crate) struct CompactionTask {
24+
pub(crate) struct CompactionTaskImpl {
2025
inputs: Vec<CompactionInput>,
2126
}
2227

23-
#[allow(unused)]
24-
impl CompactionTask {
28+
#[async_trait::async_trait]
29+
impl CompactionTask for CompactionTaskImpl {
2530
// TODO(hl): Actual SST compaction tasks
26-
pub async fn run(self) -> Result<()> {
31+
async fn run(&self) -> Result<()> {
2732
Ok(())
2833
}
2934
}
@@ -34,3 +39,40 @@ pub(crate) struct CompactionInput {
3439
output_level: u8,
3540
file: FileHandle,
3641
}
42+
43+
#[cfg(test)]
44+
pub mod tests {
45+
use std::future::Future;
46+
use std::pin::Pin;
47+
use std::sync::Arc;
48+
49+
use common_telemetry::debug;
50+
51+
use super::*;
52+
use crate::compaction::task::CompactionTask;
53+
54+
pub type CallbackRef =
55+
Arc<dyn Fn() -> Pin<Box<dyn Future<Output = ()> + Send + Sync>> + Send + Sync>;
56+
pub struct NoopCompactionTask {
57+
pub cbs: Vec<CallbackRef>,
58+
}
59+
60+
impl NoopCompactionTask {
61+
pub fn new(cbs: Vec<CallbackRef>) -> Self {
62+
Self { cbs }
63+
}
64+
}
65+
66+
#[async_trait::async_trait]
67+
impl CompactionTask for NoopCompactionTask {
68+
async fn run(&self) -> Result<()> {
69+
debug!("Running NoopCompactionTask");
70+
for cb in &self.cbs {
71+
debug!("Running callback");
72+
let f = cb();
73+
f.await;
74+
}
75+
Ok(())
76+
}
77+
}
78+
}

0 commit comments

Comments
 (0)