liquid-cache 0.1.12

10x lower latency for cloud-native DataFusion
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
use std::{
    fmt::{Display, Formatter},
    fs,
    ops::{Range, RangeInclusive},
    path::PathBuf,
    sync::Arc,
};

use async_stream::stream;
use async_trait::async_trait;
use bytes::Bytes;
use futures::{Stream, stream::BoxStream};
use object_store::{
    Error, GetOptions, GetRange, GetResult, GetResultPayload, ListResult, MultipartUpload,
    ObjectMeta, ObjectStore, PutMultipartOptions, PutOptions, PutPayload, PutResult, Result,
    path::Path,
};
use tokio::io::{AsyncReadExt, AsyncSeekExt, AsyncWriteExt};

const CACHE_BLOCK_SIZE: u64 = 1024 * 1024 * 4; // 4MB

/// Byte cache for liquid cache, act like a object store.
#[derive(Debug, Clone)]
pub struct ByteCache {
    inner: Arc<dyn ObjectStore>,
    cache_dir: PathBuf,
}

impl ByteCache {
    /// Create a new byte cache, the cache_dir is the directory to store the cached files
    /// `ByteCache` can read from a previously initialized cache directory
    pub fn new(inner: Arc<dyn ObjectStore>, cache_dir: PathBuf) -> Self {
        if !cache_dir.exists() {
            fs::create_dir_all(&cache_dir).expect("Failed to create cache directory");
        }
        Self { inner, cache_dir }
    }

    /// Convert a path to a cached directory path
    fn get_cache_dir_for_path(&self, path: &Path) -> PathBuf {
        let path_str = path.as_ref().replace("/", "_");
        self.cache_dir.join(path_str)
    }

    /// Get the path for a specific chunk of a file
    fn get_chunk_path(&self, path: &Path, chunk_index: u64) -> PathBuf {
        let cache_dir = self.get_cache_dir_for_path(path);
        cache_dir.join(format!("chunk_{chunk_index}.bin"))
    }

    /// Get the temporary path for a chunk while it's being written
    fn get_temp_chunk_path(&self, path: &Path, chunk_index: u64) -> PathBuf {
        let cache_dir = self.get_cache_dir_for_path(path);
        // Include timestamp and thread ID to avoid conflicts between concurrent writes
        let now = std::time::SystemTime::now()
            .duration_since(std::time::UNIX_EPOCH)
            .unwrap_or_default()
            .as_nanos();
        let thread_id = std::thread::current().id();
        cache_dir.join(format!(
            "in-progress-chunk_{chunk_index}_{now}_{thread_id:?}.bin"
        ))
    }

    /// Calculate which chunks are needed for a given range
    fn chunks_for_range(&self, range: &Range<u64>) -> RangeInclusive<u64> {
        let start_chunk = range.start / CACHE_BLOCK_SIZE;
        let end_chunk = (range.end - 1) / CACHE_BLOCK_SIZE; // -1 because end is exclusive
        start_chunk..=end_chunk
    }

    /// Check if a cached chunk is ready to be read (file exists, since rename is atomic)
    fn is_chunk_ready(&self, chunk_path: &std::path::Path) -> bool {
        chunk_path.exists()
    }

    /// Read data from a cached chunk
    async fn read_from_cached_chunk(
        &self,
        chunk_path: PathBuf,
        offset: u64,
        len: usize,
    ) -> Result<Bytes> {
        let mut file = tokio::fs::File::open(&chunk_path)
            .await
            .map_err(|e| Error::Generic {
                store: "ByteCache",
                source: Box::new(e),
            })?;

        let mut buffer = vec![0u8; len];
        file.seek(tokio::io::SeekFrom::Start(offset))
            .await
            .map_err(|e| Error::Generic {
                store: "ByteCache",
                source: Box::new(e),
            })?;

        file.read_exact(&mut buffer)
            .await
            .map_err(|e| Error::Generic {
                store: "ByteCache",
                source: Box::new(e),
            })?;

        Ok(Bytes::from(buffer))
    }

    /// Save a chunk to the cache
    async fn save_chunk(&self, path: &Path, chunk_index: u64, data: Bytes) -> Result<()> {
        let cache_dir = self.get_cache_dir_for_path(path);
        if !cache_dir.exists() {
            fs::create_dir_all(&cache_dir).map_err(|e| Error::Generic {
                store: "ByteCache",
                source: Box::new(e),
            })?;
        }

        // Write to a temporary file first
        let temp_chunk_path = self.get_temp_chunk_path(path, chunk_index);
        let final_chunk_path = self.get_chunk_path(path, chunk_index);

        let mut file = tokio::fs::File::create(&temp_chunk_path)
            .await
            .map_err(|e| Error::Generic {
                store: "ByteCache",
                source: Box::new(e),
            })?;

        // Write the data to the temporary file
        file.write_all(&data).await.map_err(|e| Error::Generic {
            store: "ByteCache",
            source: Box::new(e),
        })?;

        // Sync to ensure data is written to disk before rename
        file.sync_all().await.map_err(|e| Error::Generic {
            store: "ByteCache",
            source: Box::new(e),
        })?;

        // Close the file before rename
        drop(file);

        // Atomically rename the temporary file to the final location
        // If another thread already created the file, that's fine - we can just clean up our temp file
        match tokio::fs::rename(&temp_chunk_path, &final_chunk_path).await {
            Ok(_) => Ok(()),
            Err(e) => {
                let _ = tokio::fs::remove_file(&temp_chunk_path).await;
                if final_chunk_path.exists() {
                    Ok(())
                } else {
                    Err(Error::Generic {
                        store: "ByteCache",
                        source: Box::new(e),
                    })
                }
            }
        }
    }

    /// Get range data from cached chunks
    fn get_range_from_cache_stream(
        &self,
        location: &Path,
        range: &Range<u64>,
    ) -> impl Stream<Item = Result<Bytes>> + Send + 'static {
        let this = self.clone();
        let location = location.clone();
        let range = range.clone();
        stream! {
            let chunks_needed = this.chunks_for_range(&range);
            for chunk_idx in chunks_needed {
                let chunk_path = this.get_chunk_path(&location, chunk_idx);
                let chunk_start = chunk_idx * CACHE_BLOCK_SIZE;
                let chunk_end = chunk_start + CACHE_BLOCK_SIZE;

                let overlap_start = std::cmp::max(chunk_start, range.start);
                let overlap_end = std::cmp::min(chunk_end, range.end);

                if overlap_start < overlap_end {
                    let offset_in_chunk = overlap_start - chunk_start;
                    let length = overlap_end - overlap_start;

                    yield this
                        .read_from_cached_chunk(chunk_path, offset_in_chunk, length as usize)
                        .await;
                }
            }
        }
    }
}

impl Display for ByteCache {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        write!(f, "ByteCache(cache_dir: {:?})", self.cache_dir)
    }
}

#[async_trait]
impl ObjectStore for ByteCache {
    fn list(&self, prefix: Option<&Path>) -> BoxStream<'static, Result<ObjectMeta>> {
        self.inner.list(prefix)
    }

    async fn list_with_delimiter(&self, prefix: Option<&Path>) -> Result<ListResult> {
        self.inner.list_with_delimiter(prefix).await
    }

    async fn get_opts(&self, location: &Path, options: GetOptions) -> Result<GetResult> {
        // Get the metadata first to know the file size
        let meta = self.inner.head(location).await?;
        let file_size = meta.size;

        // If this is just a HEAD request, return the metadata
        if options.head {
            return self.inner.get_opts(location, options).await;
        }

        // Determine the range we need to fetch
        let range = match &options.range {
            Some(GetRange::Bounded(range)) => range.clone(),
            Some(GetRange::Suffix(suffix)) => (file_size.saturating_sub(*suffix))..file_size,
            Some(GetRange::Offset(offset)) => *offset..file_size,
            None => 0..file_size,
        };

        // Calculate which chunks we need
        let chunks_needed = self.chunks_for_range(&range);

        // Check which chunks are already cached
        let mut missing_chunks = Vec::new();
        for chunk_idx in chunks_needed {
            let chunk_path = self.get_chunk_path(location, chunk_idx);
            if !self.is_chunk_ready(&chunk_path) {
                missing_chunks.push(chunk_idx);
            }
        }

        // Fetch missing chunks from the underlying store
        for chunk_idx in missing_chunks {
            let chunk_start = chunk_idx * CACHE_BLOCK_SIZE;
            let chunk_end = std::cmp::min(chunk_start + CACHE_BLOCK_SIZE, file_size);

            let chunk_range = GetRange::Bounded(chunk_start..chunk_end);
            let chunk_options = GetOptions {
                range: Some(chunk_range),
                ..options.clone()
            };

            let chunk_result = self.inner.get_opts(location, chunk_options).await?;
            let chunk_data = chunk_result.bytes().await?;

            // Save the chunk to cache
            self.save_chunk(location, chunk_idx, chunk_data).await?;
        }

        // Return a GetResult with the stream of bytes from cache
        Ok(GetResult {
            payload: GetResultPayload::Stream(Box::pin(
                self.get_range_from_cache_stream(location, &range),
            )),
            meta,
            range,
            attributes: Default::default(),
        })
    }

    async fn put_opts(
        &self,
        _location: &Path,
        _payload: PutPayload,
        _opts: PutOptions,
    ) -> Result<PutResult> {
        unreachable!("ByteCache does not support put")
    }

    async fn put_multipart_opts(
        &self,
        _location: &Path,
        _opts: PutMultipartOptions,
    ) -> Result<Box<dyn MultipartUpload>> {
        unreachable!("ByteCache does not support multipart upload")
    }

    async fn delete(&self, _location: &Path) -> Result<()> {
        unreachable!("ByteCache does not support delete")
    }

    async fn copy(&self, _from: &Path, _to: &Path) -> Result<()> {
        unreachable!("ByteCache does not support copy")
    }

    async fn copy_if_not_exists(&self, _from: &Path, _to: &Path) -> Result<()> {
        unreachable!("ByteCache does not support copy_if_not_exists")
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use bytes::Bytes;
    use liquid_cache_common::mock_store::MockStore;
    use object_store::memory::InMemory;
    use std::ops::Range;
    use tempfile::tempdir;

    // Helper function to create a test file of specified size in the in-memory store
    async fn create_test_file(store: &InMemory, path: &str, size: u64) -> Result<()> {
        let data = vec![0u8; size as usize];
        // Fill the data with a pattern: index % 256
        // This makes it easy to verify ranges
        let data: Vec<u8> = data
            .iter()
            .enumerate()
            .map(|(i, _)| (i % 256) as u8)
            .collect();

        let path = Path::from(path);
        store.put(&path, Bytes::from(data).into()).await?;
        Ok(())
    }

    // Helper function to read a range from the store and verify it
    async fn verify_range(store: &dyn ObjectStore, path: &str, range: Range<u64>) -> Result<()> {
        let path = Path::from(path);
        let result = store.get_range(&path, range.clone()).await?;

        // Verify that the returned data matches the expected pattern
        for (i, byte) in result.iter().enumerate() {
            let expected = ((range.start + i as u64) % 256) as u8;
            assert_eq!(*byte, expected, "Mismatch at position {i}");
        }

        Ok(())
    }

    // Test reading a small file (less than one chunk)
    #[tokio::test]
    async fn test_small_file() -> Result<()> {
        let inner = Arc::new(InMemory::new());
        let temp_dir = tempdir().unwrap();
        let cache = ByteCache::new(inner.clone(), temp_dir.path().to_path_buf());

        // Create a small file (10KB)
        let file_path = "small_file.bin";
        create_test_file(&inner, file_path, 10 * 1024).await?;

        // Read the entire file
        verify_range(&cache, file_path, 0..10 * 1024).await?;

        // Verify the file was cached
        let cache_dir = cache.get_cache_dir_for_path(&Path::from(file_path));
        assert!(cache_dir.exists(), "Cache directory should exist");

        let chunk_path = cache.get_chunk_path(&Path::from(file_path), 0);
        assert!(
            cache.is_chunk_ready(&chunk_path),
            "Chunk file should be ready"
        );

        Ok(())
    }

    // Test reading a large file (multiple chunks)
    #[tokio::test]
    async fn test_large_file() -> Result<()> {
        let inner = Arc::new(InMemory::new());
        let temp_dir = tempdir().unwrap();
        let cache = ByteCache::new(inner.clone(), temp_dir.path().to_path_buf());

        // Create a file slightly larger than 2 chunks (9MB)
        let file_path = "large_file.bin";
        let file_size = CACHE_BLOCK_SIZE * 2 + 1024 * 1024; // 9MB
        create_test_file(&inner, file_path, file_size).await?;

        // Read the entire file
        verify_range(&cache, file_path, 0..file_size).await?;

        // Verify all chunks were cached
        for chunk_idx in 0..=2 {
            let chunk_path = cache.get_chunk_path(&Path::from(file_path), chunk_idx);
            assert!(
                cache.is_chunk_ready(&chunk_path),
                "Chunk {chunk_idx} should be ready"
            );
        }

        Ok(())
    }

    // Test reading a range within a single chunk
    #[tokio::test]
    async fn test_range_within_chunk() -> Result<()> {
        let inner = Arc::new(InMemory::new());
        let temp_dir = tempdir().unwrap();
        let cache = ByteCache::new(inner.clone(), temp_dir.path().to_path_buf());

        // Create a file larger than one chunk
        let file_path = "range_test.bin";
        let file_size = CACHE_BLOCK_SIZE * 3; // 12MB
        create_test_file(&inner, file_path, file_size).await?;

        // Read a range entirely within the second chunk
        let start = CACHE_BLOCK_SIZE + 1024;
        let end = CACHE_BLOCK_SIZE + 2048;
        verify_range(&cache, file_path, start..end).await?;

        // Verify only the requested chunk was cached
        let chunk1_path = cache.get_chunk_path(&Path::from(file_path), 1);
        assert!(
            cache.is_chunk_ready(&chunk1_path),
            "Chunk 1 should be ready"
        );

        // Other chunks should not be cached yet
        let chunk0_path = cache.get_chunk_path(&Path::from(file_path), 0);
        let chunk2_path = cache.get_chunk_path(&Path::from(file_path), 2);
        assert!(
            !cache.is_chunk_ready(&chunk0_path),
            "Chunk 0 should not be ready yet"
        );
        assert!(
            !cache.is_chunk_ready(&chunk2_path),
            "Chunk 2 should not be ready yet"
        );

        Ok(())
    }

    // Test reading a range that spans multiple chunks
    #[tokio::test]
    async fn test_range_across_chunks() -> Result<()> {
        let inner = Arc::new(InMemory::new());
        let temp_dir = tempdir().unwrap();
        let cache = ByteCache::new(inner.clone(), temp_dir.path().to_path_buf());

        // Create a file larger than two chunks
        let file_path = "multi_chunk_range.bin";
        let file_size = CACHE_BLOCK_SIZE * 3; // 12MB
        create_test_file(&inner, file_path, file_size).await?;

        // Read a range that spans chunk 1 and chunk 2
        let start = CACHE_BLOCK_SIZE - 1024;
        let end = CACHE_BLOCK_SIZE * 2 + 1024;
        verify_range(&cache, file_path, start..end).await?;

        // Verify the chunks were cached
        let chunk0_path = cache.get_chunk_path(&Path::from(file_path), 0);
        let chunk1_path = cache.get_chunk_path(&Path::from(file_path), 1);
        let chunk2_path = cache.get_chunk_path(&Path::from(file_path), 2);
        assert!(
            cache.is_chunk_ready(&chunk0_path),
            "Chunk 0 should be ready"
        );
        assert!(
            cache.is_chunk_ready(&chunk1_path),
            "Chunk 1 should be ready"
        );
        assert!(
            cache.is_chunk_ready(&chunk2_path),
            "Chunk 2 should be ready"
        );

        Ok(())
    }

    // Test cache hit (read the same file twice)
    #[tokio::test]
    async fn test_cache_hit() -> Result<()> {
        let inner = Arc::new(InMemory::new());
        let temp_dir = tempdir().unwrap();
        let cache = ByteCache::new(inner.clone(), temp_dir.path().to_path_buf());

        // Create a file
        let file_path = "cache_hit.bin";
        let file_size = CACHE_BLOCK_SIZE + 1024; // Slightly more than one chunk
        create_test_file(&inner, file_path, file_size).await?;

        // Read the file to populate the cache
        verify_range(&cache, file_path, 0..file_size).await?;

        // Modify the original file in the inner store to verify we're reading from cache
        let modified_data = vec![255u8; file_size as usize];
        let path = Path::from(file_path);
        inner.put(&path, Bytes::from(modified_data).into()).await?;

        // Read the same range again - should get the original data from cache, not the modified data
        verify_range(&cache, file_path, 0..file_size).await?;

        Ok(())
    }

    // Test partial range requests
    #[tokio::test]
    async fn test_suffix_range() -> Result<()> {
        let inner = Arc::new(InMemory::new());
        let temp_dir = tempdir().unwrap();
        let cache = ByteCache::new(inner.clone(), temp_dir.path().to_path_buf());

        // Create a file
        let file_path = "suffix_range.bin";
        let file_size = CACHE_BLOCK_SIZE * 2; // 8MB
        create_test_file(&inner, file_path, file_size).await?;

        // Request the last 1MB of the file using GetRange::Suffix
        let path = Path::from(file_path);
        let options = GetOptions {
            range: Some(GetRange::Suffix(1024 * 1024)),
            ..Default::default()
        };

        let result = cache.get_opts(&path, options).await?;
        let data = result.bytes().await?;

        // Verify we got the right data size
        assert_eq!(data.len(), 1024 * 1024);

        // Verify the content matches expected pattern
        let start = file_size - 1024 * 1024;
        for (i, byte) in data.iter().enumerate() {
            let expected = ((start + i as u64) % 256) as u8;
            assert_eq!(*byte, expected, "Mismatch at position {i}");
        }

        Ok(())
    }

    #[tokio::test]
    async fn test_persistent_cache() -> Result<()> {
        // Create an InMemory store as the inner store
        let inner = Arc::new(InMemory::new());
        let temp_dir = tempdir().unwrap();
        let cache_dir_path = temp_dir.path().to_path_buf();

        // Create a file in the inner store
        let file_path = "persistent_test.bin";
        let file_size = CACHE_BLOCK_SIZE + 1024; // Slightly more than one chunk
        create_test_file(&inner, file_path, file_size).await?;

        // First cache instance - read data to populate cache
        {
            let first_cache = ByteCache::new(inner.clone(), cache_dir_path.clone());
            verify_range(&first_cache, file_path, 0..file_size).await?;

            // Verify data was cached
            let chunk_path = first_cache.get_chunk_path(&Path::from(file_path), 0);
            assert!(
                first_cache.is_chunk_ready(&chunk_path),
                "First chunk should be cached"
            );
        }

        // Modify the data in the inner store to verify the second cache uses cached data
        let modified_data = vec![255u8; file_size as usize];
        let path = Path::from(file_path);
        inner.put(&path, Bytes::from(modified_data).into()).await?;

        // Create a new cache instance pointing to the same directory
        let second_cache = ByteCache::new(inner.clone(), cache_dir_path);

        // Read the data through the second cache - should get original data from cache
        verify_range(&second_cache, file_path, 0..file_size).await?;

        // Additional verification - check if a specific range is read correctly
        let mid_range = file_size / 2;
        verify_range(&second_cache, file_path, mid_range..mid_range + 1024).await?;

        Ok(())
    }

    #[tokio::test]
    async fn test_object_store_metrics() -> Result<()> {
        let inner = Arc::new(MockStore::new_with_files(
            1,
            (CACHE_BLOCK_SIZE * 3) as usize,
        ));
        let temp_dir = tempdir().unwrap();
        let cache = ByteCache::new(inner.clone(), temp_dir.path().to_path_buf());

        let path = Path::from("0.parquet");
        let start = CACHE_BLOCK_SIZE / 2;
        let end = CACHE_BLOCK_SIZE + CACHE_BLOCK_SIZE / 2;

        verify_range(&cache, path.as_ref(), start..end).await?;

        let ranges = inner.get_access_ranges(&path).unwrap();
        assert_eq!(ranges.len(), 2);
        assert_eq!(ranges[0], 0..CACHE_BLOCK_SIZE);
        assert_eq!(ranges[1], CACHE_BLOCK_SIZE..CACHE_BLOCK_SIZE * 2);

        // second request should hit cache and not increase range count
        verify_range(&cache, path.as_ref(), start + 1024..end - 1024).await?;
        let ranges_after = inner.get_access_ranges(&path).unwrap();
        assert_eq!(ranges_after.len(), 2);

        Ok(())
    }
}