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

feat: remote input subscribe on barrier mutation #17612

Merged
merged 4 commits into from
Jul 10, 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
22 changes: 18 additions & 4 deletions src/stream/src/executor/exchange/input.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
use std::pin::Pin;
use std::task::{Context, Poll};

use anyhow::Context as _;
use anyhow::{anyhow, Context as _};
use futures::pin_mut;
use futures_async_stream::try_stream;
use pin_project::pin_project;
Expand Down Expand Up @@ -147,6 +147,7 @@ impl RemoteInput {
metrics: Arc<StreamingMetrics>,
batched_permits_limit: usize,
) {
let self_actor_id = up_down_ids.1;
let client = client_pool.get_by_addr(upstream_addr).await?;
let (stream, permits_tx) = client
.get_stream(up_down_ids.0, up_down_ids.1, up_down_frag.0, up_down_frag.1)
Expand All @@ -162,6 +163,7 @@ impl RemoteInput {
let span: await_tree::Span = format!("RemoteInput (actor {up_actor_id})").into();

let mut batched_permits_accumulated = 0;
let mut mutation_subscriber = None;

pin_mut!(stream);
while let Some(data_res) = stream.next().verbose_instrument_await(span.clone()).await {
Expand Down Expand Up @@ -203,10 +205,22 @@ impl RemoteInput {
barrier.mutation.is_none(),
"Mutation should be erased in remote side"
);
let mutation = local_barrier_manager
.read_barrier_mutation(barrier)
let mutation_subscriber =
mutation_subscriber.get_or_insert_with(|| {
local_barrier_manager
.subscribe_barrier_mutation(self_actor_id, barrier)
});

let mutation = mutation_subscriber
.recv()
.await
.context("Read barrier mutation error")?;
.ok_or_else(|| {
anyhow!("failed to receive mutation of barrier {:?}", barrier)
})
.map(|(prev_epoch, mutation)| {
assert_eq!(prev_epoch, barrier.epoch.prev);
mutation
})?;
barrier.mutation = mutation;
}
}
Expand Down
46 changes: 22 additions & 24 deletions src/stream/src/task/barrier_manager.rs
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ use rw_futures_util::{pending_on_none, AttachedFuture};
use thiserror_ext::AsReport;
use tokio::select;
use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender};
use tokio::sync::oneshot;
use tokio::sync::{mpsc, oneshot};
use tokio::task::JoinHandle;
use tonic::{Code, Status};

Expand All @@ -48,6 +48,7 @@ mod tests;

pub use progress::CreateMviewProgress;
use risingwave_common::catalog::TableId;
use risingwave_common::util::epoch::EpochPair;
use risingwave_common::util::runtime::BackgroundShutdownRuntime;
use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map;
use risingwave_hummock_sdk::{LocalSstableInfo, SyncResult};
Expand Down Expand Up @@ -187,6 +188,8 @@ impl CreateActorContext {
}
}

pub(super) type SubscribeMutationItem = (u64, Option<Arc<Mutation>>);

pub(super) enum LocalBarrierEvent {
ReportActorCollected {
actor_id: ActorId,
Expand All @@ -197,9 +200,10 @@ pub(super) enum LocalBarrierEvent {
actor: ActorId,
state: BackfillState,
},
ReadBarrierMutation {
barrier: Barrier,
mutation_sender: oneshot::Sender<Option<Arc<Mutation>>>,
SubscribeBarrierMutation {
actor_id: ActorId,
epoch: EpochPair,
mutation_sender: mpsc::UnboundedSender<SubscribeMutationItem>,
},
#[cfg(test)]
Flush(oneshot::Sender<()>),
Expand Down Expand Up @@ -515,11 +519,13 @@ impl LocalBarrierWorker {
} => {
self.update_create_mview_progress(current_epoch, actor, state);
}
LocalBarrierEvent::ReadBarrierMutation {
barrier,
LocalBarrierEvent::SubscribeBarrierMutation {
actor_id,
epoch,
mutation_sender,
} => {
self.read_barrier_mutation(barrier, mutation_sender);
self.state
.subscribe_actor_mutation(actor_id, epoch.prev, mutation_sender);
}
#[cfg(test)]
LocalBarrierEvent::Flush(sender) => sender.send(()).unwrap(),
Expand Down Expand Up @@ -642,15 +648,6 @@ impl LocalBarrierWorker {
Ok(())
}

/// Read mutation from barrier state.
fn read_barrier_mutation(
&mut self,
barrier: Barrier,
sender: oneshot::Sender<Option<Arc<Mutation>>>,
) {
self.state.read_barrier_mutation(&barrier, sender);
}

/// Register sender for source actors, used to send barriers.
fn register_sender(&mut self, actor_id: ActorId, senders: Vec<UnboundedSender<Barrier>>) {
tracing::debug!(
Expand Down Expand Up @@ -907,17 +904,18 @@ impl LocalBarrierManager {
}

/// When a `RemoteInput` get a barrier, it should wait and read the barrier mutation from the barrier manager.
pub async fn read_barrier_mutation(
pub fn subscribe_barrier_mutation(
&self,
barrier: &Barrier,
) -> StreamResult<Option<Arc<Mutation>>> {
let (tx, rx) = oneshot::channel();
self.send_event(LocalBarrierEvent::ReadBarrierMutation {
barrier: barrier.clone(),
actor_id: ActorId,
first_barrier: &Barrier,
) -> mpsc::UnboundedReceiver<SubscribeMutationItem> {
let (tx, rx) = mpsc::unbounded_channel();
self.send_event(LocalBarrierEvent::SubscribeBarrierMutation {
actor_id,
epoch: first_barrier.epoch,
mutation_sender: tx,
});
rx.await
.map_err(|_| anyhow!("barrier manager maybe reset").into())
rx
}
}

Expand Down
Loading
Loading