Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

refactor(meta): simplify InflightActorInfo and extract InflightGraphInfo #17956

Merged
merged 6 commits into from
Aug 9, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
60 changes: 30 additions & 30 deletions src/meta/src/barrier/command.rs
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@

use std::collections::{HashMap, HashSet};
use std::fmt::Formatter;
use std::sync::Arc;

use futures::future::try_join_all;
use itertools::Itertools;
Expand All @@ -26,6 +25,7 @@ use risingwave_common::util::epoch::Epoch;
use risingwave_connector::source::SplitImpl;
use risingwave_hummock_sdk::HummockEpoch;
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};
Expand All @@ -42,9 +42,9 @@ use risingwave_pb::stream_service::WaitEpochCommitRequest;
use thiserror_ext::AsReport;
use tracing::warn;

use super::info::{CommandActorChanges, CommandFragmentChanges, InflightActorInfo};
use super::info::CommandFragmentChanges;
use super::trace::TracedEpoch;
use crate::barrier::GlobalBarrierManagerContext;
use crate::barrier::{GlobalBarrierManagerContext, InflightSubscriptionInfo};
use crate::manager::{DdlType, InflightFragmentInfo, MetadataManager, StreamingJob, WorkerId};
use crate::model::{ActorId, DispatcherId, FragmentId, TableFragments, TableParallelism};
use crate::stream::{build_actor_connector_splits, SplitAssignment, ThrottleConfig};
Expand Down Expand Up @@ -106,7 +106,7 @@ pub struct ReplaceTablePlan {
}

impl ReplaceTablePlan {
fn actor_changes(&self) -> CommandActorChanges {
fn fragment_changes(&self) -> HashMap<FragmentId, CommandFragmentChanges> {
let mut fragment_changes = HashMap::new();
for fragment in self.new_table_fragments.fragments.values() {
let fragment_change = CommandFragmentChanges::NewFragment(InflightFragmentInfo {
Expand Down Expand Up @@ -140,7 +140,7 @@ impl ReplaceTablePlan {
.insert(fragment.fragment_id, CommandFragmentChanges::RemoveFragment)
.is_none());
}
CommandActorChanges { fragment_changes }
fragment_changes
}
}

Expand Down Expand Up @@ -301,48 +301,47 @@ impl Command {
Self::Resume(reason)
}

pub fn actor_changes(&self) -> Option<CommandActorChanges> {
pub(crate) fn fragment_changes(&self) -> Option<HashMap<FragmentId, CommandFragmentChanges>> {
match self {
Command::Plain(_) => None,
Command::Pause(_) => None,
Command::Resume(_) => None,
Command::DropStreamingJobs {
unregistered_fragment_ids,
..
} => Some(CommandActorChanges {
fragment_changes: unregistered_fragment_ids
} => Some(
unregistered_fragment_ids
.iter()
.map(|fragment_id| (*fragment_id, CommandFragmentChanges::RemoveFragment))
.collect(),
}),
),
Command::CreateStreamingJob {
info,
replace_table,
} => {
let fragment_changes = info
let mut changes: HashMap<_, _> = info
.new_fragment_info()
.map(|(fragment_id, info)| {
(fragment_id, CommandFragmentChanges::NewFragment(info))
})
.collect();
let mut changes = CommandActorChanges { fragment_changes };

if let Some(plan) = replace_table {
let extra_change = plan.actor_changes();
let extra_change = plan.fragment_changes();
changes.extend(extra_change);
}

Some(changes)
}
Command::CancelStreamingJob(table_fragments) => Some(CommandActorChanges {
fragment_changes: table_fragments
Command::CancelStreamingJob(table_fragments) => Some(
table_fragments
.fragments
.values()
.map(|fragment| (fragment.fragment_id, CommandFragmentChanges::RemoveFragment))
.collect(),
}),
Command::RescheduleFragment { reschedules, .. } => Some(CommandActorChanges {
fragment_changes: reschedules
),
Command::RescheduleFragment { reschedules, .. } => Some(
reschedules
.iter()
.map(|(fragment_id, reschedule)| {
(
Expand All @@ -360,8 +359,8 @@ impl Command {
)
})
.collect(),
}),
Command::ReplaceTable(plan) => Some(plan.actor_changes()),
),
Command::ReplaceTable(plan) => Some(plan.fragment_changes()),
Command::SourceSplitAssignment(_) => None,
Command::Throttle(_) => None,
Command::CreateSubscription { .. } => None,
Expand Down Expand Up @@ -406,10 +405,6 @@ impl BarrierKind {
matches!(self, BarrierKind::Checkpoint(_))
}

pub fn is_initial(&self) -> bool {
matches!(self, BarrierKind::Initial)
}

pub fn as_str_name(&self) -> &'static str {
match self {
BarrierKind::Initial => "Initial",
Expand All @@ -423,7 +418,9 @@ impl BarrierKind {
/// [`Command`].
pub struct CommandContext {
/// Resolved info in this barrier loop.
pub info: Arc<InflightActorInfo>,
pub node_map: HashMap<WorkerId, PbWorkerNode>,
pub subscription_info: InflightSubscriptionInfo,
pub table_ids_to_commit: HashSet<TableId>,

pub prev_epoch: TracedEpoch,
pub curr_epoch: TracedEpoch,
Expand Down Expand Up @@ -458,7 +455,9 @@ impl std::fmt::Debug for CommandContext {
impl CommandContext {
#[allow(clippy::too_many_arguments)]
pub(super) fn new(
info: InflightActorInfo,
node_map: HashMap<WorkerId, PbWorkerNode>,
subscription_info: InflightSubscriptionInfo,
table_ids_to_commit: HashSet<TableId>,
prev_epoch: TracedEpoch,
curr_epoch: TracedEpoch,
current_paused_reason: Option<PausedReason>,
Expand All @@ -468,7 +467,9 @@ impl CommandContext {
span: tracing::Span,
) -> Self {
Self {
info: Arc::new(info),
node_map,
subscription_info,
table_ids_to_commit,
prev_epoch,
curr_epoch,
current_paused_reason,
Expand Down Expand Up @@ -861,17 +862,16 @@ impl CommandContext {
self.barrier_manager_context
.stream_rpc_manager
.drop_actors(
&self.info.node_map,
self.info
.node_map
&self.node_map,
self.node_map
.keys()
.map(|worker_id| (*worker_id, actors.clone())),
)
.await
}

pub async fn wait_epoch_commit(&self, epoch: HummockEpoch) -> MetaResult<()> {
let futures = self.info.node_map.values().map(|worker_node| async {
let futures = self.node_map.values().map(|worker_node| async {
let client = self
.barrier_manager_context
.env
Expand Down
Loading
Loading