risingwave_meta/barrier/
progress.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
// 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::hash_map::Entry;
use std::collections::HashMap;
use std::mem::take;

use risingwave_common::catalog::TableId;
use risingwave_common::util::epoch::Epoch;
use risingwave_meta_model::ObjectId;
use risingwave_pb::catalog::CreateType;
use risingwave_pb::ddl_service::DdlProgress;
use risingwave_pb::hummock::HummockVersionStats;
use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress;
use risingwave_pb::stream_service::PbBarrierCompleteResponse;

use crate::barrier::info::BarrierInfo;
use crate::barrier::{
    Command, CreateStreamingJobCommandInfo, CreateStreamingJobType, ReplaceTablePlan,
};
use crate::manager::{DdlType, MetadataManager};
use crate::model::{ActorId, BackfillUpstreamType, StreamJobFragments};
use crate::MetaResult;

type ConsumedRows = u64;

#[derive(Clone, Copy, Debug)]
enum BackfillState {
    Init,
    ConsumingUpstream(#[expect(dead_code)] Epoch, ConsumedRows),
    Done(ConsumedRows),
}

/// Progress of all actors containing backfill executors while creating mview.
#[derive(Debug)]
pub(super) struct Progress {
    // `states` and `done_count` decides whether the progress is done. See `is_done`.
    states: HashMap<ActorId, BackfillState>,
    done_count: usize,

    /// Tells whether the backfill is from source or mv.
    backfill_upstream_types: HashMap<ActorId, BackfillUpstreamType>,

    // The following row counts are used to calculate the progress. See `calculate_progress`.
    /// Upstream mv count.
    /// Keep track of how many times each upstream MV
    /// appears in this stream job.
    upstream_mv_count: HashMap<TableId, usize>,
    /// Total key count of all the upstream materialized views
    upstream_mvs_total_key_count: u64,
    mv_backfill_consumed_rows: u64,
    source_backfill_consumed_rows: u64,

    /// DDL definition
    definition: String,
}

impl Progress {
    /// Create a [`Progress`] for some creating mview, with all `actors` containing the backfill executors.
    fn new(
        actors: impl IntoIterator<Item = (ActorId, BackfillUpstreamType)>,
        upstream_mv_count: HashMap<TableId, usize>,
        upstream_total_key_count: u64,
        definition: String,
    ) -> Self {
        let mut states = HashMap::new();
        let mut backfill_upstream_types = HashMap::new();
        for (actor, backfill_upstream_type) in actors {
            states.insert(actor, BackfillState::Init);
            backfill_upstream_types.insert(actor, backfill_upstream_type);
        }
        assert!(!states.is_empty());

        Self {
            states,
            backfill_upstream_types,
            done_count: 0,
            upstream_mv_count,
            upstream_mvs_total_key_count: upstream_total_key_count,
            mv_backfill_consumed_rows: 0,
            source_backfill_consumed_rows: 0,
            definition,
        }
    }

    /// Update the progress of `actor`.
    fn update(&mut self, actor: ActorId, new_state: BackfillState, upstream_total_key_count: u64) {
        self.upstream_mvs_total_key_count = upstream_total_key_count;
        let total_actors = self.states.len();
        let backfill_upstream_type = self.backfill_upstream_types.get(&actor).unwrap();
        tracing::debug!(?actor, states = ?self.states, "update progress for actor");

        let mut old = 0;
        let mut new = 0;
        match self.states.remove(&actor).unwrap() {
            BackfillState::Init => {}
            BackfillState::ConsumingUpstream(_, old_consumed_rows) => {
                old = old_consumed_rows;
            }
            BackfillState::Done(_) => panic!("should not report done multiple times"),
        };
        match &new_state {
            BackfillState::Init => {}
            BackfillState::ConsumingUpstream(_, new_consumed_rows) => {
                new = *new_consumed_rows;
            }
            BackfillState::Done(new_consumed_rows) => {
                tracing::debug!("actor {} done", actor);
                new = *new_consumed_rows;
                self.done_count += 1;
                tracing::debug!(
                    "{} actors out of {} complete",
                    self.done_count,
                    total_actors,
                );
            }
        };
        debug_assert!(new >= old, "backfill progress should not go backward");
        match backfill_upstream_type {
            BackfillUpstreamType::MView => {
                self.mv_backfill_consumed_rows += new - old;
            }
            BackfillUpstreamType::Source => {
                self.source_backfill_consumed_rows += new - old;
            }
            BackfillUpstreamType::Values => {
                // do not consider progress for values
            }
        }
        self.states.insert(actor, new_state);
    }

    /// Returns whether all backfill executors are done.
    fn is_done(&self) -> bool {
        tracing::trace!(
            "Progress::is_done? {}, {}, {:?}",
            self.done_count,
            self.states.len(),
            self.states
        );
        self.done_count == self.states.len()
    }

    /// Returns the ids of all actors containing the backfill executors for the mview tracked by this
    /// [`Progress`].
    fn actors(&self) -> impl Iterator<Item = ActorId> + '_ {
        self.states.keys().cloned()
    }

    /// `progress` = `consumed_rows` / `upstream_total_key_count`
    fn calculate_progress(&self) -> String {
        if self.is_done() || self.states.is_empty() {
            return "100%".to_string();
        }
        let mut mv_count = 0;
        let mut source_count = 0;
        for backfill_upstream_type in self.backfill_upstream_types.values() {
            match backfill_upstream_type {
                BackfillUpstreamType::MView => mv_count += 1,
                BackfillUpstreamType::Source => source_count += 1,
                BackfillUpstreamType::Values => (),
            }
        }

        let mv_progress = (mv_count > 0).then_some({
            if self.upstream_mvs_total_key_count == 0 {
                "99.99%".to_string()
            } else {
                let mut progress = self.mv_backfill_consumed_rows as f64
                    / (self.upstream_mvs_total_key_count as f64);
                if progress > 1.0 {
                    progress = 0.9999;
                }
                format!(
                    "{:.2}% ({}/{})",
                    progress * 100.0,
                    self.mv_backfill_consumed_rows,
                    self.upstream_mvs_total_key_count
                )
            }
        });
        let source_progress = (source_count > 0).then_some(format!(
            "{} rows consumed",
            self.source_backfill_consumed_rows
        ));
        match (mv_progress, source_progress) {
            (Some(mv_progress), Some(source_progress)) => {
                format!(
                    "MView Backfill: {}, Source Backfill: {}",
                    mv_progress, source_progress
                )
            }
            (Some(mv_progress), None) => mv_progress,
            (None, Some(source_progress)) => source_progress,
            (None, None) => "Unknown".to_string(),
        }
    }
}

/// There are 2 kinds of `TrackingJobs`:
/// 1. `New`. This refers to the "New" type of tracking job.
///    It is instantiated and managed by the stream manager.
///    On recovery, the stream manager will stop managing the job.
/// 2. `Recovered`. This refers to the "Recovered" type of tracking job.
///    On recovery, the barrier manager will recover and start managing the job.
pub enum TrackingJob {
    New(TrackingCommand),
    Recovered(RecoveredTrackingJob),
}

impl TrackingJob {
    /// Notify metadata manager that the job is finished.
    pub(crate) async fn finish(self, metadata_manager: &MetadataManager) -> MetaResult<()> {
        match self {
            TrackingJob::New(command) => {
                let CreateStreamingJobCommandInfo { streaming_job, .. } = &command.info;
                metadata_manager
                    .catalog_controller
                    .finish_streaming_job(
                        streaming_job.id() as i32,
                        command.replace_table_info.clone(),
                    )
                    .await?;
                Ok(())
            }
            TrackingJob::Recovered(recovered) => {
                metadata_manager
                    .catalog_controller
                    .finish_streaming_job(recovered.id, None)
                    .await?;
                Ok(())
            }
        }
    }

    pub(crate) fn table_to_create(&self) -> TableId {
        match self {
            TrackingJob::New(command) => command.info.stream_job_fragments.stream_job_id(),
            TrackingJob::Recovered(recovered) => (recovered.id as u32).into(),
        }
    }
}

impl std::fmt::Debug for TrackingJob {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        match self {
            TrackingJob::New(command) => write!(
                f,
                "TrackingJob::New({:?})",
                command.info.stream_job_fragments.stream_job_id()
            ),
            TrackingJob::Recovered(recovered) => {
                write!(f, "TrackingJob::RecoveredV2({:?})", recovered.id)
            }
        }
    }
}

pub struct RecoveredTrackingJob {
    pub id: ObjectId,
}

/// The command tracking by the [`CreateMviewProgressTracker`].
pub(super) struct TrackingCommand {
    pub info: CreateStreamingJobCommandInfo,
    pub replace_table_info: Option<ReplaceTablePlan>,
}

/// Tracking is done as follows:
/// 1. We identify a `StreamJob` by its `TableId` of its `Materialized` table.
/// 2. For each stream job, there are several actors which run its tasks.
/// 3. With `progress_map` we can use the ID of the `StreamJob` to view its progress.
/// 4. With `actor_map` we can use an actor's `ActorId` to find the ID of the `StreamJob`.
#[derive(Default, Debug)]
pub(super) struct CreateMviewProgressTracker {
    /// Progress of the create-mview DDL indicated by the `TableId`.
    progress_map: HashMap<TableId, (Progress, TrackingJob)>,

    actor_map: HashMap<ActorId, TableId>,

    /// Stash of finished jobs. They will be finally finished on checkpoint.
    pending_finished_jobs: Vec<TrackingJob>,
}

impl CreateMviewProgressTracker {
    /// This step recovers state from the meta side:
    /// 1. `Tables`.
    /// 2. `TableFragments`.
    ///
    /// Other state are persisted by the `BackfillExecutor`, such as:
    /// 1. `CreateMviewProgress`.
    /// 2. `Backfill` position.
    pub fn recover(
        mview_map: HashMap<TableId, (String, StreamJobFragments)>,
        version_stats: &HummockVersionStats,
    ) -> Self {
        let mut actor_map = HashMap::new();
        let mut progress_map = HashMap::new();
        for (creating_table_id, (definition, table_fragments)) in mview_map {
            let mut states = HashMap::new();
            let mut backfill_upstream_types = HashMap::new();
            let actors = table_fragments.tracking_progress_actor_ids();
            for (actor, backfill_upstream_type) in actors {
                actor_map.insert(actor, creating_table_id);
                states.insert(actor, BackfillState::ConsumingUpstream(Epoch(0), 0));
                backfill_upstream_types.insert(actor, backfill_upstream_type);
            }

            let progress = Self::recover_progress(
                states,
                backfill_upstream_types,
                table_fragments.dependent_table_ids(),
                definition,
                version_stats,
            );
            let tracking_job = TrackingJob::Recovered(RecoveredTrackingJob {
                id: creating_table_id.table_id as i32,
            });
            progress_map.insert(creating_table_id, (progress, tracking_job));
        }
        Self {
            progress_map,
            actor_map,
            pending_finished_jobs: Vec::new(),
        }
    }

    /// ## How recovery works
    ///
    /// The progress (number of rows consumed) is persisted in state tables.
    /// During recovery, the backfill executor will restore the number of rows consumed,
    /// and then it will just report progress like newly created executors.
    fn recover_progress(
        states: HashMap<ActorId, BackfillState>,
        backfill_upstream_types: HashMap<ActorId, BackfillUpstreamType>,
        upstream_mv_count: HashMap<TableId, usize>,
        definition: String,
        version_stats: &HummockVersionStats,
    ) -> Progress {
        let upstream_mvs_total_key_count =
            calculate_total_key_count(&upstream_mv_count, version_stats);
        Progress {
            states,
            backfill_upstream_types,
            done_count: 0, // Fill only after first barrier pass
            upstream_mv_count,
            upstream_mvs_total_key_count,
            mv_backfill_consumed_rows: 0, // Fill only after first barrier pass
            source_backfill_consumed_rows: 0, // Fill only after first barrier pass
            definition,
        }
    }

    pub fn gen_ddl_progress(&self) -> HashMap<u32, DdlProgress> {
        self.progress_map
            .iter()
            .map(|(table_id, (x, _))| {
                let table_id = table_id.table_id;
                let ddl_progress = DdlProgress {
                    id: table_id as u64,
                    statement: x.definition.clone(),
                    progress: x.calculate_progress(),
                };
                (table_id, ddl_progress)
            })
            .collect()
    }

    pub(super) fn update_tracking_jobs<'a>(
        &mut self,
        info: Option<(&CreateStreamingJobCommandInfo, Option<&ReplaceTablePlan>)>,
        create_mview_progress: impl IntoIterator<Item = &'a CreateMviewProgress>,
        version_stats: &HummockVersionStats,
    ) {
        {
            {
                // Save `finished_commands` for Create MVs.
                let finished_commands = {
                    let mut commands = vec![];
                    // Add the command to tracker.
                    if let Some((create_job_info, replace_table)) = info
                        && let Some(command) =
                            self.add(create_job_info, replace_table, version_stats)
                    {
                        // Those with no actors to track can be finished immediately.
                        commands.push(command);
                    }
                    // Update the progress of all commands.
                    for progress in create_mview_progress {
                        // Those with actors complete can be finished immediately.
                        if let Some(command) = self.update(progress, version_stats) {
                            tracing::trace!(?progress, "finish progress");
                            commands.push(command);
                        } else {
                            tracing::trace!(?progress, "update progress");
                        }
                    }
                    commands
                };

                for command in finished_commands {
                    self.stash_command_to_finish(command);
                }
            }
        }
    }

    /// Apply a collected epoch node command to the tracker
    /// Return the finished jobs when the barrier kind is `Checkpoint`
    pub(super) fn apply_collected_command(
        &mut self,
        command: Option<&Command>,
        barrier_info: &BarrierInfo,
        resps: impl IntoIterator<Item = &PbBarrierCompleteResponse>,
        version_stats: &HummockVersionStats,
    ) -> Vec<TrackingJob> {
        let new_tracking_job_info =
            if let Some(Command::CreateStreamingJob { info, job_type }) = command {
                match job_type {
                    CreateStreamingJobType::Normal => Some((info, None)),
                    CreateStreamingJobType::SinkIntoTable(replace_table) => {
                        Some((info, Some(replace_table)))
                    }
                    CreateStreamingJobType::SnapshotBackfill(_) => {
                        // The progress of SnapshotBackfill won't be tracked here
                        None
                    }
                }
            } else {
                None
            };
        self.update_tracking_jobs(
            new_tracking_job_info,
            resps
                .into_iter()
                .flat_map(|resp| resp.create_mview_progress.iter()),
            version_stats,
        );
        for table_id in command.map(Command::tables_to_drop).into_iter().flatten() {
            // the cancelled command is possibly stashed in `finished_commands` and waiting
            // for checkpoint, we should also clear it.
            self.cancel_command(table_id);
        }
        if barrier_info.kind.is_checkpoint() {
            self.take_finished_jobs()
        } else {
            vec![]
        }
    }

    /// Stash a command to finish later.
    pub(super) fn stash_command_to_finish(&mut self, finished_job: TrackingJob) {
        self.pending_finished_jobs.push(finished_job);
    }

    /// Finish stashed jobs on checkpoint.
    pub(super) fn take_finished_jobs(&mut self) -> Vec<TrackingJob> {
        tracing::trace!(finished_jobs=?self.pending_finished_jobs, progress_map=?self.progress_map, "take_finished_jobs");
        take(&mut self.pending_finished_jobs)
    }

    pub(super) fn has_pending_finished_jobs(&self) -> bool {
        !self.pending_finished_jobs.is_empty()
    }

    pub(super) fn cancel_command(&mut self, id: TableId) {
        let _ = self.progress_map.remove(&id);
        self.pending_finished_jobs
            .retain(|x| x.table_to_create() != id);
        self.actor_map.retain(|_, table_id| *table_id != id);
    }

    /// Notify all tracked commands that error encountered and clear them.
    pub fn abort_all(&mut self) {
        self.actor_map.clear();
        self.pending_finished_jobs.clear();
        self.progress_map.clear();
    }

    /// Add a new create-mview DDL command to track.
    ///
    /// If the actors to track is empty, return the given command as it can be finished immediately.
    pub fn add(
        &mut self,
        info: &CreateStreamingJobCommandInfo,
        replace_table: Option<&ReplaceTablePlan>,
        version_stats: &HummockVersionStats,
    ) -> Option<TrackingJob> {
        tracing::trace!(?info, "add job to track");
        let (info, actors, replace_table_info) = {
            let CreateStreamingJobCommandInfo {
                stream_job_fragments: table_fragments,
                ..
            } = info;
            let actors = table_fragments.tracking_progress_actor_ids();
            if actors.is_empty() {
                // The command can be finished immediately.
                return Some(TrackingJob::New(TrackingCommand {
                    info: info.clone(),
                    replace_table_info: replace_table.cloned(),
                }));
            }
            (info.clone(), actors, replace_table.cloned())
        };

        let CreateStreamingJobCommandInfo {
            stream_job_fragments: table_fragments,
            upstream_root_actors,
            dispatchers,
            definition,
            ddl_type,
            create_type,
            ..
        } = &info;

        let creating_mv_id = table_fragments.stream_job_id();

        let (upstream_mv_count, upstream_total_key_count, ddl_type, create_type) = {
            // Keep track of how many times each upstream MV appears.
            let mut upstream_mv_count = HashMap::new();
            for (table_id, actors) in upstream_root_actors {
                assert!(!actors.is_empty());
                let dispatch_count: usize = dispatchers
                    .iter()
                    .filter(|(upstream_actor_id, _)| actors.contains(upstream_actor_id))
                    .map(|(_, v)| v.len())
                    .sum();
                upstream_mv_count.insert(*table_id, dispatch_count / actors.len());
            }

            let upstream_total_key_count: u64 =
                calculate_total_key_count(&upstream_mv_count, version_stats);
            (
                upstream_mv_count,
                upstream_total_key_count,
                ddl_type,
                create_type,
            )
        };

        for (actor, _backfill_upstream_type) in &actors {
            self.actor_map.insert(*actor, creating_mv_id);
        }

        let progress = Progress::new(
            actors,
            upstream_mv_count,
            upstream_total_key_count,
            definition.clone(),
        );
        if *ddl_type == DdlType::Sink && *create_type == CreateType::Background {
            // We return the original tracking job immediately.
            // This is because sink can be decoupled with backfill progress.
            // We don't need to wait for sink to finish backfill.
            // This still contains the notifiers, so we can tell listeners
            // that the sink job has been created.
            Some(TrackingJob::New(TrackingCommand {
                info,
                replace_table_info,
            }))
        } else {
            let old = self.progress_map.insert(
                creating_mv_id,
                (
                    progress,
                    TrackingJob::New(TrackingCommand {
                        info,
                        replace_table_info,
                    }),
                ),
            );
            assert!(old.is_none());
            None
        }
    }

    /// Update the progress of `actor` according to the Pb struct.
    ///
    /// If all actors in this MV have finished, returns the command.
    pub fn update(
        &mut self,
        progress: &CreateMviewProgress,
        version_stats: &HummockVersionStats,
    ) -> Option<TrackingJob> {
        tracing::trace!(?progress, "update progress");
        let actor = progress.backfill_actor_id;
        let Some(table_id) = self.actor_map.get(&actor).copied() else {
            // On restart, backfill will ALWAYS notify CreateMviewProgressTracker,
            // even if backfill is finished on recovery.
            // This is because we don't know if only this actor is finished,
            // OR the entire stream job is finished.
            // For the first case, we must notify meta.
            // For the second case, we can still notify meta, but ignore it here.
            tracing::info!(
                "no tracked progress for actor {}, the stream job could already be finished",
                actor
            );
            return None;
        };

        let new_state = if progress.done {
            BackfillState::Done(progress.consumed_rows)
        } else {
            BackfillState::ConsumingUpstream(progress.consumed_epoch.into(), progress.consumed_rows)
        };

        match self.progress_map.entry(table_id) {
            Entry::Occupied(mut o) => {
                let progress = &mut o.get_mut().0;

                let upstream_total_key_count: u64 =
                    calculate_total_key_count(&progress.upstream_mv_count, version_stats);

                tracing::debug!(?table_id, "updating progress for table");
                progress.update(actor, new_state, upstream_total_key_count);

                if progress.is_done() {
                    tracing::debug!(
                        "all actors done for creating mview with table_id {}!",
                        table_id
                    );

                    // Clean-up the mapping from actors to DDL table_id.
                    for actor in o.get().0.actors() {
                        self.actor_map.remove(&actor);
                    }
                    Some(o.remove().1)
                } else {
                    None
                }
            }
            Entry::Vacant(_) => {
                tracing::warn!(
                    "update the progress of an non-existent creating streaming job: {progress:?}, which could be cancelled"
                );
                None
            }
        }
    }
}

fn calculate_total_key_count(
    table_count: &HashMap<TableId, usize>,
    version_stats: &HummockVersionStats,
) -> u64 {
    table_count
        .iter()
        .map(|(table_id, count)| {
            assert_ne!(*count, 0);
            *count as u64
                * version_stats
                    .table_stats
                    .get(&table_id.table_id)
                    .map_or(0, |stat| stat.total_key_count as u64)
        })
        .sum()
}