risingwave_meta/barrier/command.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
// 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::fmt::Formatter;
use risingwave_common::bitmap::Bitmap;
use risingwave_common::catalog::TableId;
use risingwave_common::hash::ActorMapping;
use risingwave_common::must_match;
use risingwave_common::types::Timestamptz;
use risingwave_common::util::epoch::Epoch;
use risingwave_connector::source::SplitImpl;
use risingwave_hummock_sdk::change_log::build_table_change_log_delta;
use risingwave_meta_model::WorkerId;
use risingwave_pb::catalog::{CreateType, Table};
use risingwave_pb::common::PbWorkerNode;
use risingwave_pb::meta::table_fragments::PbActorStatus;
use risingwave_pb::meta::PausedReason;
use risingwave_pb::source::{ConnectorSplit, ConnectorSplits};
use risingwave_pb::stream_plan::barrier::BarrierKind as PbBarrierKind;
use risingwave_pb::stream_plan::barrier_mutation::Mutation;
use risingwave_pb::stream_plan::throttle_mutation::RateLimit;
use risingwave_pb::stream_plan::update_mutation::*;
use risingwave_pb::stream_plan::{
AddMutation, BarrierMutation, CombinedMutation, Dispatcher, Dispatchers,
DropSubscriptionsMutation, PauseMutation, ResumeMutation, SourceChangeSplitMutation,
StopMutation, StreamActor, SubscriptionUpstreamInfo, ThrottleMutation, UpdateMutation,
};
use risingwave_pb::stream_service::BarrierCompleteResponse;
use tracing::warn;
use super::info::{CommandFragmentChanges, InflightStreamingJobInfo};
use crate::barrier::info::BarrierInfo;
use crate::barrier::utils::collect_resp_info;
use crate::barrier::InflightSubscriptionInfo;
use crate::controller::fragment::InflightFragmentInfo;
use crate::hummock::{CommitEpochInfo, NewTableFragmentInfo};
use crate::manager::{DdlType, StreamingJob};
use crate::model::{ActorId, DispatcherId, FragmentId, StreamJobFragments, TableParallelism};
use crate::stream::{build_actor_connector_splits, SplitAssignment, ThrottleConfig};
/// [`Reschedule`] is for the [`Command::RescheduleFragment`], which is used for rescheduling actors
/// in some fragment, like scaling or migrating.
#[derive(Debug, Clone)]
pub struct Reschedule {
/// Added actors in this fragment.
pub added_actors: HashMap<WorkerId, Vec<ActorId>>,
/// Removed actors in this fragment.
pub removed_actors: Vec<ActorId>,
/// Vnode bitmap updates for some actors in this fragment.
pub vnode_bitmap_updates: HashMap<ActorId, Bitmap>,
/// The upstream fragments of this fragment, and the dispatchers that should be updated.
pub upstream_fragment_dispatcher_ids: Vec<(FragmentId, DispatcherId)>,
/// New hash mapping of the upstream dispatcher to be updated.
///
/// This field exists only when there's upstream fragment and the current fragment is
/// hash-sharded.
pub upstream_dispatcher_mapping: Option<ActorMapping>,
/// The downstream fragments of this fragment.
pub downstream_fragment_ids: Vec<FragmentId>,
/// Reassigned splits for source actors.
/// It becomes the `actor_splits` in [`UpdateMutation`].
/// `Source` and `SourceBackfill` are handled together here.
pub actor_splits: HashMap<ActorId, Vec<SplitImpl>>,
pub newly_created_actors: Vec<(StreamActor, PbActorStatus)>,
}
/// Replacing an old table with a new one. All actors in the table job will be rebuilt.
/// Used for `ALTER TABLE` ([`Command::ReplaceTable`]) and sink into table ([`Command::CreateStreamingJob`]).
#[derive(Debug, Clone)]
pub struct ReplaceTablePlan {
pub old_fragments: StreamJobFragments,
pub new_fragments: StreamJobFragments,
pub merge_updates: Vec<MergeUpdate>,
pub dispatchers: HashMap<ActorId, Vec<Dispatcher>>,
/// For a table with connector, the `SourceExecutor` actor will also be rebuilt with new actor ids.
/// We need to reassign splits for it.
///
/// Note that there's no `SourceBackfillExecutor` involved for table with connector, so we don't need to worry about
/// `backfill_splits`.
pub init_split_assignment: SplitAssignment,
/// The `StreamingJob` info of the table to be replaced. Must be `StreamingJob::Table`
pub streaming_job: StreamingJob,
/// The temporary dummy table fragments id of new table fragment
pub tmp_id: u32,
}
impl ReplaceTablePlan {
fn fragment_changes(&self) -> HashMap<FragmentId, CommandFragmentChanges> {
let mut fragment_changes = HashMap::new();
for fragment in self.new_fragments.fragments.values() {
let fragment_change = CommandFragmentChanges::NewFragment(
self.streaming_job.id().into(),
InflightFragmentInfo {
actors: fragment
.actors
.iter()
.map(|actor| {
(
actor.actor_id,
self.new_fragments
.actor_status
.get(&actor.actor_id)
.expect("should exist")
.worker_id() as WorkerId,
)
})
.collect(),
state_table_ids: fragment
.state_table_ids
.iter()
.map(|table_id| TableId::new(*table_id))
.collect(),
},
);
assert!(fragment_changes
.insert(fragment.fragment_id, fragment_change)
.is_none());
}
for fragment in self.old_fragments.fragments.values() {
assert!(fragment_changes
.insert(fragment.fragment_id, CommandFragmentChanges::RemoveFragment)
.is_none());
}
fragment_changes
}
}
#[derive(educe::Educe, Clone)]
#[educe(Debug)]
pub struct CreateStreamingJobCommandInfo {
#[educe(Debug(ignore))]
pub stream_job_fragments: StreamJobFragments,
/// Refer to the doc on [`crate::manager::MetadataManager::get_upstream_root_fragments`] for the meaning of "root".
pub upstream_root_actors: HashMap<TableId, Vec<ActorId>>,
pub dispatchers: HashMap<ActorId, Vec<Dispatcher>>,
pub init_split_assignment: SplitAssignment,
pub definition: String,
pub ddl_type: DdlType,
pub create_type: CreateType,
pub streaming_job: StreamingJob,
pub internal_tables: Vec<Table>,
}
impl CreateStreamingJobCommandInfo {
pub(super) fn new_fragment_info(
&self,
) -> impl Iterator<Item = (FragmentId, InflightFragmentInfo)> + '_ {
self.stream_job_fragments
.fragments
.values()
.map(|fragment| {
(
fragment.fragment_id,
InflightFragmentInfo {
actors: fragment
.actors
.iter()
.map(|actor| {
(
actor.actor_id,
self.stream_job_fragments
.actor_status
.get(&actor.actor_id)
.expect("should exist")
.worker_id()
as WorkerId,
)
})
.collect(),
state_table_ids: fragment
.state_table_ids
.iter()
.map(|table_id| TableId::new(*table_id))
.collect(),
},
)
})
}
}
#[derive(Debug, Clone)]
pub struct SnapshotBackfillInfo {
pub upstream_mv_table_ids: HashSet<TableId>,
}
#[derive(Debug, Clone)]
pub enum CreateStreamingJobType {
Normal,
SinkIntoTable(ReplaceTablePlan),
SnapshotBackfill(SnapshotBackfillInfo),
}
/// [`Command`] is the input of [`crate::barrier::worker::GlobalBarrierWorker`]. For different commands,
/// it will build different barriers to send, and may do different stuffs after the barrier is
/// collected.
#[derive(Debug, strum::Display)]
pub enum Command {
/// `Flush` command will generate a checkpoint barrier. After the barrier is collected and committed
/// all messages before the checkpoint barrier should have been committed.
Flush,
/// `Pause` command generates a `Pause` barrier with the provided [`PausedReason`] **only if**
/// the cluster is not already paused. Otherwise, a barrier with no mutation will be generated.
Pause(PausedReason),
/// `Resume` command generates a `Resume` barrier with the provided [`PausedReason`] **only
/// if** the cluster is paused with the same reason. Otherwise, a barrier with no mutation
/// will be generated.
Resume(PausedReason),
/// `DropStreamingJobs` command generates a `Stop` barrier to stop the given
/// [`Vec<ActorId>`]. The catalog has ensured that these streaming jobs are safe to be
/// dropped by reference counts before.
///
/// Barriers from the actors to be dropped will STILL be collected.
/// After the barrier is collected, it notifies the local stream manager of compute nodes to
/// drop actors, and then delete the table fragments info from meta store.
DropStreamingJobs {
table_fragments_ids: HashSet<TableId>,
actors: Vec<ActorId>,
unregistered_state_table_ids: HashSet<TableId>,
unregistered_fragment_ids: HashSet<FragmentId>,
},
/// `CreateStreamingJob` command generates a `Add` barrier by given info.
///
/// Barriers from the actors to be created, which is marked as `Inactive` at first, will STILL
/// be collected since the barrier should be passthrough.
///
/// After the barrier is collected, these newly created actors will be marked as `Running`. And
/// it adds the table fragments info to meta store. However, the creating progress will **last
/// for a while** until the `finish` channel is signaled, then the state of `TableFragments`
/// will be set to `Created`.
CreateStreamingJob {
info: CreateStreamingJobCommandInfo,
job_type: CreateStreamingJobType,
},
MergeSnapshotBackfillStreamingJobs(
HashMap<TableId, (SnapshotBackfillInfo, InflightStreamingJobInfo)>,
),
/// `Reschedule` command generates a `Update` barrier by the [`Reschedule`] of each fragment.
/// Mainly used for scaling and migration.
///
/// Barriers from which actors should be collected, and the post behavior of this command are
/// very similar to `Create` and `Drop` commands, for added and removed actors, respectively.
RescheduleFragment {
reschedules: HashMap<FragmentId, Reschedule>,
table_parallelism: HashMap<TableId, TableParallelism>,
// should contain the actor ids in upstream and downstream fragment of `reschedules`
fragment_actors: HashMap<FragmentId, HashSet<ActorId>>,
},
/// `ReplaceTable` command generates a `Update` barrier with the given `merge_updates`. This is
/// essentially switching the downstream of the old table fragments to the new ones, and
/// dropping the old table fragments. Used for table schema change.
///
/// This can be treated as a special case of `RescheduleFragment`, while the upstream fragment
/// of the Merge executors are changed additionally.
ReplaceTable(ReplaceTablePlan),
/// `SourceSplitAssignment` generates a `Splits` barrier for pushing initialized splits or
/// changed splits.
SourceSplitAssignment(SplitAssignment),
/// `Throttle` command generates a `Throttle` barrier with the given throttle config to change
/// the `rate_limit` of `FlowControl` Executor after `StreamScan` or Source.
Throttle(ThrottleConfig),
/// `CreateSubscription` command generates a `CreateSubscriptionMutation` to notify
/// materialize executor to start storing old value for subscription.
CreateSubscription {
subscription_id: u32,
upstream_mv_table_id: TableId,
retention_second: u64,
},
/// `DropSubscription` command generates a `DropSubscriptionsMutation` to notify
/// materialize executor to stop storing old value when there is no
/// subscription depending on it.
DropSubscription {
subscription_id: u32,
upstream_mv_table_id: TableId,
},
}
impl Command {
pub fn pause(reason: PausedReason) -> Self {
Self::Pause(reason)
}
pub fn resume(reason: PausedReason) -> Self {
Self::Resume(reason)
}
pub fn cancel(table_fragments: &StreamJobFragments) -> Self {
Self::DropStreamingJobs {
table_fragments_ids: HashSet::from_iter([table_fragments.stream_job_id()]),
actors: table_fragments.actor_ids(),
unregistered_state_table_ids: table_fragments
.all_table_ids()
.map(TableId::new)
.collect(),
unregistered_fragment_ids: table_fragments.fragment_ids().collect(),
}
}
pub(crate) fn fragment_changes(&self) -> Option<HashMap<FragmentId, CommandFragmentChanges>> {
match self {
Command::Flush => None,
Command::Pause(_) => None,
Command::Resume(_) => None,
Command::DropStreamingJobs {
unregistered_fragment_ids,
..
} => Some(
unregistered_fragment_ids
.iter()
.map(|fragment_id| (*fragment_id, CommandFragmentChanges::RemoveFragment))
.collect(),
),
Command::CreateStreamingJob { info, job_type } => {
assert!(
!matches!(job_type, CreateStreamingJobType::SnapshotBackfill(_)),
"should handle fragment changes separately for snapshot backfill"
);
let mut changes: HashMap<_, _> = info
.new_fragment_info()
.map(|(fragment_id, fragment_info)| {
(
fragment_id,
CommandFragmentChanges::NewFragment(
info.streaming_job.id().into(),
fragment_info,
),
)
})
.collect();
if let CreateStreamingJobType::SinkIntoTable(plan) = job_type {
let extra_change = plan.fragment_changes();
changes.extend(extra_change);
}
Some(changes)
}
Command::RescheduleFragment { reschedules, .. } => Some(
reschedules
.iter()
.map(|(fragment_id, reschedule)| {
(
*fragment_id,
CommandFragmentChanges::Reschedule {
new_actors: reschedule
.added_actors
.iter()
.flat_map(|(node_id, actors)| {
actors.iter().map(|actor_id| (*actor_id, *node_id))
})
.collect(),
to_remove: reschedule.removed_actors.iter().cloned().collect(),
},
)
})
.collect(),
),
Command::ReplaceTable(plan) => Some(plan.fragment_changes()),
Command::MergeSnapshotBackfillStreamingJobs(_) => None,
Command::SourceSplitAssignment(_) => None,
Command::Throttle(_) => None,
Command::CreateSubscription { .. } => None,
Command::DropSubscription { .. } => None,
}
}
/// If we need to send a barrier to modify actor configuration, we will pause the barrier
/// injection. return true.
pub fn should_pause_inject_barrier(&self) -> bool {
// Note: the meaning for `Pause` is not pausing the periodic barrier injection, but for
// pausing the sources on compute nodes. However, when `Pause` is used for configuration
// change like scaling and migration, it must pause the concurrent checkpoint to ensure the
// previous checkpoint has been done.
matches!(self, Self::Pause(PausedReason::ConfigChange))
}
pub fn need_checkpoint(&self) -> bool {
// todo! Reviewing the flow of different command to reduce the amount of checkpoint
!matches!(self, Command::Resume(_))
}
}
#[derive(Debug, Clone)]
pub enum BarrierKind {
Initial,
Barrier,
/// Hold a list of previous non-checkpoint prev-epoch + current prev-epoch
Checkpoint(Vec<u64>),
}
impl BarrierKind {
pub fn to_protobuf(&self) -> PbBarrierKind {
match self {
BarrierKind::Initial => PbBarrierKind::Initial,
BarrierKind::Barrier => PbBarrierKind::Barrier,
BarrierKind::Checkpoint(_) => PbBarrierKind::Checkpoint,
}
}
pub fn is_checkpoint(&self) -> bool {
matches!(self, BarrierKind::Checkpoint(_))
}
pub fn as_str_name(&self) -> &'static str {
match self {
BarrierKind::Initial => "Initial",
BarrierKind::Barrier => "Barrier",
BarrierKind::Checkpoint(_) => "Checkpoint",
}
}
}
/// [`CommandContext`] is used for generating barrier and doing post stuffs according to the given
/// [`Command`].
pub(super) struct CommandContext {
/// Resolved info in this barrier loop.
pub(super) node_map: HashMap<WorkerId, PbWorkerNode>,
subscription_info: InflightSubscriptionInfo,
pub(super) barrier_info: BarrierInfo,
pub(super) table_ids_to_commit: HashSet<TableId>,
pub(super) command: Option<Command>,
/// The tracing span of this command.
///
/// Differs from [`crate::barrier::TracedEpoch`], this span focuses on the lifetime of the corresponding
/// barrier, including the process of waiting for the barrier to be sent, flowing through the
/// stream graph on compute nodes, and finishing its `post_collect` stuffs.
_span: tracing::Span,
}
impl std::fmt::Debug for CommandContext {
fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
f.debug_struct("CommandContext")
.field("barrier_info", &self.barrier_info)
.field("command", &self.command)
.finish()
}
}
impl CommandContext {
pub(super) fn new(
node_map: HashMap<WorkerId, PbWorkerNode>,
barrier_info: BarrierInfo,
subscription_info: InflightSubscriptionInfo,
table_ids_to_commit: HashSet<TableId>,
command: Option<Command>,
span: tracing::Span,
) -> Self {
Self {
node_map,
subscription_info,
barrier_info,
table_ids_to_commit,
command,
_span: span,
}
}
fn get_truncate_epoch(&self, retention_second: u64) -> Epoch {
let Some(truncate_timestamptz) = Timestamptz::from_secs(
self.barrier_info
.prev_epoch
.value()
.as_timestamptz()
.timestamp()
- retention_second as i64,
) else {
warn!(retention_second, prev_epoch = ?self.barrier_info.prev_epoch.value(), "invalid retention second value");
return self.barrier_info.prev_epoch.value();
};
Epoch::from_unix_millis(truncate_timestamptz.timestamp_millis() as u64)
}
pub(super) fn collect_commit_epoch_info(
&self,
info: &mut CommitEpochInfo,
resps: Vec<BarrierCompleteResponse>,
backfill_pinned_log_epoch: HashMap<TableId, (u64, HashSet<TableId>)>,
) {
let (sst_to_context, synced_ssts, new_table_watermarks, old_value_ssts) =
collect_resp_info(resps);
let new_table_fragment_infos = if let Some(Command::CreateStreamingJob { info, job_type }) =
&self.command
&& !matches!(job_type, CreateStreamingJobType::SnapshotBackfill(_))
{
let table_fragments = &info.stream_job_fragments;
let mut table_ids: HashSet<_> = table_fragments
.internal_table_ids()
.into_iter()
.map(TableId::new)
.collect();
if let Some(mv_table_id) = table_fragments.mv_table_id() {
table_ids.insert(TableId::new(mv_table_id));
}
vec![NewTableFragmentInfo { table_ids }]
} else {
vec![]
};
let mut mv_log_store_truncate_epoch = HashMap::new();
let mut update_truncate_epoch =
|table_id: TableId, truncate_epoch| match mv_log_store_truncate_epoch
.entry(table_id.table_id)
{
Entry::Occupied(mut entry) => {
let prev_truncate_epoch = entry.get_mut();
if truncate_epoch < *prev_truncate_epoch {
*prev_truncate_epoch = truncate_epoch;
}
}
Entry::Vacant(entry) => {
entry.insert(truncate_epoch);
}
};
for (mv_table_id, subscriptions) in &self.subscription_info.mv_depended_subscriptions {
if let Some(truncate_epoch) = subscriptions
.values()
.max()
.map(|max_retention| self.get_truncate_epoch(*max_retention).0)
{
update_truncate_epoch(*mv_table_id, truncate_epoch);
}
}
for (_, (backfill_epoch, upstream_mv_table_ids)) in backfill_pinned_log_epoch {
for mv_table_id in upstream_mv_table_ids {
update_truncate_epoch(mv_table_id, backfill_epoch);
}
}
let table_new_change_log = build_table_change_log_delta(
old_value_ssts.into_iter(),
synced_ssts.iter().map(|sst| &sst.sst_info),
must_match!(&self.barrier_info.kind, BarrierKind::Checkpoint(epochs) => epochs),
mv_log_store_truncate_epoch.into_iter(),
);
let epoch = self.barrier_info.prev_epoch();
for table_id in &self.table_ids_to_commit {
info.tables_to_commit
.try_insert(*table_id, epoch)
.expect("non duplicate");
}
info.sstables.extend(synced_ssts);
info.new_table_watermarks.extend(new_table_watermarks);
info.sst_to_context.extend(sst_to_context);
info.new_table_fragment_infos
.extend(new_table_fragment_infos);
info.change_log_delta.extend(table_new_change_log);
}
}
impl Command {
/// Generate a mutation for the given command.
pub fn to_mutation(&self, current_paused_reason: Option<PausedReason>) -> Option<Mutation> {
let mutation =
match self {
Command::Flush => None,
Command::Pause(_) => {
// Only pause when the cluster is not already paused.
if current_paused_reason.is_none() {
Some(Mutation::Pause(PauseMutation {}))
} else {
None
}
}
Command::Resume(reason) => {
// Only resume when the cluster is paused with the same reason.
if current_paused_reason == Some(*reason) {
Some(Mutation::Resume(ResumeMutation {}))
} else {
None
}
}
Command::SourceSplitAssignment(change) => {
let mut diff = HashMap::new();
for actor_splits in change.values() {
diff.extend(actor_splits.clone());
}
Some(Mutation::Splits(SourceChangeSplitMutation {
actor_splits: build_actor_connector_splits(&diff),
}))
}
Command::Throttle(config) => {
let mut actor_to_apply = HashMap::new();
for per_fragment in config.values() {
actor_to_apply.extend(per_fragment.iter().map(|(actor_id, limit)| {
(*actor_id, RateLimit { rate_limit: *limit })
}));
}
Some(Mutation::Throttle(ThrottleMutation {
actor_throttle: actor_to_apply,
}))
}
Command::DropStreamingJobs { actors, .. } => Some(Mutation::Stop(StopMutation {
actors: actors.clone(),
})),
Command::CreateStreamingJob {
info:
CreateStreamingJobCommandInfo {
stream_job_fragments: table_fragments,
dispatchers,
init_split_assignment: split_assignment,
..
},
job_type,
} => {
let actor_dispatchers = dispatchers
.iter()
.map(|(&actor_id, dispatchers)| {
(
actor_id,
Dispatchers {
dispatchers: dispatchers.clone(),
},
)
})
.collect();
let added_actors = table_fragments.actor_ids();
let actor_splits = split_assignment
.values()
.flat_map(build_actor_connector_splits)
.collect();
let subscriptions_to_add =
if let CreateStreamingJobType::SnapshotBackfill(snapshot_backfill_info) =
job_type
{
snapshot_backfill_info
.upstream_mv_table_ids
.iter()
.map(|table_id| SubscriptionUpstreamInfo {
subscriber_id: table_fragments.stream_job_id().table_id,
upstream_mv_table_id: table_id.table_id,
})
.collect()
} else {
Default::default()
};
let add = Some(Mutation::Add(AddMutation {
actor_dispatchers,
added_actors,
actor_splits,
// If the cluster is already paused, the new actors should be paused too.
pause: current_paused_reason.is_some(),
subscriptions_to_add,
}));
if let CreateStreamingJobType::SinkIntoTable(ReplaceTablePlan {
old_fragments,
new_fragments: _,
merge_updates,
dispatchers,
init_split_assignment,
..
}) = job_type
{
// TODO: support in v2.
let update = Self::generate_update_mutation_for_replace_table(
old_fragments,
merge_updates,
dispatchers,
init_split_assignment,
);
Some(Mutation::Combined(CombinedMutation {
mutations: vec![
BarrierMutation { mutation: add },
BarrierMutation { mutation: update },
],
}))
} else {
add
}
}
Command::MergeSnapshotBackfillStreamingJobs(jobs_to_merge) => {
Some(Mutation::DropSubscriptions(DropSubscriptionsMutation {
info: jobs_to_merge
.iter()
.flat_map(|(table_id, (backfill_info, _))| {
backfill_info.upstream_mv_table_ids.iter().map(
move |upstream_table_id| SubscriptionUpstreamInfo {
subscriber_id: table_id.table_id,
upstream_mv_table_id: upstream_table_id.table_id,
},
)
})
.collect(),
}))
}
Command::ReplaceTable(ReplaceTablePlan {
old_fragments,
merge_updates,
dispatchers,
init_split_assignment,
..
}) => Self::generate_update_mutation_for_replace_table(
old_fragments,
merge_updates,
dispatchers,
init_split_assignment,
),
Command::RescheduleFragment {
reschedules,
fragment_actors,
..
} => {
let mut dispatcher_update = HashMap::new();
for reschedule in reschedules.values() {
for &(upstream_fragment_id, dispatcher_id) in
&reschedule.upstream_fragment_dispatcher_ids
{
// Find the actors of the upstream fragment.
let upstream_actor_ids = fragment_actors
.get(&upstream_fragment_id)
.expect("should contain");
// Record updates for all actors.
for &actor_id in upstream_actor_ids {
// Index with the dispatcher id to check duplicates.
dispatcher_update
.try_insert(
(actor_id, dispatcher_id),
DispatcherUpdate {
actor_id,
dispatcher_id,
hash_mapping: reschedule
.upstream_dispatcher_mapping
.as_ref()
.map(|m| m.to_protobuf()),
added_downstream_actor_id: reschedule
.added_actors
.values()
.flatten()
.cloned()
.collect(),
removed_downstream_actor_id: reschedule
.removed_actors
.clone(),
},
)
.unwrap();
}
}
}
let dispatcher_update = dispatcher_update.into_values().collect();
let mut merge_update = HashMap::new();
for (&fragment_id, reschedule) in reschedules {
for &downstream_fragment_id in &reschedule.downstream_fragment_ids {
// Find the actors of the downstream fragment.
let downstream_actor_ids = fragment_actors
.get(&downstream_fragment_id)
.expect("should contain");
// Downstream removed actors should be skipped
// Newly created actors of the current fragment will not dispatch Update
// barriers to them
let downstream_removed_actors: HashSet<_> = reschedules
.get(&downstream_fragment_id)
.map(|downstream_reschedule| {
downstream_reschedule
.removed_actors
.iter()
.copied()
.collect()
})
.unwrap_or_default();
// Record updates for all actors.
for &actor_id in downstream_actor_ids {
if downstream_removed_actors.contains(&actor_id) {
continue;
}
// Index with the fragment id to check duplicates.
merge_update
.try_insert(
(actor_id, fragment_id),
MergeUpdate {
actor_id,
upstream_fragment_id: fragment_id,
new_upstream_fragment_id: None,
added_upstream_actor_id: reschedule
.added_actors
.values()
.flatten()
.cloned()
.collect(),
removed_upstream_actor_id: reschedule
.removed_actors
.clone(),
},
)
.unwrap();
}
}
}
let merge_update = merge_update.into_values().collect();
let mut actor_vnode_bitmap_update = HashMap::new();
for reschedule in reschedules.values() {
// Record updates for all actors in this fragment.
for (&actor_id, bitmap) in &reschedule.vnode_bitmap_updates {
let bitmap = bitmap.to_protobuf();
actor_vnode_bitmap_update
.try_insert(actor_id, bitmap)
.unwrap();
}
}
let dropped_actors = reschedules
.values()
.flat_map(|r| r.removed_actors.iter().copied())
.collect();
let mut actor_splits = HashMap::new();
for reschedule in reschedules.values() {
for (actor_id, splits) in &reschedule.actor_splits {
actor_splits.insert(
*actor_id as ActorId,
ConnectorSplits {
splits: splits.iter().map(ConnectorSplit::from).collect(),
},
);
}
}
// we don't create dispatchers in reschedule scenario
let actor_new_dispatchers = HashMap::new();
let mutation = Mutation::Update(UpdateMutation {
dispatcher_update,
merge_update,
actor_vnode_bitmap_update,
dropped_actors,
actor_splits,
actor_new_dispatchers,
});
tracing::debug!("update mutation: {mutation:?}");
Some(mutation)
}
Command::CreateSubscription {
upstream_mv_table_id,
subscription_id,
..
} => Some(Mutation::Add(AddMutation {
actor_dispatchers: Default::default(),
added_actors: vec![],
actor_splits: Default::default(),
pause: false,
subscriptions_to_add: vec![SubscriptionUpstreamInfo {
upstream_mv_table_id: upstream_mv_table_id.table_id,
subscriber_id: *subscription_id,
}],
})),
Command::DropSubscription {
upstream_mv_table_id,
subscription_id,
} => Some(Mutation::DropSubscriptions(DropSubscriptionsMutation {
info: vec![SubscriptionUpstreamInfo {
subscriber_id: *subscription_id,
upstream_mv_table_id: upstream_mv_table_id.table_id,
}],
})),
};
mutation
}
pub fn actors_to_create(&self) -> Option<HashMap<WorkerId, Vec<StreamActor>>> {
match self {
Command::CreateStreamingJob { info, job_type } => {
let mut map = match job_type {
CreateStreamingJobType::Normal => HashMap::new(),
CreateStreamingJobType::SinkIntoTable(replace_table) => {
replace_table.new_fragments.actors_to_create()
}
CreateStreamingJobType::SnapshotBackfill(_) => {
// for snapshot backfill, the actors to create is measured separately
return None;
}
};
for (worker_id, new_actors) in info.stream_job_fragments.actors_to_create() {
map.entry(worker_id).or_default().extend(new_actors)
}
Some(map)
}
Command::RescheduleFragment { reschedules, .. } => {
let mut map: HashMap<WorkerId, Vec<_>> = HashMap::new();
for (actor, status) in reschedules
.values()
.flat_map(|reschedule| reschedule.newly_created_actors.iter())
{
let worker_id = status.location.as_ref().unwrap().worker_node_id as _;
map.entry(worker_id).or_default().push(actor.clone());
}
Some(map)
}
Command::ReplaceTable(replace_table) => {
Some(replace_table.new_fragments.actors_to_create())
}
_ => None,
}
}
fn generate_update_mutation_for_replace_table(
old_fragments: &StreamJobFragments,
merge_updates: &[MergeUpdate],
dispatchers: &HashMap<ActorId, Vec<Dispatcher>>,
init_split_assignment: &SplitAssignment,
) -> Option<Mutation> {
let dropped_actors = old_fragments.actor_ids();
let actor_new_dispatchers = dispatchers
.iter()
.map(|(&actor_id, dispatchers)| {
(
actor_id,
Dispatchers {
dispatchers: dispatchers.clone(),
},
)
})
.collect();
let actor_splits = init_split_assignment
.values()
.flat_map(build_actor_connector_splits)
.collect();
Some(Mutation::Update(UpdateMutation {
actor_new_dispatchers,
merge_update: merge_updates.to_owned(),
dropped_actors,
actor_splits,
..Default::default()
}))
}
/// Returns the paused reason after executing the current command.
pub fn next_paused_reason(
this: Option<&Self>,
current_paused_reason: Option<PausedReason>,
) -> Option<PausedReason> {
match this {
Some(Command::Pause(reason)) => {
// Only pause when the cluster is not already paused.
if current_paused_reason.is_none() {
Some(*reason)
} else {
current_paused_reason
}
}
Some(Command::Resume(reason)) => {
// Only resume when the cluster is paused with the same reason.
if current_paused_reason == Some(*reason) {
None
} else {
current_paused_reason
}
}
_ => current_paused_reason,
}
}
/// For `CancelStreamingJob`, returns the table id of the target table.
pub fn tables_to_drop(&self) -> impl Iterator<Item = TableId> + '_ {
match self {
Command::DropStreamingJobs {
table_fragments_ids,
..
} => Some(table_fragments_ids.iter().cloned()),
_ => None,
}
.into_iter()
.flatten()
}
}