risingwave_storage/hummock/event_handler/
refiller.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
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
// Copyright 2024 RisingWave Labs
//
// 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::collections::{HashMap, HashSet, VecDeque};
use std::future::poll_fn;
use std::ops::Range;
use std::sync::{Arc, LazyLock};
use std::task::{ready, Poll};
use std::time::{Duration, Instant};

use foyer::{HybridCacheEntry, RangeBoundsExt};
use futures::future::{join_all, try_join_all};
use futures::{Future, FutureExt};
use itertools::Itertools;
use prometheus::core::{AtomicU64, GenericCounter, GenericCounterVec};
use prometheus::{
    register_histogram_vec_with_registry, register_int_counter_vec_with_registry,
    register_int_gauge_with_registry, Histogram, HistogramVec, IntGauge, Registry,
};
use risingwave_common::monitor::GLOBAL_METRICS_REGISTRY;
use risingwave_hummock_sdk::compaction_group::hummock_version_ext::SstDeltaInfo;
use risingwave_hummock_sdk::{HummockSstableObjectId, KeyComparator};
use thiserror_ext::AsReport;
use tokio::sync::Semaphore;
use tokio::task::JoinHandle;

use crate::hummock::local_version::pinned_version::PinnedVersion;
use crate::hummock::{
    Block, HummockError, HummockResult, Sstable, SstableBlockIndex, SstableStoreRef, TableHolder,
};
use crate::monitor::StoreLocalStatistic;
use crate::opts::StorageOpts;

pub static GLOBAL_CACHE_REFILL_METRICS: LazyLock<CacheRefillMetrics> =
    LazyLock::new(|| CacheRefillMetrics::new(&GLOBAL_METRICS_REGISTRY));

pub struct CacheRefillMetrics {
    pub refill_duration: HistogramVec,
    pub refill_total: GenericCounterVec<AtomicU64>,
    pub refill_bytes: GenericCounterVec<AtomicU64>,

    pub data_refill_success_duration: Histogram,
    pub meta_refill_success_duration: Histogram,

    pub data_refill_filtered_total: GenericCounter<AtomicU64>,
    pub data_refill_attempts_total: GenericCounter<AtomicU64>,
    pub data_refill_started_total: GenericCounter<AtomicU64>,
    pub meta_refill_attempts_total: GenericCounter<AtomicU64>,

    pub data_refill_parent_meta_lookup_hit_total: GenericCounter<AtomicU64>,
    pub data_refill_parent_meta_lookup_miss_total: GenericCounter<AtomicU64>,
    pub data_refill_unit_inheritance_hit_total: GenericCounter<AtomicU64>,
    pub data_refill_unit_inheritance_miss_total: GenericCounter<AtomicU64>,

    pub data_refill_block_unfiltered_total: GenericCounter<AtomicU64>,
    pub data_refill_block_success_total: GenericCounter<AtomicU64>,

    pub data_refill_ideal_bytes: GenericCounter<AtomicU64>,
    pub data_refill_success_bytes: GenericCounter<AtomicU64>,

    pub refill_queue_total: IntGauge,
}

impl CacheRefillMetrics {
    pub fn new(registry: &Registry) -> Self {
        let refill_duration = register_histogram_vec_with_registry!(
            "refill_duration",
            "refill duration",
            &["type", "op"],
            registry,
        )
        .unwrap();
        let refill_total = register_int_counter_vec_with_registry!(
            "refill_total",
            "refill total",
            &["type", "op"],
            registry,
        )
        .unwrap();
        let refill_bytes = register_int_counter_vec_with_registry!(
            "refill_bytes",
            "refill bytes",
            &["type", "op"],
            registry,
        )
        .unwrap();

        let data_refill_success_duration = refill_duration
            .get_metric_with_label_values(&["data", "success"])
            .unwrap();
        let meta_refill_success_duration = refill_duration
            .get_metric_with_label_values(&["meta", "success"])
            .unwrap();

        let data_refill_filtered_total = refill_total
            .get_metric_with_label_values(&["data", "filtered"])
            .unwrap();
        let data_refill_attempts_total = refill_total
            .get_metric_with_label_values(&["data", "attempts"])
            .unwrap();
        let data_refill_started_total = refill_total
            .get_metric_with_label_values(&["data", "started"])
            .unwrap();
        let meta_refill_attempts_total = refill_total
            .get_metric_with_label_values(&["meta", "attempts"])
            .unwrap();

        let data_refill_parent_meta_lookup_hit_total = refill_total
            .get_metric_with_label_values(&["parent_meta", "hit"])
            .unwrap();
        let data_refill_parent_meta_lookup_miss_total = refill_total
            .get_metric_with_label_values(&["parent_meta", "miss"])
            .unwrap();
        let data_refill_unit_inheritance_hit_total = refill_total
            .get_metric_with_label_values(&["unit_inheritance", "hit"])
            .unwrap();
        let data_refill_unit_inheritance_miss_total = refill_total
            .get_metric_with_label_values(&["unit_inheritance", "miss"])
            .unwrap();

        let data_refill_block_unfiltered_total = refill_total
            .get_metric_with_label_values(&["block", "unfiltered"])
            .unwrap();
        let data_refill_block_success_total = refill_total
            .get_metric_with_label_values(&["block", "success"])
            .unwrap();

        let data_refill_ideal_bytes = refill_bytes
            .get_metric_with_label_values(&["data", "ideal"])
            .unwrap();
        let data_refill_success_bytes = refill_bytes
            .get_metric_with_label_values(&["data", "success"])
            .unwrap();

        let refill_queue_total = register_int_gauge_with_registry!(
            "refill_queue_total",
            "refill queue total",
            registry,
        )
        .unwrap();

        Self {
            refill_duration,
            refill_total,
            refill_bytes,

            data_refill_success_duration,
            meta_refill_success_duration,
            data_refill_filtered_total,
            data_refill_attempts_total,
            data_refill_started_total,
            meta_refill_attempts_total,

            data_refill_parent_meta_lookup_hit_total,
            data_refill_parent_meta_lookup_miss_total,
            data_refill_unit_inheritance_hit_total,
            data_refill_unit_inheritance_miss_total,

            data_refill_block_unfiltered_total,
            data_refill_block_success_total,

            data_refill_ideal_bytes,
            data_refill_success_bytes,

            refill_queue_total,
        }
    }
}

#[derive(Debug)]
pub struct CacheRefillConfig {
    /// Cache refill timeout.
    pub timeout: Duration,

    /// Data file cache refill levels.
    pub data_refill_levels: HashSet<u32>,

    /// Data file cache refill concurrency.
    pub concurrency: usize,

    /// Data file cache refill unit (blocks).
    pub unit: usize,

    /// Data file cache reill unit threshold.
    ///
    /// Only units whose admit rate > threshold will be refilled.
    pub threshold: f64,
}

impl CacheRefillConfig {
    pub fn from_storage_opts(options: &StorageOpts) -> Self {
        Self {
            timeout: Duration::from_millis(options.cache_refill_timeout_ms),
            data_refill_levels: options
                .cache_refill_data_refill_levels
                .iter()
                .copied()
                .collect(),
            concurrency: options.cache_refill_concurrency,
            unit: options.cache_refill_unit,
            threshold: options.cache_refill_threshold,
        }
    }
}

struct Item {
    handle: JoinHandle<()>,
    event: CacheRefillerEvent,
}

pub(crate) type SpawnRefillTask = Arc<
    // first current version, second new version
    dyn Fn(Vec<SstDeltaInfo>, CacheRefillContext, PinnedVersion, PinnedVersion) -> JoinHandle<()>
        + Send
        + Sync
        + 'static,
>;

/// A cache refiller for hummock data.
pub(crate) struct CacheRefiller {
    /// order: old => new
    queue: VecDeque<Item>,

    context: CacheRefillContext,

    spawn_refill_task: SpawnRefillTask,
}

impl CacheRefiller {
    pub(crate) fn new(
        config: CacheRefillConfig,
        sstable_store: SstableStoreRef,
        spawn_refill_task: SpawnRefillTask,
    ) -> Self {
        let config = Arc::new(config);
        let concurrency = Arc::new(Semaphore::new(config.concurrency));
        Self {
            queue: VecDeque::new(),
            context: CacheRefillContext {
                config,
                concurrency,
                sstable_store,
            },
            spawn_refill_task,
        }
    }

    pub(crate) fn default_spawn_refill_task() -> SpawnRefillTask {
        Arc::new(|deltas, context, _, _| {
            let task = CacheRefillTask { deltas, context };
            tokio::spawn(task.run())
        })
    }

    pub(crate) fn start_cache_refill(
        &mut self,
        deltas: Vec<SstDeltaInfo>,
        pinned_version: PinnedVersion,
        new_pinned_version: PinnedVersion,
    ) {
        let handle = (self.spawn_refill_task)(
            deltas,
            self.context.clone(),
            pinned_version.clone(),
            new_pinned_version.clone(),
        );
        let event = CacheRefillerEvent {
            pinned_version,
            new_pinned_version,
        };
        let item = Item { handle, event };
        self.queue.push_back(item);
        GLOBAL_CACHE_REFILL_METRICS.refill_queue_total.add(1);
    }

    pub(crate) fn last_new_pinned_version(&self) -> Option<&PinnedVersion> {
        self.queue.back().map(|item| &item.event.new_pinned_version)
    }
}

impl CacheRefiller {
    pub(crate) fn next_event(&mut self) -> impl Future<Output = CacheRefillerEvent> + '_ {
        poll_fn(|cx| {
            if let Some(item) = self.queue.front_mut() {
                ready!(item.handle.poll_unpin(cx)).unwrap();
                let item = self.queue.pop_front().unwrap();
                GLOBAL_CACHE_REFILL_METRICS.refill_queue_total.sub(1);
                return Poll::Ready(item.event);
            }
            Poll::Pending
        })
    }
}

pub struct CacheRefillerEvent {
    pub pinned_version: PinnedVersion,
    pub new_pinned_version: PinnedVersion,
}

#[derive(Clone)]
pub(crate) struct CacheRefillContext {
    config: Arc<CacheRefillConfig>,
    concurrency: Arc<Semaphore>,
    sstable_store: SstableStoreRef,
}

struct CacheRefillTask {
    deltas: Vec<SstDeltaInfo>,
    context: CacheRefillContext,
}

impl CacheRefillTask {
    async fn run(self) {
        let tasks = self
            .deltas
            .iter()
            .map(|delta| {
                let context = self.context.clone();
                async move {
                    let holders = match Self::meta_cache_refill(&context, delta).await {
                        Ok(holders) => holders,
                        Err(e) => {
                            tracing::warn!(error = %e.as_report(), "meta cache refill error");
                            return;
                        }
                    };
                    Self::data_cache_refill(&context, delta, holders).await;
                }
            })
            .collect_vec();
        let future = join_all(tasks);

        let _ = tokio::time::timeout(self.context.config.timeout, future).await;
    }

    async fn meta_cache_refill(
        context: &CacheRefillContext,
        delta: &SstDeltaInfo,
    ) -> HummockResult<Vec<TableHolder>> {
        let tasks = delta
            .insert_sst_infos
            .iter()
            .map(|info| async {
                let mut stats = StoreLocalStatistic::default();
                GLOBAL_CACHE_REFILL_METRICS.meta_refill_attempts_total.inc();

                let now = Instant::now();
                let res = context.sstable_store.sstable(info, &mut stats).await;
                stats.discard();
                GLOBAL_CACHE_REFILL_METRICS
                    .meta_refill_success_duration
                    .observe(now.elapsed().as_secs_f64());
                res
            })
            .collect_vec();
        let holders = try_join_all(tasks).await?;
        Ok(holders)
    }

    /// Get sstable inheritance info in unit level.
    fn get_units_to_refill_by_inheritance(
        context: &CacheRefillContext,
        ssts: &[TableHolder],
        parent_ssts: impl IntoIterator<Item = HybridCacheEntry<HummockSstableObjectId, Box<Sstable>>>,
    ) -> HashSet<SstableUnit> {
        let mut res = HashSet::default();

        let Some(filter) = context.sstable_store.data_recent_filter() else {
            return res;
        };

        let units = {
            let unit = context.config.unit;
            ssts.iter()
                .flat_map(|sst| {
                    let units = Unit::units(sst, unit);
                    (0..units).map(|uidx| Unit::new(sst, unit, uidx))
                })
                .collect_vec()
        };

        if cfg!(debug_assertions) {
            // assert units in asc order
            units.iter().tuple_windows().for_each(|(a, b)| {
                debug_assert_ne!(
                    KeyComparator::compare_encoded_full_key(a.largest_key(), b.smallest_key()),
                    std::cmp::Ordering::Greater
                )
            });
        }

        for psst in parent_ssts {
            for pblk in 0..psst.block_count() {
                let pleft = &psst.meta.block_metas[pblk].smallest_key;
                let pright = if pblk + 1 == psst.block_count() {
                    // `largest_key` can be included or excluded, both are treated as included here
                    &psst.meta.largest_key
                } else {
                    &psst.meta.block_metas[pblk + 1].smallest_key
                };

                // partition point: unit.right < pblk.left
                let uleft = units.partition_point(|unit| {
                    KeyComparator::compare_encoded_full_key(unit.largest_key(), pleft)
                        == std::cmp::Ordering::Less
                });
                // partition point: unit.left <= pblk.right
                let uright = units.partition_point(|unit| {
                    KeyComparator::compare_encoded_full_key(unit.smallest_key(), pright)
                        != std::cmp::Ordering::Greater
                });

                // overlapping: uleft..uright
                for u in units.iter().take(uright).skip(uleft) {
                    let unit = SstableUnit {
                        sst_obj_id: u.sst.id,
                        blks: u.blks.clone(),
                    };
                    if res.contains(&unit) {
                        continue;
                    }
                    if filter.contains(&(psst.id, pblk)) {
                        res.insert(unit);
                    }
                }
            }
        }

        let hit = res.len();
        let miss = units.len() - res.len();
        GLOBAL_CACHE_REFILL_METRICS
            .data_refill_unit_inheritance_hit_total
            .inc_by(hit as u64);
        GLOBAL_CACHE_REFILL_METRICS
            .data_refill_unit_inheritance_miss_total
            .inc_by(miss as u64);

        res
    }

    async fn data_cache_refill(
        context: &CacheRefillContext,
        delta: &SstDeltaInfo,
        holders: Vec<TableHolder>,
    ) {
        // return if data file cache is disabled
        let Some(filter) = context.sstable_store.data_recent_filter() else {
            return;
        };

        // return if no data to refill
        if delta.insert_sst_infos.is_empty() || delta.delete_sst_object_ids.is_empty() {
            return;
        }

        // return if recent filter miss
        if !context
            .config
            .data_refill_levels
            .contains(&delta.insert_sst_level)
            || !delta
                .delete_sst_object_ids
                .iter()
                .any(|&id| filter.contains(&(id, usize::MAX)))
        {
            GLOBAL_CACHE_REFILL_METRICS.data_refill_filtered_total.inc();
            return;
        }

        GLOBAL_CACHE_REFILL_METRICS
            .data_refill_block_unfiltered_total
            .inc_by(
                holders
                    .iter()
                    .map(|sst| sst.block_count() as u64)
                    .sum::<u64>(),
            );

        if delta.insert_sst_level == 0 {
            Self::data_file_cache_refill_l0_impl(context, delta, holders).await;
        } else {
            Self::data_file_cache_impl(context, delta, holders).await;
        }
    }

    async fn data_file_cache_refill_l0_impl(
        context: &CacheRefillContext,
        _delta: &SstDeltaInfo,
        holders: Vec<TableHolder>,
    ) {
        let unit = context.config.unit;

        let mut futures = vec![];

        for sst in &holders {
            for blk_start in (0..sst.block_count()).step_by(unit) {
                let blk_end = std::cmp::min(sst.block_count(), blk_start + unit);
                let unit = SstableUnit {
                    sst_obj_id: sst.id,
                    blks: blk_start..blk_end,
                };
                futures.push(
                    async move { Self::data_file_cache_refill_unit(context, sst, unit).await },
                );
            }
        }
        join_all(futures).await;
    }

    async fn data_file_cache_impl(
        context: &CacheRefillContext,
        delta: &SstDeltaInfo,
        holders: Vec<TableHolder>,
    ) {
        let sstable_store = context.sstable_store.clone();
        let futures = delta.delete_sst_object_ids.iter().map(|sst_obj_id| {
            let store = &sstable_store;
            async move {
                let res = store.sstable_cached(*sst_obj_id).await;
                match res {
                    Ok(Some(_)) => GLOBAL_CACHE_REFILL_METRICS
                        .data_refill_parent_meta_lookup_hit_total
                        .inc(),
                    Ok(None) => GLOBAL_CACHE_REFILL_METRICS
                        .data_refill_parent_meta_lookup_miss_total
                        .inc(),
                    _ => {}
                }
                res
            }
        });
        let parent_ssts = match try_join_all(futures).await {
            Ok(parent_ssts) => parent_ssts.into_iter().flatten(),
            Err(e) => {
                return tracing::error!(error = %e.as_report(), "get old meta from cache error")
            }
        };
        let units = Self::get_units_to_refill_by_inheritance(context, &holders, parent_ssts);

        let ssts: HashMap<HummockSstableObjectId, TableHolder> =
            holders.into_iter().map(|meta| (meta.id, meta)).collect();
        let futures = units.into_iter().map(|unit| {
            let ssts = &ssts;
            async move {
                let sst = ssts.get(&unit.sst_obj_id).unwrap();
                if let Err(e) = Self::data_file_cache_refill_unit(context, sst, unit).await {
                    tracing::error!(error = %e.as_report(), "data file cache unit refill error");
                }
            }
        });
        join_all(futures).await;
    }

    async fn data_file_cache_refill_unit(
        context: &CacheRefillContext,
        sst: &Sstable,
        unit: SstableUnit,
    ) -> HummockResult<()> {
        let sstable_store = &context.sstable_store;
        let threshold = context.config.threshold;

        // update filter for sst id only
        if let Some(filter) = sstable_store.data_recent_filter() {
            filter.insert((sst.id, usize::MAX));
        }

        let blocks = unit.blks.size().unwrap();

        let mut tasks = vec![];
        let mut contexts = Vec::with_capacity(blocks);
        let mut admits = 0;

        let (range_first, _) = sst.calculate_block_info(unit.blks.start);
        let (range_last, _) = sst.calculate_block_info(unit.blks.end - 1);
        let range = range_first.start..range_last.end;

        GLOBAL_CACHE_REFILL_METRICS
            .data_refill_ideal_bytes
            .inc_by(range.size().unwrap() as u64);

        for blk in unit.blks {
            let (range, uncompressed_capacity) = sst.calculate_block_info(blk);
            let key = SstableBlockIndex {
                sst_id: sst.id,
                block_idx: blk as u64,
            };

            let mut writer = sstable_store.block_cache().storage_writer(key);

            if writer.pick() {
                admits += 1;
            }

            contexts.push((writer, range, uncompressed_capacity))
        }

        if admits as f64 / contexts.len() as f64 >= threshold {
            let task = async move {
                GLOBAL_CACHE_REFILL_METRICS.data_refill_attempts_total.inc();

                let permit = context.concurrency.acquire().await.unwrap();

                GLOBAL_CACHE_REFILL_METRICS.data_refill_started_total.inc();

                let timer = GLOBAL_CACHE_REFILL_METRICS
                    .data_refill_success_duration
                    .start_timer();

                let data = sstable_store
                    .store()
                    .read(&sstable_store.get_sst_data_path(sst.id), range.clone())
                    .await?;
                let mut futures = vec![];
                for (w, r, uc) in contexts {
                    let offset = r.start - range.start;
                    let len = r.end - r.start;
                    let bytes = data.slice(offset..offset + len);
                    let future = async move {
                        let value = Box::new(Block::decode(bytes, uc)?);
                        // The entry should always be `Some(..)`, use if here for compatible.
                        if let Some(_entry) = w.force().insert(value) {
                            GLOBAL_CACHE_REFILL_METRICS
                                .data_refill_success_bytes
                                .inc_by(len as u64);
                            GLOBAL_CACHE_REFILL_METRICS
                                .data_refill_block_success_total
                                .inc();
                        }
                        Ok::<_, HummockError>(())
                    };
                    futures.push(future);
                }
                try_join_all(futures)
                    .await
                    .map_err(HummockError::file_cache)?;

                drop(permit);
                drop(timer);

                Ok::<_, HummockError>(())
            };
            tasks.push(task);
        }

        try_join_all(tasks).await?;

        Ok(())
    }
}

#[derive(Debug)]
pub struct SstableBlock {
    pub sst_obj_id: HummockSstableObjectId,
    pub blk_idx: usize,
}

#[derive(Debug, Hash, PartialEq, Eq)]
pub struct SstableUnit {
    pub sst_obj_id: HummockSstableObjectId,
    pub blks: Range<usize>,
}

impl Ord for SstableUnit {
    fn cmp(&self, other: &Self) -> std::cmp::Ordering {
        match self.sst_obj_id.cmp(&other.sst_obj_id) {
            std::cmp::Ordering::Equal => {}
            ord => return ord,
        }
        match self.blks.start.cmp(&other.blks.start) {
            std::cmp::Ordering::Equal => {}
            ord => return ord,
        }
        self.blks.end.cmp(&other.blks.end)
    }
}

impl PartialOrd for SstableUnit {
    fn partial_cmp(&self, other: &Self) -> Option<std::cmp::Ordering> {
        Some(self.cmp(other))
    }
}

#[derive(Debug)]
struct Unit<'a> {
    sst: &'a Sstable,
    blks: Range<usize>,
}

impl<'a> Unit<'a> {
    fn new(sst: &'a Sstable, unit: usize, uidx: usize) -> Self {
        let blks = unit * uidx..std::cmp::min(unit * (uidx + 1), sst.block_count());
        Self { sst, blks }
    }

    fn smallest_key(&self) -> &Vec<u8> {
        &self.sst.meta.block_metas[self.blks.start].smallest_key
    }

    // `largest_key` can be included or excluded, both are treated as included here
    fn largest_key(&self) -> &Vec<u8> {
        if self.blks.end == self.sst.block_count() {
            &self.sst.meta.largest_key
        } else {
            &self.sst.meta.block_metas[self.blks.end].smallest_key
        }
    }

    fn units(sst: &Sstable, unit: usize) -> usize {
        sst.block_count() / unit + if sst.block_count() % unit == 0 { 0 } else { 1 }
    }
}