risingwave_stream/executor/source/source_backfill_executor.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 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762 763 764 765 766 767 768 769 770 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804 805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855 856 857 858 859 860 861 862 863 864 865 866 867 868 869 870 871 872 873 874 875 876 877 878 879 880 881 882 883 884 885 886 887 888 889 890 891 892 893 894 895 896 897 898 899 900 901 902 903 904 905 906 907 908 909 910 911 912 913 914 915 916 917 918 919 920 921 922 923 924 925 926 927 928 929 930 931 932 933 934 935 936 937 938 939 940 941 942 943 944 945 946 947 948 949 950 951 952 953 954 955 956 957 958 959 960 961 962 963 964 965 966 967 968 969 970 971 972 973 974 975 976 977 978 979 980 981 982 983 984 985 986 987 988 989 990 991 992 993 994 995 996 997 998 999 1000 1001 1002 1003 1004 1005 1006 1007 1008 1009 1010 1011 1012 1013 1014 1015 1016 1017 1018 1019 1020 1021 1022 1023 1024 1025 1026 1027 1028 1029 1030 1031 1032 1033 1034 1035 1036 1037 1038 1039 1040 1041 1042 1043 1044 1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059 1060 1061 1062 1063 1064 1065 1066 1067
// 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::cmp::Ordering;
use std::collections::{HashMap, HashSet};
use std::sync::Once;
use std::time::Instant;
use anyhow::anyhow;
use either::Either;
use futures::stream::{select_with_strategy, PollNext};
use itertools::Itertools;
use risingwave_common::bitmap::BitmapBuilder;
use risingwave_common::metrics::{LabelGuardedIntCounter, GLOBAL_ERROR_METRICS};
use risingwave_common::system_param::local_manager::SystemParamsReaderRef;
use risingwave_common::system_param::reader::SystemParamsRead;
use risingwave_common::types::JsonbVal;
use risingwave_connector::source::reader::desc::{SourceDesc, SourceDescBuilder};
use risingwave_connector::source::{
BackfillInfo, BoxChunkSourceStream, SourceContext, SourceCtrlOpts, SplitId, SplitImpl,
SplitMetaData,
};
use risingwave_hummock_sdk::HummockReadEpoch;
use risingwave_storage::store::TryWaitEpochOptions;
use serde::{Deserialize, Serialize};
use thiserror_ext::AsReport;
use super::executor_core::StreamSourceCore;
use super::source_backfill_state_table::BackfillStateTableHandler;
use super::{apply_rate_limit, get_split_offset_col_idx};
use crate::common::rate_limit::limited_chunk_size;
use crate::executor::prelude::*;
use crate::executor::source::source_executor::WAIT_BARRIER_MULTIPLE_TIMES;
use crate::executor::UpdateMutation;
use crate::task::CreateMviewProgressReporter;
#[derive(Clone, Debug, Deserialize, Serialize, PartialEq)]
pub enum BackfillState {
/// `None` means not started yet. It's the initial state.
/// XXX: perhaps we can also set to low-watermark instead of `None`
Backfilling(Option<String>),
/// Backfill is stopped at this offset (inclusive). Source needs to filter out messages before this offset.
SourceCachingUp(String),
Finished,
}
pub type BackfillStates = HashMap<SplitId, BackfillStateWithProgress>;
/// Only `state` field is the real state for fail-over.
/// Other fields are for observability (but we still need to persist them).
#[derive(Clone, Debug, Deserialize, Serialize, PartialEq)]
pub struct BackfillStateWithProgress {
pub state: BackfillState,
pub num_consumed_rows: u64,
/// The latest offset from upstream (inclusive). After we reach this offset, we can stop backfilling.
/// This is initialized with the latest available offset in the connector (if the connector provides the ability to fetch it)
/// so that we can finish backfilling even when upstream doesn't emit any data.
pub target_offset: Option<String>,
}
impl BackfillStateWithProgress {
pub fn encode_to_json(self) -> JsonbVal {
serde_json::to_value(self).unwrap().into()
}
pub fn restore_from_json(value: JsonbVal) -> anyhow::Result<Self> {
serde_json::from_value(value.take()).map_err(|e| anyhow!(e))
}
}
pub struct SourceBackfillExecutor<S: StateStore> {
pub inner: SourceBackfillExecutorInner<S>,
/// Upstream changelog stream which may contain metadata columns, e.g. `_rw_offset`
pub input: Executor,
}
pub struct SourceBackfillExecutorInner<S: StateStore> {
actor_ctx: ActorContextRef,
info: ExecutorInfo,
/// Streaming source for external
// FIXME: some fields e.g. its state table is not used. We might need to refactor. Even latest_split_info is not used.
stream_source_core: StreamSourceCore<S>,
backfill_state_store: BackfillStateTableHandler<S>,
/// Metrics for monitor.
metrics: Arc<StreamingMetrics>,
source_split_change_count: LabelGuardedIntCounter<4>,
// /// Receiver of barrier channel.
// barrier_receiver: Option<UnboundedReceiver<Barrier>>,
/// System parameter reader to read barrier interval
system_params: SystemParamsReaderRef,
/// Rate limit in rows/s.
rate_limit_rps: Option<u32>,
progress: CreateMviewProgressReporter,
}
/// Local variables used in the backfill stage.
///
/// See <https://github.com/risingwavelabs/risingwave/issues/18299> for a state diagram about how it works.
///
/// Note: all off the fields should contain all available splits, and we can `unwrap()` safely when `get()`.
#[derive(Debug)]
struct BackfillStage {
states: BackfillStates,
/// A copy of all splits (incl unfinished and finished ones) assigned to the actor.
///
/// Note: the offsets are not updated. Should use `state`'s offset to update before using it (`get_latest_unfinished_splits`).
splits: Vec<SplitImpl>,
}
impl BackfillStage {
fn total_backfilled_rows(&self) -> u64 {
self.states.values().map(|s| s.num_consumed_rows).sum()
}
fn debug_assert_consistent(&self) {
if cfg!(debug_assertions) {
let all_splits: HashSet<_> =
self.splits.iter().map(|split| split.id().clone()).collect();
assert_eq!(
self.states.keys().cloned().collect::<HashSet<_>>(),
all_splits
);
}
}
/// Get unfinished splits with latest offsets according to the backfill states.
fn get_latest_unfinished_splits(&self) -> StreamExecutorResult<Vec<SplitImpl>> {
let mut unfinished_splits = Vec::new();
for split in &self.splits {
let state = &self.states.get(split.id().as_ref()).unwrap().state;
match state {
BackfillState::Backfilling(Some(offset)) => {
let mut updated_split = split.clone();
updated_split.update_in_place(offset.clone())?;
unfinished_splits.push(updated_split);
}
BackfillState::Backfilling(None) => unfinished_splits.push(split.clone()),
BackfillState::SourceCachingUp(_) | BackfillState::Finished => {}
}
}
Ok(unfinished_splits)
}
/// Updates backfill states and `target_offsets` and returns whether the row from upstream `SourceExecutor` is visible.
fn handle_upstream_row(&mut self, split_id: &str, offset: &str) -> bool {
let mut vis = false;
let state = self.states.get_mut(split_id).unwrap();
let state_inner = &mut state.state;
match state_inner {
BackfillState::Backfilling(None) => {
// backfilling for this split is not started yet. Ignore this row
}
BackfillState::Backfilling(Some(backfill_offset)) => {
match compare_kafka_offset(backfill_offset, offset) {
Ordering::Less => {
// continue backfilling. Ignore this row
}
Ordering::Equal => {
// backfilling for this split is finished just right.
*state_inner = BackfillState::Finished;
}
Ordering::Greater => {
// backfilling for this split produced more data than current source's progress.
// We should stop backfilling, and filter out rows from upstream with offset <= backfill_offset.
*state_inner = BackfillState::SourceCachingUp(backfill_offset.clone());
}
}
}
BackfillState::SourceCachingUp(backfill_offset) => {
match compare_kafka_offset(backfill_offset, offset) {
Ordering::Less => {
// Source caught up, but doesn't contain the last backfilled row.
// This may happen e.g., if Kafka performed compaction.
vis = true;
*state_inner = BackfillState::Finished;
}
Ordering::Equal => {
// Source just caught up with backfilling.
*state_inner = BackfillState::Finished;
}
Ordering::Greater => {
// Source is still behind backfilling.
}
}
}
BackfillState::Finished => {
vis = true;
// This split's backfilling is finished, we are waiting for other splits
}
}
if matches!(state_inner, BackfillState::Backfilling(_)) {
state.target_offset = Some(offset.to_string());
}
if vis {
debug_assert_eq!(*state_inner, BackfillState::Finished);
}
vis
}
/// Updates backfill states and returns whether the row backfilled from external system is visible.
fn handle_backfill_row(&mut self, split_id: &str, offset: &str) -> bool {
let state = self.states.get_mut(split_id).unwrap();
state.num_consumed_rows += 1;
let state_inner = &mut state.state;
match state_inner {
BackfillState::Backfilling(_old_offset) => {
if let Some(target_offset) = &state.target_offset
&& compare_kafka_offset(offset, target_offset).is_ge()
{
// Note1: If target_offset = offset, it seems we can mark the state as Finished without waiting for upstream to catch up
// and dropping duplicated messages.
// But it's not true if target_offset is fetched from other places, like Kafka high watermark.
// In this case, upstream hasn't reached the target_offset yet.
//
// Note2: after this, all following rows in the current chunk will be invisible.
//
// Note3: if target_offset is None (e.g., when upstream doesn't emit messages at all), we will
// keep backfilling.
*state_inner = BackfillState::SourceCachingUp(offset.to_string());
} else {
*state_inner = BackfillState::Backfilling(Some(offset.to_string()));
}
true
}
BackfillState::SourceCachingUp(_) | BackfillState::Finished => {
// backfilling stopped. ignore
false
}
}
}
}
impl<S: StateStore> SourceBackfillExecutorInner<S> {
#[expect(clippy::too_many_arguments)]
pub fn new(
actor_ctx: ActorContextRef,
info: ExecutorInfo,
stream_source_core: StreamSourceCore<S>,
metrics: Arc<StreamingMetrics>,
system_params: SystemParamsReaderRef,
backfill_state_store: BackfillStateTableHandler<S>,
rate_limit_rps: Option<u32>,
progress: CreateMviewProgressReporter,
) -> Self {
let source_split_change_count = metrics
.source_split_change_count
.with_guarded_label_values(&[
&stream_source_core.source_id.to_string(),
&stream_source_core.source_name,
&actor_ctx.id.to_string(),
&actor_ctx.fragment_id.to_string(),
]);
Self {
actor_ctx,
info,
stream_source_core,
backfill_state_store,
metrics,
source_split_change_count,
system_params,
rate_limit_rps,
progress,
}
}
async fn build_stream_source_reader(
&self,
source_desc: &SourceDesc,
splits: Vec<SplitImpl>,
) -> StreamExecutorResult<(BoxChunkSourceStream, HashMap<SplitId, BackfillInfo>)> {
let column_ids = source_desc
.columns
.iter()
.map(|column_desc| column_desc.column_id)
.collect_vec();
let source_ctx = SourceContext::new(
self.actor_ctx.id,
self.stream_source_core.source_id,
self.actor_ctx.fragment_id,
self.stream_source_core.source_name.clone(),
source_desc.metrics.clone(),
SourceCtrlOpts {
chunk_size: limited_chunk_size(self.rate_limit_rps),
rate_limit: self.rate_limit_rps,
},
source_desc.source.config.clone(),
None,
);
// We will check watermark to decide whether we need to backfill.
// e.g., when there's a Kafka topic-partition without any data,
// we don't need to backfill at all. But if we do not check here,
// the executor can only know it's finished when data coming in.
// For blocking DDL, this would be annoying.
let (stream, backfill_info) = source_desc
.source
.build_stream_for_backfill(Some(splits), column_ids, Arc::new(source_ctx))
.await
.map_err(StreamExecutorError::connector_error)?;
Ok((
apply_rate_limit(stream, self.rate_limit_rps).boxed(),
backfill_info,
))
}
#[try_stream(ok = Message, error = StreamExecutorError)]
async fn execute(mut self, input: Executor) {
let mut input = input.execute();
// Poll the upstream to get the first barrier.
let barrier = expect_first_barrier(&mut input).await?;
let first_epoch = barrier.epoch;
let owned_splits = barrier
.initial_split_assignment(self.actor_ctx.id)
.unwrap_or(&[])
.to_vec();
let is_pause_on_startup = barrier.is_pause_on_startup();
yield Message::Barrier(barrier);
let mut core = self.stream_source_core;
let source_desc_builder: SourceDescBuilder = core.source_desc_builder.take().unwrap();
let source_desc = source_desc_builder
.build()
.map_err(StreamExecutorError::connector_error)?;
let (Some(split_idx), Some(offset_idx)) = get_split_offset_col_idx(&source_desc.columns)
else {
unreachable!("Partition and offset columns must be set.");
};
self.backfill_state_store.init_epoch(first_epoch).await?;
let mut backfill_states: BackfillStates = HashMap::new();
for split in &owned_splits {
let split_id = split.id();
let backfill_state = self
.backfill_state_store
.try_recover_from_state_store(&split_id)
.await?
.unwrap_or(BackfillStateWithProgress {
state: BackfillState::Backfilling(None),
num_consumed_rows: 0,
target_offset: None, // init with None
});
backfill_states.insert(split_id, backfill_state);
}
let mut backfill_stage = BackfillStage {
states: backfill_states,
splits: owned_splits,
};
backfill_stage.debug_assert_consistent();
// Return the ownership of `stream_source_core` to the source executor.
self.stream_source_core = core;
let (source_chunk_reader, backfill_info) = self
.build_stream_source_reader(
&source_desc,
backfill_stage.get_latest_unfinished_splits()?,
)
.instrument_await("source_build_reader")
.await?;
for (split_id, info) in &backfill_info {
let state = backfill_stage.states.get_mut(split_id).unwrap();
match info {
BackfillInfo::NoDataToBackfill => {
state.state = BackfillState::Finished;
}
BackfillInfo::HasDataToBackfill { latest_offset } => {
// Note: later we will override it with the offset from the source message, and it's possible to become smaller than this value.
state.target_offset = Some(latest_offset.clone());
}
}
}
tracing::debug!(?backfill_stage, "source backfill started");
fn select_strategy(_: &mut ()) -> PollNext {
futures::stream::PollNext::Left
}
// We choose "preferring upstream" strategy here, because:
// - When the upstream source's workload is high (i.e., Kafka has new incoming data), it just makes backfilling slower.
// For chunks from upstream, they are simply dropped, so there's no much overhead.
// So possibly this can also affect other running jobs less.
// - When the upstream Source's becomes less busy, SourceBackfill can begin to catch up.
let mut backfill_stream = select_with_strategy(
input.by_ref().map(Either::Left),
source_chunk_reader.map(Either::Right),
select_strategy,
);
type PausedReader = Option<impl Stream>;
let mut paused_reader: PausedReader = None;
let mut command_paused = false;
macro_rules! pause_reader {
() => {
let (left, right) = backfill_stream.into_inner();
backfill_stream = select_with_strategy(
left,
futures::stream::pending().boxed().map(Either::Right),
select_strategy,
);
// XXX: do we have to store the original reader? Can we simply rebuild the reader later?
paused_reader = Some(right);
};
}
// If the first barrier requires us to pause on startup, pause the stream.
if is_pause_on_startup {
command_paused = true;
pause_reader!();
}
let state_store = self.backfill_state_store.state_store.state_store().clone();
let table_id = self.backfill_state_store.state_store.table_id().into();
static STATE_TABLE_INITIALIZED: Once = Once::new();
tokio::spawn(async move {
// This is for self.backfill_finished() to be safe.
// We wait for 1st epoch's curr, i.e., the 2nd epoch's prev.
let epoch = first_epoch.curr;
tracing::info!("waiting for epoch: {}", epoch);
state_store
.try_wait_epoch(
HummockReadEpoch::Committed(epoch),
TryWaitEpochOptions { table_id },
)
.await
.expect("failed to wait epoch");
STATE_TABLE_INITIALIZED.call_once(|| ());
tracing::info!("finished waiting for epoch: {}", epoch);
});
{
let source_backfill_row_count = self
.metrics
.source_backfill_row_count
.with_guarded_label_values(&[
&self.stream_source_core.source_id.to_string(),
&self.stream_source_core.source_name,
&self.actor_ctx.id.to_string(),
&self.actor_ctx.fragment_id.to_string(),
]);
// We allow data to flow for `WAIT_BARRIER_MULTIPLE_TIMES` * `expected_barrier_latency_ms`
// milliseconds, considering some other latencies like network and cost in Meta.
let mut max_wait_barrier_time_ms = self.system_params.load().barrier_interval_ms()
as u128
* WAIT_BARRIER_MULTIPLE_TIMES;
let mut last_barrier_time = Instant::now();
let mut self_paused = false;
// The main logic of the loop is in handle_upstream_row and handle_backfill_row.
'backfill_loop: while let Some(either) = backfill_stream.next().await {
match either {
// Upstream
Either::Left(msg) => {
let Ok(msg) = msg else {
let e = msg.unwrap_err();
let core = &self.stream_source_core;
tracing::warn!(
error = ?e.as_report(),
source_id = %core.source_id,
"stream source reader error",
);
GLOBAL_ERROR_METRICS.user_source_error.report([
"SourceReaderError".to_owned(),
core.source_id.to_string(),
core.source_name.to_owned(),
self.actor_ctx.fragment_id.to_string(),
]);
let (reader, _backfill_info) = self
.build_stream_source_reader(
&source_desc,
backfill_stage.get_latest_unfinished_splits()?,
)
.await?;
backfill_stream = select_with_strategy(
input.by_ref().map(Either::Left),
reader.map(Either::Right),
select_strategy,
);
continue;
};
match msg {
Message::Barrier(barrier) => {
last_barrier_time = Instant::now();
if self_paused {
// command_paused has a higher priority.
if !command_paused {
backfill_stream = select_with_strategy(
input.by_ref().map(Either::Left),
paused_reader
.take()
.expect("no paused reader to resume"),
select_strategy,
);
}
self_paused = false;
}
let mut split_changed = false;
if let Some(ref mutation) = barrier.mutation.as_deref() {
match mutation {
Mutation::Pause => {
// pause_reader should not be invoked consecutively more than once.
if !command_paused {
pause_reader!();
command_paused = true;
} else {
tracing::warn!(command_paused, "unexpected pause");
}
}
Mutation::Resume => {
// pause_reader.take should not be invoked consecutively more than once.
if command_paused {
backfill_stream = select_with_strategy(
input.by_ref().map(Either::Left),
paused_reader
.take()
.expect("no paused reader to resume"),
select_strategy,
);
command_paused = false;
} else {
tracing::warn!(command_paused, "unexpected resume");
}
}
Mutation::SourceChangeSplit(actor_splits) => {
tracing::info!(
actor_splits = ?actor_splits,
"source change split received"
);
split_changed = self
.apply_split_change(
actor_splits,
&mut backfill_stage,
true,
)
.await?;
}
Mutation::Update(UpdateMutation {
actor_splits, ..
}) => {
split_changed = self
.apply_split_change(
actor_splits,
&mut backfill_stage,
false,
)
.await?;
}
Mutation::Throttle(actor_to_apply) => {
if let Some(new_rate_limit) =
actor_to_apply.get(&self.actor_ctx.id)
&& *new_rate_limit != self.rate_limit_rps
{
tracing::info!(
"updating rate limit from {:?} to {:?}",
self.rate_limit_rps,
*new_rate_limit
);
self.rate_limit_rps = *new_rate_limit;
// rebuild reader
let (reader, _backfill_info) = self
.build_stream_source_reader(
&source_desc,
backfill_stage
.get_latest_unfinished_splits()?,
)
.await?;
backfill_stream = select_with_strategy(
input.by_ref().map(Either::Left),
reader.map(Either::Right),
select_strategy,
);
}
}
_ => {}
}
}
if split_changed {
// rebuild backfill_stream
// Note: we don't put this part in a method, due to some complex lifetime issues.
let latest_unfinished_splits =
backfill_stage.get_latest_unfinished_splits()?;
tracing::info!(
"actor {:?} apply source split change to {:?}",
self.actor_ctx.id,
latest_unfinished_splits
);
// Replace the source reader with a new one of the new state.
let (reader, _backfill_info) = self
.build_stream_source_reader(
&source_desc,
latest_unfinished_splits,
)
.await?;
backfill_stream = select_with_strategy(
input.by_ref().map(Either::Left),
reader.map(Either::Right),
select_strategy,
);
}
self.backfill_state_store
.set_states(backfill_stage.states.clone())
.await?;
self.backfill_state_store
.state_store
.commit(barrier.epoch)
.await?;
if self.should_report_finished(&backfill_stage.states) {
self.progress.finish(
barrier.epoch,
backfill_stage.total_backfilled_rows(),
);
// yield barrier after reporting progress
yield Message::Barrier(barrier);
// After we reported finished, we still don't exit the loop.
// Because we need to handle split migration.
if STATE_TABLE_INITIALIZED.is_completed()
&& self.backfill_finished(&backfill_stage.states).await?
{
break 'backfill_loop;
}
} else {
self.progress.update_for_source_backfill(
barrier.epoch,
backfill_stage.total_backfilled_rows(),
);
// yield barrier after reporting progress
yield Message::Barrier(barrier);
}
}
Message::Chunk(chunk) => {
// We need to iterate over all rows because there might be multiple splits in a chunk.
// Note: We assume offset from the source is monotonically increasing for the algorithm to work correctly.
let mut new_vis = BitmapBuilder::zeroed(chunk.visibility().len());
for (i, (_, row)) in chunk.rows().enumerate() {
let split = row.datum_at(split_idx).unwrap().into_utf8();
let offset = row.datum_at(offset_idx).unwrap().into_utf8();
let vis = backfill_stage.handle_upstream_row(split, offset);
new_vis.set(i, vis);
}
// emit chunk if vis is not empty. i.e., some splits finished backfilling.
let new_vis = new_vis.finish();
if new_vis.count_ones() != 0 {
let new_chunk = chunk.clone_with_vis(new_vis);
yield Message::Chunk(new_chunk);
}
}
Message::Watermark(_) => {
// Ignore watermark during backfill.
}
}
}
// backfill
Either::Right(msg) => {
let chunk = msg?;
if last_barrier_time.elapsed().as_millis() > max_wait_barrier_time_ms {
assert!(!command_paused, "command_paused should be false");
// pause_reader should not be invoked consecutively more than once.
if !self_paused {
pause_reader!();
} else {
tracing::warn!(self_paused, "unexpected self pause");
}
// Exceeds the max wait barrier time, the source will be paused.
// Currently we can guarantee the
// source is not paused since it received stream
// chunks.
self_paused = true;
tracing::warn!(
"source {} paused, wait barrier for {:?}",
self.info.identity,
last_barrier_time.elapsed()
);
// Only update `max_wait_barrier_time_ms` to capture
// `barrier_interval_ms`
// changes here to avoid frequently accessing the shared
// `system_params`.
max_wait_barrier_time_ms =
self.system_params.load().barrier_interval_ms() as u128
* WAIT_BARRIER_MULTIPLE_TIMES;
}
let mut new_vis = BitmapBuilder::zeroed(chunk.visibility().len());
for (i, (_, row)) in chunk.rows().enumerate() {
let split_id = row.datum_at(split_idx).unwrap().into_utf8();
let offset = row.datum_at(offset_idx).unwrap().into_utf8();
let vis = backfill_stage.handle_backfill_row(split_id, offset);
new_vis.set(i, vis);
}
let new_vis = new_vis.finish();
let card = new_vis.count_ones();
if card != 0 {
let new_chunk = chunk.clone_with_vis(new_vis);
yield Message::Chunk(new_chunk);
source_backfill_row_count.inc_by(card as u64);
}
}
}
}
}
let mut states = backfill_stage.states;
// Make sure `Finished` state is persisted.
self.backfill_state_store.set_states(states.clone()).await?;
// All splits finished backfilling. Now we only forward the source data.
#[for_await]
for msg in input {
let msg = msg?;
match msg {
Message::Barrier(barrier) => {
if let Some(ref mutation) = barrier.mutation.as_deref() {
match mutation {
Mutation::Pause | Mutation::Resume => {
// We don't need to do anything. Handled by upstream.
}
Mutation::SourceChangeSplit(actor_splits) => {
tracing::info!(
actor_splits = ?actor_splits,
"source change split received"
);
self.apply_split_change_forward_stage(
actor_splits,
&mut states,
true,
)
.await?;
}
Mutation::Update(UpdateMutation { actor_splits, .. }) => {
self.apply_split_change_forward_stage(
actor_splits,
&mut states,
false,
)
.await?;
}
_ => {}
}
}
self.backfill_state_store
.state_store
.commit(barrier.epoch)
.await?;
yield Message::Barrier(barrier);
}
Message::Chunk(chunk) => {
yield Message::Chunk(chunk);
}
Message::Watermark(watermark) => {
yield Message::Watermark(watermark);
}
}
}
}
/// When we should call `progress.finish()` to let blocking DDL return.
/// We report as soon as `SourceCachingUp`. Otherwise the DDL might be blocked forever until upstream messages come.
///
/// Note: split migration (online scaling) is related with progress tracking.
/// - For foreground DDL, scaling is not allowed before progress is finished.
/// - For background DDL, scaling is skipped when progress is not finished, and can be triggered by recreating actors during recovery.
///
/// See <https://github.com/risingwavelabs/risingwave/issues/18300> for more details.
fn should_report_finished(&self, states: &BackfillStates) -> bool {
states.values().all(|state| {
matches!(
state.state,
BackfillState::Finished | BackfillState::SourceCachingUp(_)
)
})
}
/// All splits entered `Finished` state.
///
/// We check all splits for the source, including other actors' splits here, before going to the forward stage.
/// Otherwise if we `break` early, but after rescheduling, an unfinished split is migrated to
/// this actor, we still need to backfill it.
///
/// Note: at the beginning, the actor will only read the state written by itself.
/// It needs to _wait until it can read all actors' written data_.
/// i.e., wait for the first checkpoint has been available.
///
/// See <https://github.com/risingwavelabs/risingwave/issues/18300> for more details.
async fn backfill_finished(&self, states: &BackfillStates) -> StreamExecutorResult<bool> {
Ok(states
.values()
.all(|state| matches!(state.state, BackfillState::Finished))
&& self
.backfill_state_store
.scan()
.await?
.into_iter()
.all(|state| matches!(state.state, BackfillState::Finished)))
}
/// For newly added splits, we do not need to backfill and can directly forward from upstream.
async fn apply_split_change(
&mut self,
split_assignment: &HashMap<ActorId, Vec<SplitImpl>>,
stage: &mut BackfillStage,
should_trim_state: bool,
) -> StreamExecutorResult<bool> {
self.source_split_change_count.inc();
if let Some(target_splits) = split_assignment.get(&self.actor_ctx.id).cloned() {
if self
.update_state_if_changed(target_splits, stage, should_trim_state)
.await?
{
// Note: we don't rebuild backfill_stream here, due to some complex lifetime issues.
return Ok(true);
}
}
Ok(false)
}
/// Returns `true` if split changed. Otherwise `false`.
async fn update_state_if_changed(
&mut self,
target_splits: Vec<SplitImpl>,
stage: &mut BackfillStage,
should_trim_state: bool,
) -> StreamExecutorResult<bool> {
let mut target_state: BackfillStates = HashMap::with_capacity(target_splits.len());
let mut split_changed = false;
// Take out old states (immutable, only used to build target_state and check for added/dropped splits).
// Will be set to target_state in the end.
let old_states = std::mem::take(&mut stage.states);
// Iterate over the target (assigned) splits
// - check if any new splits are added
// - build target_state
for split in &target_splits {
let split_id = split.id();
if let Some(s) = old_states.get(&split_id) {
target_state.insert(split_id, s.clone());
} else {
split_changed = true;
let backfill_state = self
.backfill_state_store
.try_recover_from_state_store(&split_id)
.await?;
match backfill_state {
None => {
// Newly added split. We don't need to backfill.
// Note that this branch is different from the initial barrier (BackfillStateInner::Backfilling(None) there).
target_state.insert(
split_id,
BackfillStateWithProgress {
state: BackfillState::Finished,
num_consumed_rows: 0,
target_offset: None,
},
);
}
Some(backfill_state) => {
// Migrated split. Backfill if unfinished.
target_state.insert(split_id, backfill_state);
}
}
}
}
// Checks dropped splits
for existing_split_id in old_states.keys() {
if !target_state.contains_key(existing_split_id) {
tracing::info!("split dropping detected: {}", existing_split_id);
split_changed = true;
}
}
if split_changed {
let dropped_splits = stage
.states
.extract_if(|split_id, _| !target_state.contains_key(split_id))
.map(|(split_id, _)| split_id);
if should_trim_state {
// trim dropped splits' state
self.backfill_state_store.trim_state(dropped_splits).await?;
}
tracing::info!(old_state=?old_states, new_state=?target_state, "finish split change");
} else {
debug_assert_eq!(old_states, target_state);
}
stage.states = target_state;
stage.splits = target_splits;
stage.debug_assert_consistent();
Ok(split_changed)
}
/// For split change during forward stage, all newly added splits should be already finished.
// We just need to update the state store if necessary.
async fn apply_split_change_forward_stage(
&mut self,
split_assignment: &HashMap<ActorId, Vec<SplitImpl>>,
states: &mut BackfillStates,
should_trim_state: bool,
) -> StreamExecutorResult<()> {
self.source_split_change_count.inc();
if let Some(target_splits) = split_assignment.get(&self.actor_ctx.id).cloned() {
self.update_state_if_changed_forward_stage(target_splits, states, should_trim_state)
.await?;
}
Ok(())
}
async fn update_state_if_changed_forward_stage(
&mut self,
target_splits: Vec<SplitImpl>,
states: &mut BackfillStates,
should_trim_state: bool,
) -> StreamExecutorResult<()> {
let target_splits: HashSet<SplitId> = target_splits
.into_iter()
.map(|split| (split.id()))
.collect();
let mut split_changed = false;
let mut newly_added_splits = vec![];
// Checks added splits
for split_id in &target_splits {
if !states.contains_key(split_id) {
split_changed = true;
let backfill_state = self
.backfill_state_store
.try_recover_from_state_store(split_id)
.await?;
match &backfill_state {
None => {
// Newly added split. We don't need to backfill!
newly_added_splits.push(split_id.clone());
}
Some(backfill_state) => {
// Migrated split. It should also be finished since we are in forwarding stage.
match backfill_state.state {
BackfillState::Finished => {}
_ => {
return Err(anyhow::anyhow!(
"Unexpected backfill state: {:?}",
backfill_state
)
.into());
}
}
}
}
states.insert(
split_id.clone(),
backfill_state.unwrap_or(BackfillStateWithProgress {
state: BackfillState::Finished,
num_consumed_rows: 0,
target_offset: None,
}),
);
}
}
// Checks dropped splits
for existing_split_id in states.keys() {
if !target_splits.contains(existing_split_id) {
tracing::info!("split dropping detected: {}", existing_split_id);
split_changed = true;
}
}
if split_changed {
tracing::info!(
target_splits = ?target_splits,
"apply split change"
);
let dropped_splits = states.extract_if(|split_id, _| !target_splits.contains(split_id));
if should_trim_state {
// trim dropped splits' state
self.backfill_state_store
.trim_state(dropped_splits.map(|(k, _v)| k))
.await?;
}
// For migrated splits, and existing splits, we do not need to update
// state store, but only for newly added splits.
self.backfill_state_store
.set_states(
newly_added_splits
.into_iter()
.map(|split_id| {
(
split_id,
BackfillStateWithProgress {
state: BackfillState::Finished,
num_consumed_rows: 0,
target_offset: None,
},
)
})
.collect(),
)
.await?;
}
Ok(())
}
}
fn compare_kafka_offset(a: &str, b: &str) -> Ordering {
let a = a.parse::<i64>().unwrap();
let b = b.parse::<i64>().unwrap();
a.cmp(&b)
}
impl<S: StateStore> Execute for SourceBackfillExecutor<S> {
fn execute(self: Box<Self>) -> BoxedMessageStream {
self.inner.execute(self.input).boxed()
}
}
impl<S: StateStore> Debug for SourceBackfillExecutorInner<S> {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
let core = &self.stream_source_core;
f.debug_struct("SourceBackfillExecutor")
.field("source_id", &core.source_id)
.field("column_ids", &core.column_ids)
.field("pk_indices", &self.info.pk_indices)
.finish()
}
}