Skip to content

Commit

Permalink
fix(streaming): drop actor on another blocking thread (risingwavelabs…
Browse files Browse the repository at this point in the history
…#8624)

Signed-off-by: Bugen Zhao <i@bugenzhao.com>
  • Loading branch information
BugenZhao authored Mar 20, 2023
1 parent f111bfb commit da57717
Show file tree
Hide file tree
Showing 3 changed files with 40 additions and 25 deletions.
56 changes: 36 additions & 20 deletions src/stream/src/executor/actor.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,9 @@
use std::sync::atomic::{AtomicUsize, Ordering};
use std::sync::Arc;

use anyhow::anyhow;
use await_tree::InstrumentAwait;
use futures::future::join_all;
use futures::pin_mut;
use hytra::TrAdder;
use minitrace::prelude::*;
use parking_lot::Mutex;
Expand Down Expand Up @@ -173,33 +173,33 @@ where
};

let mut last_epoch: Option<EpochPair> = None;

let stream = Box::new(self.consumer).execute();
pin_mut!(stream);
let mut stream = Box::pin(Box::new(self.consumer).execute());

// Drive the streaming task with an infinite loop
while let Some(barrier) = stream
.next()
.in_span(span)
.instrument_await(
last_epoch.map_or("Epoch <initial>".into(), |e| format!("Epoch {}", e.curr)),
)
.await
.transpose()?
{
last_epoch = Some(barrier.epoch);
let result = loop {
let barrier = match stream
.try_next()
.in_span(span)
.instrument_await(
last_epoch.map_or("Epoch <initial>".into(), |e| format!("Epoch {}", e.curr)),
)
.await
{
Ok(Some(barrier)) => barrier,
Ok(None) => break Err(anyhow!("actor exited unexpectedly").into()),
Err(err) => break Err(err),
};

// Collect barriers to local barrier manager
self.context.lock_barrier_manager().collect(id, &barrier);

// Then stop this actor if asked
let to_stop = barrier.is_stop_or_update_drop_actor(id);
if to_stop {
tracing::trace!(actor_id = id, "actor exit");
return Ok(());
if barrier.is_stop_or_update_drop_actor(id) {
break Ok(());
}

// Tracing related work
last_epoch = Some(barrier.epoch);
span = {
let mut span = Span::enter_with_local_parent("actor_poll");
span.add_property(|| ("otel.name", span_name.to_string()));
Expand All @@ -208,8 +208,24 @@ where
span.add_property(|| ("epoch", barrier.epoch.curr.to_string()));
span
};
}
};

spawn_blocking_drop_stream(stream).await;

Ok(())
tracing::trace!(actor_id = id, "actor exit");
result
}
}

/// Drop the stream in a blocking task to avoid interfering with other actors.
///
/// Logically the actor is dropped after we send the barrier with `Drop` mutation to the
/// downstream,thus making the `drop`'s progress asynchronous. However, there might be a
/// considerable amount of data in the executors' in-memory cache, dropping these structures might
/// be a CPU-intensive task. This may lead to the runtime being unable to schedule other actors if
/// the `drop` is called on the current thread.
pub async fn spawn_blocking_drop_stream<T: Send + 'static>(stream: T) {
let _ = tokio::task::spawn_blocking(move || drop(stream))
.instrument_await("drop_stream")
.await;
}
7 changes: 2 additions & 5 deletions src/stream/src/executor/dispatch.rs
Original file line number Diff line number Diff line change
Expand Up @@ -262,15 +262,12 @@ impl StreamConsumer for DispatchExecutor {
#[for_await]
for msg in input {
let msg: Message = msg?;
let (barrier, message) = match msg {
let (barrier, span) = match msg {
Message::Chunk(_) => (None, "dispatch_chunk"),
Message::Barrier(ref barrier) => (Some(barrier.clone()), "dispatch_barrier"),
Message::Watermark(_) => (None, "dispatch_watermark"),
};
self.inner
.dispatch(msg)
.verbose_instrument_await(message)
.await?;
self.inner.dispatch(msg).instrument_await(span).await?;
if let Some(barrier) = barrier {
yield barrier;
}
Expand Down
2 changes: 2 additions & 0 deletions src/stream/src/executor/subtask.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ use tokio::sync::mpsc;
use tokio::sync::mpsc::error::SendError;
use tokio_stream::wrappers::ReceiverStream;

use super::actor::spawn_blocking_drop_stream;
use super::{BoxedExecutor, Executor, ExecutorInfo, MessageStreamItem};

/// Handle used to drive the subtask.
Expand Down Expand Up @@ -85,6 +86,7 @@ pub fn wrap(input: BoxedExecutor) -> (SubtaskHandle, SubtaskRxExecutor) {
break;
}
}
spawn_blocking_drop_stream(input).await;
}
.instrument_await("Subtask");

Expand Down

0 comments on commit da57717

Please sign in to comment.