catalog/kvbackend/
client.rs

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
601
602
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::any::Any;
use std::fmt::Debug;
use std::sync::atomic::{AtomicUsize, Ordering};
use std::sync::{Arc, Mutex};
use std::time::Duration;

use common_error::ext::BoxedError;
use common_meta::cache_invalidator::KvCacheInvalidator;
use common_meta::error::Error::CacheNotGet;
use common_meta::error::{CacheNotGetSnafu, Error, ExternalSnafu, GetKvCacheSnafu, Result};
use common_meta::kv_backend::{KvBackend, KvBackendRef, TxnService};
use common_meta::rpc::store::{
    BatchDeleteRequest, BatchDeleteResponse, BatchGetRequest, BatchGetResponse, BatchPutRequest,
    BatchPutResponse, CompareAndPutRequest, CompareAndPutResponse, DeleteRangeRequest,
    DeleteRangeResponse, PutRequest, PutResponse, RangeRequest, RangeResponse,
};
use common_meta::rpc::KeyValue;
use common_telemetry::debug;
use meta_client::client::MetaClient;
use moka::future::{Cache, CacheBuilder};
use snafu::{OptionExt, ResultExt};

use crate::metrics::{
    METRIC_CATALOG_KV_BATCH_GET, METRIC_CATALOG_KV_GET, METRIC_CATALOG_KV_REMOTE_GET,
};

const DEFAULT_CACHE_MAX_CAPACITY: u64 = 10000;
const DEFAULT_CACHE_TTL: Duration = Duration::from_secs(10 * 60);
const DEFAULT_CACHE_TTI: Duration = Duration::from_secs(5 * 60);

pub struct CachedMetaKvBackendBuilder {
    cache_max_capacity: Option<u64>,
    cache_ttl: Option<Duration>,
    cache_tti: Option<Duration>,
    meta_client: Arc<MetaClient>,
}

impl CachedMetaKvBackendBuilder {
    pub fn new(meta_client: Arc<MetaClient>) -> Self {
        Self {
            cache_max_capacity: None,
            cache_ttl: None,
            cache_tti: None,
            meta_client,
        }
    }

    pub fn cache_max_capacity(mut self, cache_max_capacity: u64) -> Self {
        self.cache_max_capacity.replace(cache_max_capacity);
        self
    }

    pub fn cache_ttl(mut self, cache_ttl: Duration) -> Self {
        self.cache_ttl.replace(cache_ttl);
        self
    }

    pub fn cache_tti(mut self, cache_tti: Duration) -> Self {
        self.cache_tti.replace(cache_tti);
        self
    }

    pub fn build(self) -> CachedMetaKvBackend {
        let cache_max_capacity = self
            .cache_max_capacity
            .unwrap_or(DEFAULT_CACHE_MAX_CAPACITY);
        let cache_ttl = self.cache_ttl.unwrap_or(DEFAULT_CACHE_TTL);
        let cache_tti = self.cache_tti.unwrap_or(DEFAULT_CACHE_TTI);

        let cache = CacheBuilder::new(cache_max_capacity)
            .time_to_live(cache_ttl)
            .time_to_idle(cache_tti)
            .build();

        let kv_backend = Arc::new(MetaKvBackend {
            client: self.meta_client,
        });
        let name = format!("CachedKvBackend({})", kv_backend.name());
        let version = AtomicUsize::new(0);

        CachedMetaKvBackend {
            kv_backend,
            cache,
            name,
            version,
        }
    }
}

pub type CacheBackend = Cache<Vec<u8>, KeyValue>;

/// A wrapper of `MetaKvBackend` with cache support.
///
/// CachedMetaKvBackend is mainly used to read metadata information from Metasrv, and provides
/// cache for get and batch_get. One way to trigger cache invalidation of CachedMetaKvBackend:
/// when metadata information changes, Metasrv will broadcast a metadata invalidation request.
///
/// Therefore, it is recommended to use CachedMetaKvBackend to only read metadata related
/// information. Note: If you read other information, you may read expired data, which depends on
/// TTL and TTI for cache.
pub struct CachedMetaKvBackend {
    kv_backend: KvBackendRef,
    cache: CacheBackend,
    name: String,
    version: AtomicUsize,
}

impl TxnService for CachedMetaKvBackend {
    type Error = Error;
}

#[async_trait::async_trait]
impl KvBackend for CachedMetaKvBackend {
    fn name(&self) -> &str {
        &self.name
    }

    fn as_any(&self) -> &dyn Any {
        self
    }

    async fn range(&self, req: RangeRequest) -> Result<RangeResponse> {
        self.kv_backend.range(req).await
    }

    async fn put(&self, req: PutRequest) -> Result<PutResponse> {
        let key = &req.key.clone();

        let ret = self.kv_backend.put(req).await;

        if ret.is_ok() {
            self.invalidate_key(key).await;
        }

        ret
    }

    async fn batch_put(&self, req: BatchPutRequest) -> Result<BatchPutResponse> {
        let keys = req
            .kvs
            .iter()
            .map(|kv| kv.key().to_vec())
            .collect::<Vec<_>>();

        let resp = self.kv_backend.batch_put(req).await;

        if resp.is_ok() {
            for key in keys {
                self.invalidate_key(&key).await;
            }
        }

        resp
    }

    async fn batch_get(&self, req: BatchGetRequest) -> Result<BatchGetResponse> {
        let _timer = METRIC_CATALOG_KV_BATCH_GET.start_timer();

        let mut kvs = Vec::with_capacity(req.keys.len());
        let mut miss_keys = Vec::with_capacity(req.keys.len());

        for key in req.keys {
            if let Some(val) = self.cache.get(&key).await {
                kvs.push(val);
            } else {
                miss_keys.push(key);
            }
        }

        let batch_get_req = BatchGetRequest::new().with_keys(miss_keys.clone());

        let pre_version = self.version();

        let unhit_kvs = self.kv_backend.batch_get(batch_get_req).await?.kvs;

        for kv in unhit_kvs.iter() {
            self.cache.insert(kv.key().to_vec(), kv.clone()).await;
        }

        if !self.validate_version(pre_version) {
            for key in miss_keys.iter() {
                self.cache.invalidate(key).await;
            }
        }

        kvs.extend(unhit_kvs);

        Ok(BatchGetResponse { kvs })
    }

    async fn compare_and_put(&self, req: CompareAndPutRequest) -> Result<CompareAndPutResponse> {
        let key = &req.key.clone();

        let ret = self.kv_backend.compare_and_put(req).await;

        if ret.is_ok() {
            self.invalidate_key(key).await;
        }

        ret
    }

    async fn delete_range(&self, mut req: DeleteRangeRequest) -> Result<DeleteRangeResponse> {
        let prev_kv = req.prev_kv;

        req.prev_kv = true;
        let resp = self.kv_backend.delete_range(req).await;
        match resp {
            Ok(mut resp) => {
                for prev_kv in resp.prev_kvs.iter() {
                    self.invalidate_key(prev_kv.key()).await;
                }

                if !prev_kv {
                    resp.prev_kvs = vec![];
                }
                Ok(resp)
            }
            Err(e) => Err(e),
        }
    }

    async fn batch_delete(&self, mut req: BatchDeleteRequest) -> Result<BatchDeleteResponse> {
        let prev_kv = req.prev_kv;

        req.prev_kv = true;
        let resp = self.kv_backend.batch_delete(req).await;
        match resp {
            Ok(mut resp) => {
                for prev_kv in resp.prev_kvs.iter() {
                    self.invalidate_key(prev_kv.key()).await;
                }

                if !prev_kv {
                    resp.prev_kvs = vec![];
                }
                Ok(resp)
            }
            Err(e) => Err(e),
        }
    }

    async fn get(&self, key: &[u8]) -> Result<Option<KeyValue>> {
        let _timer = METRIC_CATALOG_KV_GET.start_timer();

        let pre_version = Arc::new(Mutex::new(None));

        let init = async {
            let version_clone = pre_version.clone();
            let _timer = METRIC_CATALOG_KV_REMOTE_GET.start_timer();

            version_clone.lock().unwrap().replace(self.version());

            self.kv_backend.get(key).await.map(|val| {
                val.with_context(|| CacheNotGetSnafu {
                    key: String::from_utf8_lossy(key),
                })
            })?
        };

        // currently moka doesn't have `optionally_try_get_with_by_ref`
        // TODO(fys): change to moka method when available
        // https://github.com/moka-rs/moka/issues/254
        let ret = match self.cache.try_get_with_by_ref(key, init).await {
            Ok(val) => Ok(Some(val)),
            Err(e) => match e.as_ref() {
                CacheNotGet { .. } => Ok(None),
                _ => Err(e),
            },
        }
        .map_err(|e| {
            GetKvCacheSnafu {
                err_msg: e.to_string(),
            }
            .build()
        });

        // "cache.invalidate_key" and "cache.try_get_with_by_ref" are not mutually exclusive. So we need
        // to use the version mechanism to prevent expired data from being put into the cache.
        if pre_version
            .lock()
            .unwrap()
            .as_ref()
            .map_or(false, |v| !self.validate_version(*v))
        {
            self.cache.invalidate(key).await;
        }

        ret
    }
}

#[async_trait::async_trait]
impl KvCacheInvalidator for CachedMetaKvBackend {
    async fn invalidate_key(&self, key: &[u8]) {
        self.create_new_version();
        self.cache.invalidate(key).await;
        debug!("invalidated cache key: {}", String::from_utf8_lossy(key));
    }
}

impl CachedMetaKvBackend {
    // only for test
    #[cfg(test)]
    fn wrap(kv_backend: KvBackendRef) -> Self {
        let cache = CacheBuilder::new(DEFAULT_CACHE_MAX_CAPACITY)
            .time_to_live(DEFAULT_CACHE_TTL)
            .time_to_idle(DEFAULT_CACHE_TTI)
            .build();

        let name = format!("CachedKvBackend({})", kv_backend.name());
        Self {
            kv_backend,
            cache,
            name,
            version: AtomicUsize::new(0),
        }
    }

    pub fn cache(&self) -> &CacheBackend {
        &self.cache
    }

    fn version(&self) -> usize {
        self.version.load(Ordering::Relaxed)
    }

    fn validate_version(&self, pre_version: usize) -> bool {
        self.version() == pre_version
    }

    fn create_new_version(&self) -> usize {
        self.version.fetch_add(1, Ordering::Relaxed) + 1
    }
}

#[derive(Debug)]
pub struct MetaKvBackend {
    pub client: Arc<MetaClient>,
}

impl MetaKvBackend {
    /// Constructs a [MetaKvBackend].
    pub fn new(client: Arc<MetaClient>) -> MetaKvBackend {
        MetaKvBackend { client }
    }
}

impl TxnService for MetaKvBackend {
    type Error = Error;
}

/// Implement `KvBackend` trait for `MetaKvBackend` instead of opendal's `Accessor` since
/// `MetaClient`'s range method can return both keys and values, which can reduce IO overhead
/// comparing to `Accessor`'s list and get method.
#[async_trait::async_trait]
impl KvBackend for MetaKvBackend {
    fn name(&self) -> &str {
        "MetaKvBackend"
    }

    fn as_any(&self) -> &dyn Any {
        self
    }

    async fn range(&self, req: RangeRequest) -> Result<RangeResponse> {
        self.client
            .range(req)
            .await
            .map_err(BoxedError::new)
            .context(ExternalSnafu)
    }

    async fn put(&self, req: PutRequest) -> Result<PutResponse> {
        self.client
            .put(req)
            .await
            .map_err(BoxedError::new)
            .context(ExternalSnafu)
    }

    async fn batch_put(&self, req: BatchPutRequest) -> Result<BatchPutResponse> {
        self.client
            .batch_put(req)
            .await
            .map_err(BoxedError::new)
            .context(ExternalSnafu)
    }

    async fn batch_get(&self, req: BatchGetRequest) -> Result<BatchGetResponse> {
        self.client
            .batch_get(req)
            .await
            .map_err(BoxedError::new)
            .context(ExternalSnafu)
    }

    async fn compare_and_put(
        &self,
        request: CompareAndPutRequest,
    ) -> Result<CompareAndPutResponse> {
        self.client
            .compare_and_put(request)
            .await
            .map_err(BoxedError::new)
            .context(ExternalSnafu)
    }

    async fn delete_range(&self, req: DeleteRangeRequest) -> Result<DeleteRangeResponse> {
        self.client
            .delete_range(req)
            .await
            .map_err(BoxedError::new)
            .context(ExternalSnafu)
    }

    async fn batch_delete(&self, req: BatchDeleteRequest) -> Result<BatchDeleteResponse> {
        self.client
            .batch_delete(req)
            .await
            .map_err(BoxedError::new)
            .context(ExternalSnafu)
    }

    async fn get(&self, key: &[u8]) -> Result<Option<KeyValue>> {
        let mut response = self
            .client
            .range(RangeRequest::new().with_key(key))
            .await
            .map_err(BoxedError::new)
            .context(ExternalSnafu)?;
        Ok(response.take_kvs().get_mut(0).map(|kv| KeyValue {
            key: kv.take_key(),
            value: kv.take_value(),
        }))
    }
}

#[cfg(test)]
mod tests {
    use std::any::Any;
    use std::sync::atomic::{AtomicU32, Ordering};
    use std::sync::Arc;

    use async_trait::async_trait;
    use common_meta::kv_backend::{KvBackend, TxnService};
    use common_meta::rpc::store::{
        BatchDeleteRequest, BatchDeleteResponse, BatchGetRequest, BatchGetResponse,
        BatchPutRequest, BatchPutResponse, DeleteRangeRequest, DeleteRangeResponse, PutRequest,
        PutResponse, RangeRequest, RangeResponse,
    };
    use common_meta::rpc::KeyValue;
    use dashmap::DashMap;

    use super::CachedMetaKvBackend;

    #[derive(Default)]
    pub struct SimpleKvBackend {
        inner_map: DashMap<Vec<u8>, Vec<u8>>,
        get_execute_times: Arc<AtomicU32>,
    }

    impl TxnService for SimpleKvBackend {
        type Error = common_meta::error::Error;
    }

    #[async_trait]
    impl KvBackend for SimpleKvBackend {
        fn name(&self) -> &str {
            "SimpleKvBackend"
        }

        fn as_any(&self) -> &dyn Any {
            self
        }

        async fn batch_get(&self, req: BatchGetRequest) -> Result<BatchGetResponse, Self::Error> {
            let mut kvs = Vec::with_capacity(req.keys.len());
            for key in req.keys.iter() {
                if let Some(kv) = self.get(key).await? {
                    kvs.push(kv);
                }
            }
            Ok(BatchGetResponse { kvs })
        }

        async fn put(&self, req: PutRequest) -> Result<PutResponse, Self::Error> {
            self.inner_map.insert(req.key, req.value);
            // always return None as prev_kv, since we don't use it in this test.
            Ok(PutResponse { prev_kv: None })
        }

        async fn get(&self, key: &[u8]) -> Result<Option<KeyValue>, Self::Error> {
            self.get_execute_times
                .fetch_add(1, std::sync::atomic::Ordering::SeqCst);
            Ok(self.inner_map.get(key).map(|v| KeyValue {
                key: key.to_vec(),
                value: v.value().clone(),
            }))
        }

        async fn range(&self, _req: RangeRequest) -> Result<RangeResponse, Self::Error> {
            unimplemented!()
        }

        async fn batch_put(&self, _req: BatchPutRequest) -> Result<BatchPutResponse, Self::Error> {
            unimplemented!()
        }

        async fn delete_range(
            &self,
            _req: DeleteRangeRequest,
        ) -> Result<DeleteRangeResponse, Self::Error> {
            unimplemented!()
        }

        async fn batch_delete(
            &self,
            _req: BatchDeleteRequest,
        ) -> Result<BatchDeleteResponse, Self::Error> {
            unimplemented!()
        }
    }

    #[tokio::test]
    async fn test_cached_kv_backend() {
        let simple_kv = Arc::new(SimpleKvBackend::default());
        let get_execute_times = simple_kv.get_execute_times.clone();
        let cached_kv = CachedMetaKvBackend::wrap(simple_kv);

        add_some_vals(&cached_kv).await;

        let batch_get_req = BatchGetRequest {
            keys: vec![b"k1".to_vec(), b"k2".to_vec()],
        };

        assert_eq!(get_execute_times.load(Ordering::SeqCst), 0);

        for _ in 0..10 {
            let _batch_get_resp = cached_kv.batch_get(batch_get_req.clone()).await.unwrap();

            assert_eq!(get_execute_times.load(Ordering::SeqCst), 2);
        }

        let batch_get_req = BatchGetRequest {
            keys: vec![b"k1".to_vec(), b"k2".to_vec(), b"k3".to_vec()],
        };

        let _batch_get_resp = cached_kv.batch_get(batch_get_req.clone()).await.unwrap();

        assert_eq!(get_execute_times.load(Ordering::SeqCst), 3);

        for _ in 0..10 {
            let _batch_get_resp = cached_kv.batch_get(batch_get_req.clone()).await.unwrap();

            assert_eq!(get_execute_times.load(Ordering::SeqCst), 3);
        }
    }

    async fn add_some_vals(kv_backend: &impl KvBackend) {
        kv_backend
            .put(PutRequest {
                key: b"k1".to_vec(),
                value: b"v1".to_vec(),
                prev_kv: false,
            })
            .await
            .unwrap();

        kv_backend
            .put(PutRequest {
                key: b"k2".to_vec(),
                value: b"v2".to_vec(),
                prev_kv: false,
            })
            .await
            .unwrap();

        kv_backend
            .put(PutRequest {
                key: b"k3".to_vec(),
                value: b"v3".to_vec(),
                prev_kv: false,
            })
            .await
            .unwrap();
    }
}