risingwave_stream/task/barrier_manager/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
// 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::assert_matches::assert_matches;
use std::fmt::{Display, Formatter};
use risingwave_common::util::epoch::EpochPair;
use risingwave_pb::stream_service::barrier_complete_response::PbCreateMviewProgress;
use super::LocalBarrierManager;
use crate::task::barrier_manager::LocalBarrierEvent::ReportCreateProgress;
use crate::task::barrier_manager::LocalBarrierWorker;
use crate::task::ActorId;
type ConsumedEpoch = u64;
type ConsumedRows = u64;
#[derive(Debug, Clone, Copy)]
pub(crate) enum BackfillState {
ConsumingUpstreamTableOrSource(ConsumedEpoch, ConsumedRows),
DoneConsumingUpstreamTableOrSource(ConsumedRows),
ConsumingLogStore { pending_barrier_num: usize },
DoneConsumingLogStore,
}
impl BackfillState {
pub fn to_pb(self, actor_id: ActorId) -> PbCreateMviewProgress {
let (done, consumed_epoch, consumed_rows, pending_barrier_num) = match self {
BackfillState::ConsumingUpstreamTableOrSource(consumed_epoch, consumed_rows) => {
(false, consumed_epoch, consumed_rows, 0)
}
BackfillState::DoneConsumingUpstreamTableOrSource(consumed_rows) => {
(true, 0, consumed_rows, 0)
} // unused field for done
BackfillState::ConsumingLogStore {
pending_barrier_num,
} => (false, 0, 0, pending_barrier_num as _),
BackfillState::DoneConsumingLogStore => (true, 0, 0, 0),
};
PbCreateMviewProgress {
backfill_actor_id: actor_id,
done,
consumed_epoch,
consumed_rows,
pending_barrier_num,
}
}
}
impl Display for BackfillState {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
match self {
BackfillState::ConsumingUpstreamTableOrSource(epoch, rows) => {
write!(
f,
"ConsumingUpstreamTable(epoch: {}, rows: {})",
epoch, rows
)
}
BackfillState::DoneConsumingUpstreamTableOrSource(rows) => {
write!(f, "DoneConsumingUpstreamTable(rows: {})", rows)
}
BackfillState::ConsumingLogStore {
pending_barrier_num,
} => {
write!(
f,
"ConsumingLogStore(pending_barrier_num: {pending_barrier_num})"
)
}
BackfillState::DoneConsumingLogStore => {
write!(f, "DoneConsumingLogStore")
}
}
}
}
impl LocalBarrierWorker {
pub(crate) fn update_create_mview_progress(
&mut self,
epoch: EpochPair,
actor: ActorId,
state: BackfillState,
) {
if let Some(actor_state) = self.state.actor_states.get(&actor)
&& let Some(partial_graph_id) = actor_state.inflight_barriers.get(&epoch.prev)
&& let Some(graph_state) = self.state.graph_states.get_mut(partial_graph_id)
{
graph_state
.create_mview_progress
.entry(epoch.curr)
.or_default()
.insert(actor, state);
} else {
warn!(?epoch, actor, ?state, "ignore create mview progress");
}
}
}
impl LocalBarrierManager {
fn update_create_mview_progress(&self, epoch: EpochPair, actor: ActorId, state: BackfillState) {
self.send_event(ReportCreateProgress {
epoch,
actor,
state,
})
}
}
/// The progress held by the backfill executors to report to the local barrier manager.
///
/// Progress can be computed by
/// `total_rows_consumed` / `total_rows_upstream`.
/// This yields the (approximate) percentage of rows we are done backfilling.
///
/// For `total_rows_consumed`, the progress is tracked in the following way:
/// 1. Fetching the row count from our state table.
/// This number is the total number, NOT incremental.
/// This is done per actor.
/// 2. Refreshing this number on the meta side, on every barrier.
/// This is done by just summing up all the row counts from the actors.
///
/// For `total_rows_upstream`,
/// this is fetched from `HummockVersion`'s statistics (`TableStats::total_key_count`).
///
/// This is computed per `HummockVersion`, which is updated whenever a checkpoint is committed.
/// The `total_key_count` figure just counts the number of storage keys.
/// For example, if a key is inserted and then deleted,
/// it results two storage entries in `LSMt`, so count=2.
/// Only after compaction, the count will drop back to 0.
///
/// So the total count could be more pessimistic, than actual progress.
///
/// It is fine for this number not to be precise,
/// since we don't rely on it to update the status of a stream job internally.
///
/// TODO(kwannoel): Perhaps it is possible to get total key count of the replicated state table
/// for arrangement backfill. We can use that to estimate the progress as well, and avoid recording
/// `row_count` state for it.
pub struct CreateMviewProgressReporter {
barrier_manager: LocalBarrierManager,
/// The id of the actor containing the backfill executors.
backfill_actor_id: ActorId,
state: Option<BackfillState>,
}
impl CreateMviewProgressReporter {
pub fn new(barrier_manager: LocalBarrierManager, backfill_actor_id: ActorId) -> Self {
Self {
barrier_manager,
backfill_actor_id,
state: None,
}
}
#[cfg(test)]
pub fn for_test(barrier_manager: LocalBarrierManager) -> Self {
Self::new(barrier_manager, 0)
}
pub fn actor_id(&self) -> u32 {
self.backfill_actor_id
}
fn update_inner(&mut self, epoch: EpochPair, state: BackfillState) {
self.state = Some(state);
self.barrier_manager
.update_create_mview_progress(epoch, self.backfill_actor_id, state);
}
/// Update the progress to `ConsumingUpstream(consumed_epoch, consumed_rows)`. The epoch must be
/// monotonically increasing.
/// `current_epoch` should be provided to locate the barrier under concurrent checkpoint.
/// `current_consumed_rows` is an accumulated value.
pub fn update(
&mut self,
epoch: EpochPair,
consumed_epoch: ConsumedEpoch,
current_consumed_rows: ConsumedRows,
) {
match self.state {
Some(BackfillState::ConsumingUpstreamTableOrSource(last, last_consumed_rows)) => {
assert!(
last < consumed_epoch,
"last_epoch: {:#?} must be greater than consumed epoch: {:#?}",
last,
consumed_epoch
);
assert!(last_consumed_rows <= current_consumed_rows);
}
Some(state) => {
panic!(
"should not update consuming progress at invalid state: {:?}",
state
)
}
None => {}
};
self.update_inner(
epoch,
BackfillState::ConsumingUpstreamTableOrSource(consumed_epoch, current_consumed_rows),
);
}
/// The difference from [`Self::update`] (MV backfill) is that we
/// don't care `ConsumedEpoch` here.
pub fn update_for_source_backfill(
&mut self,
epoch: EpochPair,
current_consumed_rows: ConsumedRows,
) {
match self.state {
Some(BackfillState::ConsumingUpstreamTableOrSource(
dummy_last_epoch,
_last_consumed_rows,
)) => {
debug_assert_eq!(dummy_last_epoch, 0);
}
Some(state) => {
panic!(
"should not update consuming progress at invalid state: {:?}",
state
)
}
None => {}
};
self.update_inner(
epoch,
// fill a dummy ConsumedEpoch
BackfillState::ConsumingUpstreamTableOrSource(0, current_consumed_rows),
);
}
/// Finish the progress. If the progress is already finished, then perform no-op.
/// `current_epoch` should be provided to locate the barrier under concurrent checkpoint.
pub fn finish(&mut self, epoch: EpochPair, current_consumed_rows: ConsumedRows) {
if let Some(BackfillState::DoneConsumingUpstreamTableOrSource(_)) = self.state {
return;
}
tracing::debug!("progress finish");
self.update_inner(
epoch,
BackfillState::DoneConsumingUpstreamTableOrSource(current_consumed_rows),
);
}
pub(crate) fn update_create_mview_log_store_progress(
&mut self,
epoch: EpochPair,
pending_barrier_num: usize,
) {
assert_matches!(
self.state,
Some(BackfillState::DoneConsumingUpstreamTableOrSource(_))
| Some(BackfillState::ConsumingLogStore { .. }),
"cannot update log store progress at state {:?}",
self.state
);
self.update_inner(
epoch,
BackfillState::ConsumingLogStore {
pending_barrier_num,
},
);
}
pub(crate) fn finish_consuming_log_store(&mut self, epoch: EpochPair) {
assert_matches!(
self.state,
Some(BackfillState::DoneConsumingUpstreamTableOrSource(_))
| Some(BackfillState::ConsumingLogStore { .. }),
"cannot finish log store progress at state {:?}",
self.state
);
self.update_inner(epoch, BackfillState::DoneConsumingLogStore);
}
}
impl LocalBarrierManager {
/// Create a struct for reporting the progress of creating mview. The backfill executors should
/// report the progress of barrier rearranging continuously using this. The updated progress
/// will be collected by the local barrier manager and reported to the meta service in this
/// epoch.
///
/// When all backfill executors of the creating mview finish, the creation progress will be done at
/// frontend and the mview will be exposed to the user.
pub(crate) fn register_create_mview_progress(
&self,
backfill_actor_id: ActorId,
) -> CreateMviewProgressReporter {
trace!("register create mview progress: {}", backfill_actor_id);
CreateMviewProgressReporter::new(self.clone(), backfill_actor_id)
}
}