risingwave_meta/barrier/checkpoint/creating_job/
status.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
// 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, HashSet};
use std::mem::take;

use risingwave_common::hash::ActorId;
use risingwave_common::util::epoch::Epoch;
use risingwave_meta_model::WorkerId;
use risingwave_pb::hummock::HummockVersionStats;
use risingwave_pb::stream_plan::barrier_mutation::Mutation;
use risingwave_pb::stream_plan::StreamActor;
use risingwave_pb::stream_service::barrier_complete_response::{
    CreateMviewProgress, PbCreateMviewProgress,
};
use tracing::warn;

use crate::barrier::progress::CreateMviewProgressTracker;
use crate::barrier::{BarrierInfo, BarrierKind, TracedEpoch};

#[derive(Debug)]
pub(super) struct CreateMviewLogStoreProgressTracker {
    /// `actor_id` -> `pending_barrier_count`
    ongoing_actors: HashMap<ActorId, usize>,
    finished_actors: HashSet<ActorId>,
}

impl CreateMviewLogStoreProgressTracker {
    fn new(actors: impl Iterator<Item = ActorId>, initial_pending_count: usize) -> Self {
        Self {
            ongoing_actors: HashMap::from_iter(actors.map(|actor| (actor, initial_pending_count))),
            finished_actors: HashSet::new(),
        }
    }

    pub(super) fn gen_ddl_progress(&self) -> String {
        let sum = self.ongoing_actors.values().sum::<usize>() as f64;
        let count = if self.ongoing_actors.is_empty() {
            1
        } else {
            self.ongoing_actors.len()
        } as f64;
        let avg = sum / count;
        format!(
            "finished: {}/{}, avg epoch count {}",
            self.finished_actors.len(),
            self.ongoing_actors.len() + self.finished_actors.len(),
            avg
        )
    }

    fn update(&mut self, progress: impl IntoIterator<Item = &PbCreateMviewProgress>) {
        for progress in progress {
            match self.ongoing_actors.entry(progress.backfill_actor_id) {
                Entry::Occupied(mut entry) => {
                    if progress.done {
                        entry.remove_entry();
                        assert!(
                            self.finished_actors.insert(progress.backfill_actor_id),
                            "non-duplicate"
                        );
                    } else {
                        *entry.get_mut() = progress.pending_barrier_num as _;
                    }
                }
                Entry::Vacant(_) => {
                    if cfg!(debug_assertions) {
                        panic!(
                            "reporting progress on non-inflight actor: {:?} {:?}",
                            progress, self
                        );
                    } else {
                        warn!(?progress, progress_tracker = ?self, "reporting progress on non-inflight actor");
                    }
                }
            }
        }
    }

    pub(super) fn is_finished(&self) -> bool {
        self.ongoing_actors.is_empty()
    }
}

#[derive(Debug)]
pub(super) enum CreatingStreamingJobStatus {
    /// The creating job is consuming upstream snapshot.
    /// Will transit to `ConsumingLogStore` on `update_progress` when
    /// the snapshot has been fully consumed after `update_progress`.
    ConsumingSnapshot {
        prev_epoch_fake_physical_time: u64,
        pending_upstream_barriers: Vec<BarrierInfo>,
        version_stats: HummockVersionStats,
        create_mview_tracker: CreateMviewProgressTracker,
        snapshot_backfill_actors: HashSet<ActorId>,
        backfill_epoch: u64,
        /// The `prev_epoch` of pending non checkpoint barriers
        pending_non_checkpoint_barriers: Vec<u64>,
        /// Info of the first barrier: (`actors_to_create`, `mutation`)
        /// Take the mutation out when injecting the first barrier
        initial_barrier_info: Option<(HashMap<WorkerId, Vec<StreamActor>>, Mutation)>,
    },
    /// The creating job is consuming log store.
    ///
    /// Will transit to `Finishing` on `on_new_upstream_epoch` when `start_consume_upstream` is `true`.
    ConsumingLogStore {
        log_store_progress_tracker: CreateMviewLogStoreProgressTracker,
    },
    /// All backfill actors have started consuming upstream, and the job
    /// will be finished when all previously injected barriers have been collected
    /// Store the `prev_epoch` that will finish at.
    Finishing(u64),
}

pub(super) struct CreatingJobInjectBarrierInfo {
    pub barrier_info: BarrierInfo,
    pub new_actors: Option<HashMap<WorkerId, Vec<StreamActor>>>,
    pub mutation: Option<Mutation>,
}

impl CreatingStreamingJobStatus {
    pub(super) fn update_progress(
        &mut self,
        create_mview_progress: impl IntoIterator<Item = &CreateMviewProgress>,
    ) -> Option<Vec<CreatingJobInjectBarrierInfo>> {
        match self {
            Self::ConsumingSnapshot {
                create_mview_tracker,
                ref version_stats,
                prev_epoch_fake_physical_time,
                pending_upstream_barriers,
                pending_non_checkpoint_barriers,
                ref backfill_epoch,
                initial_barrier_info,
                ref snapshot_backfill_actors,
                ..
            } => {
                create_mview_tracker.update_tracking_jobs(
                    None,
                    create_mview_progress,
                    version_stats,
                );
                if create_mview_tracker.has_pending_finished_jobs() {
                    let (new_actors, mutation) = match initial_barrier_info.take() {
                        Some((new_actors, mutation)) => (Some(new_actors), Some(mutation)),
                        None => (None, None),
                    };
                    assert!(initial_barrier_info.is_none());
                    pending_non_checkpoint_barriers.push(*backfill_epoch);

                    let prev_epoch = Epoch::from_physical_time(*prev_epoch_fake_physical_time);
                    let barriers_to_inject: Vec<_> = [CreatingJobInjectBarrierInfo {
                        barrier_info: BarrierInfo {
                            curr_epoch: TracedEpoch::new(Epoch(*backfill_epoch)),
                            prev_epoch: TracedEpoch::new(prev_epoch),
                            kind: BarrierKind::Checkpoint(take(pending_non_checkpoint_barriers)),
                        },
                        new_actors,
                        mutation,
                    }]
                    .into_iter()
                    .chain(pending_upstream_barriers.drain(..).map(|barrier_info| {
                        CreatingJobInjectBarrierInfo {
                            barrier_info,
                            new_actors: None,
                            mutation: None,
                        }
                    }))
                    .collect();

                    *self = CreatingStreamingJobStatus::ConsumingLogStore {
                        log_store_progress_tracker: CreateMviewLogStoreProgressTracker::new(
                            snapshot_backfill_actors.iter().cloned(),
                            barriers_to_inject.len(),
                        ),
                    };
                    Some(barriers_to_inject)
                } else {
                    None
                }
            }
            CreatingStreamingJobStatus::ConsumingLogStore {
                log_store_progress_tracker,
                ..
            } => {
                log_store_progress_tracker.update(create_mview_progress);
                None
            }
            CreatingStreamingJobStatus::Finishing(_) => None,
        }
    }

    pub(super) fn on_new_upstream_epoch(
        &mut self,
        barrier_info: &BarrierInfo,
        start_consume_upstream: bool,
    ) -> Option<CreatingJobInjectBarrierInfo> {
        match self {
            CreatingStreamingJobStatus::ConsumingSnapshot {
                pending_upstream_barriers,
                prev_epoch_fake_physical_time,
                pending_non_checkpoint_barriers,
                initial_barrier_info,
                ..
            } => {
                assert!(
                    !start_consume_upstream,
                    "should not start consuming upstream for a job that are consuming snapshot"
                );
                pending_upstream_barriers.push(barrier_info.clone());
                Some(CreatingStreamingJobStatus::new_fake_barrier(
                    prev_epoch_fake_physical_time,
                    pending_non_checkpoint_barriers,
                    initial_barrier_info,
                    barrier_info.kind.is_checkpoint(),
                ))
            }
            CreatingStreamingJobStatus::ConsumingLogStore { .. } => {
                let prev_epoch = barrier_info.prev_epoch();
                if start_consume_upstream {
                    assert!(barrier_info.kind.is_checkpoint());
                    *self = CreatingStreamingJobStatus::Finishing(prev_epoch);
                }
                Some(CreatingJobInjectBarrierInfo {
                    barrier_info: barrier_info.clone(),
                    new_actors: None,
                    mutation: None,
                })
            }
            CreatingStreamingJobStatus::Finishing { .. } => {
                assert!(
                    !start_consume_upstream,
                    "should not start consuming upstream for a job again"
                );
                None
            }
        }
    }

    pub(super) fn new_fake_barrier(
        prev_epoch_fake_physical_time: &mut u64,
        pending_non_checkpoint_barriers: &mut Vec<u64>,
        initial_barrier_info: &mut Option<(HashMap<WorkerId, Vec<StreamActor>>, Mutation)>,
        is_checkpoint: bool,
    ) -> CreatingJobInjectBarrierInfo {
        {
            {
                let prev_epoch =
                    TracedEpoch::new(Epoch::from_physical_time(*prev_epoch_fake_physical_time));
                *prev_epoch_fake_physical_time += 1;
                let curr_epoch =
                    TracedEpoch::new(Epoch::from_physical_time(*prev_epoch_fake_physical_time));
                pending_non_checkpoint_barriers.push(prev_epoch.value().0);
                let kind = if is_checkpoint {
                    BarrierKind::Checkpoint(take(pending_non_checkpoint_barriers))
                } else {
                    BarrierKind::Barrier
                };
                let (new_actors, mutation) =
                    if let Some((new_actors, mutation)) = initial_barrier_info.take() {
                        (Some(new_actors), Some(mutation))
                    } else {
                        Default::default()
                    };
                CreatingJobInjectBarrierInfo {
                    barrier_info: BarrierInfo {
                        prev_epoch,
                        curr_epoch,
                        kind,
                    },
                    new_actors,
                    mutation,
                }
            }
        }
    }

    pub(super) fn is_finishing(&self) -> bool {
        matches!(self, Self::Finishing(_))
    }
}