risingwave_storage/hummock/
time_travel_version_cache.rs1use std::future::Future;
16use std::pin::Pin;
17
18use futures::FutureExt;
19use futures::future::Shared;
20use moka::sync::Cache;
21use risingwave_common::id::TableId;
22use risingwave_hummock_sdk::HummockEpoch;
23
24use crate::hummock::HummockResult;
25use crate::hummock::local_version::pinned_version::PinnedVersion;
26
27type InflightResult = Shared<Pin<Box<dyn Future<Output = HummockResult<PinnedVersion>> + Send>>>;
28
29pub struct SimpleTimeTravelVersionCache {
31 cache: Cache<(TableId, HummockEpoch), InflightResult>,
32}
33
34impl SimpleTimeTravelVersionCache {
35 pub fn new(capacity: u64) -> Self {
36 let cache = Cache::builder().max_capacity(capacity).build();
37 Self { cache }
38 }
39
40 pub async fn get_or_insert(
41 &self,
42 table_id: TableId,
43 epoch: HummockEpoch,
44 fetch: impl Future<Output = HummockResult<PinnedVersion>> + Send + 'static,
45 ) -> HummockResult<PinnedVersion> {
46 self.cache
47 .entry((table_id, epoch))
48 .or_insert_with_if(
49 || fetch.boxed().shared(),
50 |inflight| {
51 if let Some(result) = inflight.peek() {
52 return result.is_err();
53 }
54 false
55 },
56 )
57 .value()
58 .clone()
59 .await
60 }
61}