Skip to content

Commit 48af997

Browse files
authored
chore(cubestore): Tuning local files cleanup (#6570)
1 parent c227ae9 commit 48af997

File tree

4 files changed

+54
-9
lines changed

4 files changed

+54
-9
lines changed

rust/cubestore/cubestore/src/config/mod.rs

Lines changed: 26 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -458,6 +458,10 @@ pub trait ConfigObj: DIService {
458458

459459
fn transport_max_message_size(&self) -> usize;
460460
fn transport_max_frame_size(&self) -> usize;
461+
462+
fn local_files_cleanup_interval_secs(&self) -> u64;
463+
464+
fn local_files_cleanup_size_threshold(&self) -> u64;
461465
}
462466

463467
#[derive(Debug, Clone)]
@@ -526,6 +530,8 @@ pub struct ConfigObjImpl {
526530
pub disk_space_cache_duration_secs: u64,
527531
pub transport_max_message_size: usize,
528532
pub transport_max_frame_size: usize,
533+
pub local_files_cleanup_interval_secs: u64,
534+
pub local_files_cleanup_size_threshold: u64,
529535
}
530536

531537
crate::di_service!(ConfigObjImpl, [ConfigObj]);
@@ -772,6 +778,14 @@ impl ConfigObj for ConfigObjImpl {
772778
fn transport_max_frame_size(&self) -> usize {
773779
self.transport_max_frame_size
774780
}
781+
782+
fn local_files_cleanup_interval_secs(&self) -> u64 {
783+
self.local_files_cleanup_interval_secs
784+
}
785+
786+
fn local_files_cleanup_size_threshold(&self) -> u64 {
787+
self.local_files_cleanup_size_threshold
788+
}
775789
}
776790

777791
lazy_static! {
@@ -1075,6 +1089,16 @@ impl Config {
10751089
Some(256 << 20),
10761090
Some(4 << 20),
10771091
),
1092+
local_files_cleanup_interval_secs: env_parse(
1093+
"CUBESTORE_LOCAL_FILES_CLEANUP_INTERVAL_SECS",
1094+
3 * 600,
1095+
),
1096+
local_files_cleanup_size_threshold: env_parse_size(
1097+
"CUBESTORE_LOCAL_FILES_CLEANUP_SIZE_THRESHOLD",
1098+
1024 * 1024 * 1024,
1099+
None,
1100+
None,
1101+
) as u64,
10781102
}),
10791103
}
10801104
}
@@ -1155,6 +1179,8 @@ impl Config {
11551179
disk_space_cache_duration_secs: 0,
11561180
transport_max_message_size: 64 << 20,
11571181
transport_max_frame_size: 16 << 20,
1182+
local_files_cleanup_interval_secs: 600,
1183+
local_files_cleanup_size_threshold: 0,
11581184
}),
11591185
}
11601186
}

rust/cubestore/cubestore/src/remotefs/gcs.rs

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -278,6 +278,9 @@ impl RemoteFs for GCSRemoteFs {
278278
if result.len() % 1_000 > 0 {
279279
pages_count += 1;
280280
}
281+
if pages_count > 100 {
282+
log::warn!("S3 list returned more than 100 pages: {}", pages_count);
283+
}
281284
app_metrics::REMOTE_FS_OPERATION_CORE.add_with_tags(
282285
pages_count as i64,
283286
Some(&vec![

rust/cubestore/cubestore/src/remotefs/queue.rs

Lines changed: 22 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -191,7 +191,6 @@ impl QueueRemoteFs {
191191
Ok(())
192192
}
193193

194-
const CLEANUP_INTERVAL: Duration = Duration::from_secs(600);
195194
/// Periodically cleans up the local directory from the files removed on the remote side.
196195
/// This function currently removes only direct sibling files and does not touch subdirectories.
197196
/// So e.g. we remove the `.parquet` files, but not directories like `metastore` or heartbeat.
@@ -201,10 +200,12 @@ impl QueueRemoteFs {
201200
async fn cleanup_loop(&self) -> () {
202201
let local_dir = self.local_path().await;
203202
let mut stopped_rx = self.stopped_rx.clone();
203+
let cleanup_interval = Duration::from_secs(self.config.local_files_cleanup_interval_secs());
204+
let cleanup_files_size_threshold = self.config.local_files_cleanup_size_threshold();
204205
loop {
205206
// Do the cleanup every now and then.
206207
tokio::select! {
207-
() = tokio::time::sleep(Self::CLEANUP_INTERVAL) => {},
208+
() = tokio::time::sleep(cleanup_interval) => {},
208209
res = stopped_rx.changed() => {
209210
if res.is_err() || *stopped_rx.borrow() {
210211
return;
@@ -216,9 +217,10 @@ impl QueueRemoteFs {
216217
// We rely on RemoteFs implementations to upload the file to the server before they make
217218
// it available on the local filesystem.
218219
let local_dir_copy = local_dir.clone();
219-
let res_local_files =
220-
cube_ext::spawn_blocking(move || -> Result<HashSet<String>, std::io::Error> {
220+
let res_local_files = cube_ext::spawn_blocking(
221+
move || -> Result<(HashSet<String>, u64), std::io::Error> {
221222
let mut local_files = HashSet::new();
223+
let mut local_files_size = 0;
222224
for res_entry in Path::new(&local_dir_copy).read_dir()? {
223225
let entry = match res_entry {
224226
Err(_) => continue, // ignore errors, might come from concurrent fs ops.
@@ -233,6 +235,12 @@ impl QueueRemoteFs {
233235
continue;
234236
}
235237

238+
local_files_size = if let Ok(metadata) = entry.metadata() {
239+
metadata.len()
240+
} else {
241+
0
242+
};
243+
236244
let file_name = match entry.file_name().into_string() {
237245
Err(_) => {
238246
log::error!("could not convert file name {:?}", entry.file_name());
@@ -243,19 +251,24 @@ impl QueueRemoteFs {
243251

244252
local_files.insert(file_name);
245253
}
246-
Ok(local_files)
247-
})
248-
.await
249-
.unwrap();
254+
Ok((local_files, local_files_size))
255+
},
256+
)
257+
.await
258+
.unwrap();
250259

251-
let mut local_files = match res_local_files {
260+
let (mut local_files, local_files_size) = match res_local_files {
252261
Err(e) => {
253262
log::error!("error while trying to list local files: {}", e);
254263
continue;
255264
}
256265
Ok(f) => f,
257266
};
258267

268+
if local_files_size < cleanup_files_size_threshold {
269+
continue;
270+
}
271+
259272
let res_remote_files = self.list("").await;
260273
let remote_files = match res_remote_files {
261274
Err(e) => {

rust/cubestore/cubestore/src/remotefs/s3.rs

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -282,6 +282,9 @@ impl RemoteFs for S3RemoteFs {
282282
pages_count as i64,
283283
Some(&vec!["operation:list".to_string(), "driver:s3".to_string()]),
284284
);
285+
if pages_count > 100 {
286+
log::warn!("S3 list returned more than 100 pages: {}", pages_count);
287+
}
285288
let leading_slash = Regex::new(format!("^{}", self.s3_path("")).as_str()).unwrap();
286289
let result = list
287290
.iter()

0 commit comments

Comments
 (0)