risingwave_stream/executor/
actor.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use std::collections::{HashMap, HashSet};
16use std::sync::atomic::{AtomicUsize, Ordering};
17use std::sync::{Arc, LazyLock};
18
19use anyhow::anyhow;
20use await_tree::InstrumentAwait;
21use futures::FutureExt;
22use futures::future::join_all;
23use hytra::TrAdder;
24use risingwave_common::bitmap::Bitmap;
25use risingwave_common::catalog::TableId;
26use risingwave_common::config::StreamingConfig;
27use risingwave_common::hash::VirtualNode;
28use risingwave_common::log::LogSuppresser;
29use risingwave_common::metrics::{GLOBAL_ERROR_METRICS, IntGaugeExt};
30use risingwave_common::util::epoch::EpochPair;
31use risingwave_expr::ExprError;
32use risingwave_expr::expr_context::{FRAGMENT_ID, VNODE_COUNT, expr_context_scope};
33use risingwave_pb::plan_common::ExprContext;
34use risingwave_pb::stream_service::inject_barrier_request::BuildActorInfo;
35use risingwave_pb::stream_service::inject_barrier_request::build_actor_info::UpstreamActors;
36use risingwave_rpc_client::MetaClient;
37use thiserror_ext::AsReport;
38use tokio_stream::StreamExt;
39use tracing::Instrument;
40
41use super::StreamConsumer;
42use super::monitor::StreamingMetrics;
43use super::subtask::SubtaskHandle;
44use crate::error::StreamResult;
45use crate::task::{ActorId, FragmentId, LocalBarrierManager};
46
47/// Shared by all operators of an actor.
48pub struct ActorContext {
49    pub id: ActorId,
50    pub fragment_id: u32,
51    pub vnode_count: usize,
52    pub mview_definition: String,
53
54    // TODO(eric): these seem to be useless now?
55    last_mem_val: Arc<AtomicUsize>,
56    cur_mem_val: Arc<AtomicUsize>,
57    total_mem_val: Arc<TrAdder<i64>>,
58
59    pub streaming_metrics: Arc<StreamingMetrics>,
60
61    /// This is the number of dispatchers when the actor is created. It will not be updated during runtime when new downstreams are added.
62    pub initial_dispatch_num: usize,
63    // mv_table_id to subscription id
64    pub related_subscriptions: Arc<HashMap<TableId, HashSet<u32>>>,
65    pub initial_upstream_actors: HashMap<FragmentId, UpstreamActors>,
66
67    // Meta client. currently used for auto schema change. `None` for test only
68    pub meta_client: Option<MetaClient>,
69
70    pub streaming_config: Arc<StreamingConfig>,
71}
72
73pub type ActorContextRef = Arc<ActorContext>;
74
75impl ActorContext {
76    pub fn for_test(id: ActorId) -> ActorContextRef {
77        Arc::new(Self {
78            id,
79            fragment_id: 0,
80            vnode_count: VirtualNode::COUNT_FOR_TEST,
81            mview_definition: "".to_owned(),
82            cur_mem_val: Arc::new(0.into()),
83            last_mem_val: Arc::new(0.into()),
84            total_mem_val: Arc::new(TrAdder::new()),
85            streaming_metrics: Arc::new(StreamingMetrics::unused()),
86            // Set 1 for test to enable sanity check on table
87            initial_dispatch_num: 1,
88            related_subscriptions: HashMap::new().into(),
89            initial_upstream_actors: Default::default(),
90            meta_client: None,
91            streaming_config: Arc::new(StreamingConfig::default()),
92        })
93    }
94
95    pub fn create(
96        stream_actor: &BuildActorInfo,
97        fragment_id: FragmentId,
98        total_mem_val: Arc<TrAdder<i64>>,
99        streaming_metrics: Arc<StreamingMetrics>,
100        related_subscriptions: Arc<HashMap<TableId, HashSet<u32>>>,
101        meta_client: Option<MetaClient>,
102        streaming_config: Arc<StreamingConfig>,
103    ) -> ActorContextRef {
104        Arc::new(Self {
105            id: stream_actor.actor_id,
106            fragment_id,
107            mview_definition: stream_actor.mview_definition.clone(),
108            vnode_count: (stream_actor.vnode_bitmap.as_ref())
109                // An unset `vnode_bitmap` means the actor is a singleton,
110                // where only `SINGLETON_VNODE` is set.
111                .map_or(1, |b| Bitmap::from(b).len()),
112            cur_mem_val: Arc::new(0.into()),
113            last_mem_val: Arc::new(0.into()),
114            total_mem_val,
115            streaming_metrics,
116            initial_dispatch_num: stream_actor.dispatchers.len(),
117            related_subscriptions,
118            initial_upstream_actors: stream_actor.fragment_upstreams.clone(),
119            meta_client,
120            streaming_config,
121        })
122    }
123
124    pub fn on_compute_error(&self, err: ExprError, identity: &str) {
125        static LOG_SUPPERSSER: LazyLock<LogSuppresser> = LazyLock::new(LogSuppresser::default);
126        if let Ok(suppressed_count) = LOG_SUPPERSSER.check() {
127            tracing::error!(identity, error = %err.as_report(), suppressed_count, "failed to evaluate expression");
128        }
129
130        let executor_name = identity.split(' ').next().unwrap_or("name_not_found");
131        GLOBAL_ERROR_METRICS.user_compute_error.report([
132            "ExprError".to_owned(),
133            executor_name.to_owned(),
134            self.fragment_id.to_string(),
135        ]);
136    }
137
138    pub fn store_mem_usage(&self, val: usize) {
139        // Record the last mem val.
140        // Calculate the difference between old val and new value, and apply the diff to total
141        // memory usage value.
142        let old_value = self.cur_mem_val.load(Ordering::Relaxed);
143        self.last_mem_val.store(old_value, Ordering::Relaxed);
144        let diff = val as i64 - old_value as i64;
145
146        self.total_mem_val.inc(diff);
147
148        self.cur_mem_val.store(val, Ordering::Relaxed);
149    }
150
151    pub fn mem_usage(&self) -> usize {
152        self.cur_mem_val.load(Ordering::Relaxed)
153    }
154}
155
156/// `Actor` is the basic execution unit in the streaming framework.
157pub struct Actor<C> {
158    /// The [`StreamConsumer`] of the actor.
159    consumer: C,
160    /// The subtasks to execute concurrently.
161    subtasks: Vec<SubtaskHandle>,
162
163    pub actor_context: ActorContextRef,
164    expr_context: ExprContext,
165    barrier_manager: LocalBarrierManager,
166}
167
168impl<C> Actor<C>
169where
170    C: StreamConsumer,
171{
172    pub fn new(
173        consumer: C,
174        subtasks: Vec<SubtaskHandle>,
175        _metrics: Arc<StreamingMetrics>,
176        actor_context: ActorContextRef,
177        expr_context: ExprContext,
178        barrier_manager: LocalBarrierManager,
179    ) -> Self {
180        Self {
181            consumer,
182            subtasks,
183            actor_context,
184            expr_context,
185            barrier_manager,
186        }
187    }
188
189    #[inline(always)]
190    pub async fn run(mut self) -> StreamResult<()> {
191        let expr_context = self.expr_context.clone();
192        let fragment_id = self.actor_context.fragment_id;
193        let vnode_count = self.actor_context.vnode_count;
194
195        let run = async move {
196            tokio::join!(
197                // Drive the subtasks concurrently.
198                join_all(std::mem::take(&mut self.subtasks)),
199                self.run_consumer(),
200            )
201            .1
202        }
203        .boxed();
204
205        // Attach contexts to the future.
206        let run = expr_context_scope(expr_context, run);
207        let run = FRAGMENT_ID::scope(fragment_id, run);
208        let run = VNODE_COUNT::scope(vnode_count, run);
209
210        run.await
211    }
212
213    async fn run_consumer(self) -> StreamResult<()> {
214        fail::fail_point!("start_actors_err", |_| Err(anyhow::anyhow!(
215            "intentional start_actors_err"
216        )
217        .into()));
218
219        let id = self.actor_context.id;
220        let span_name = format!("Actor {id}");
221
222        let new_span = |epoch: Option<EpochPair>| {
223            tracing::info_span!(
224                parent: None,
225                "actor",
226                "otel.name" = span_name,
227                actor_id = id,
228                prev_epoch = epoch.map(|e| e.prev),
229                curr_epoch = epoch.map(|e| e.curr),
230            )
231        };
232        let mut span = new_span(None);
233
234        let actor_count = self
235            .actor_context
236            .streaming_metrics
237            .actor_count
238            .with_guarded_label_values(&[&self.actor_context.fragment_id.to_string()]);
239        let _actor_count_guard = actor_count.inc_guard();
240
241        let current_epoch = self
242            .actor_context
243            .streaming_metrics
244            .actor_current_epoch
245            .with_guarded_label_values(&[
246                &self.actor_context.id.to_string(),
247                &self.actor_context.fragment_id.to_string(),
248            ]);
249
250        let mut last_epoch: Option<EpochPair> = None;
251        let mut stream = Box::pin(Box::new(self.consumer).execute());
252
253        // Drive the streaming task with an infinite loop
254        let result = loop {
255            let barrier = match stream
256                .try_next()
257                .instrument(span.clone())
258                .instrument_await(
259                    last_epoch.map_or(await_tree::span!("Epoch <initial>"), |e| {
260                        await_tree::span!("Epoch {}", e.curr)
261                    }),
262                )
263                .await
264            {
265                Ok(Some(barrier)) => barrier,
266                Ok(None) => break Err(anyhow!("actor exited unexpectedly").into()),
267                Err(err) => break Err(err),
268            };
269
270            fail::fail_point!("collect_actors_err", id == 10, |_| Err(anyhow::anyhow!(
271                "intentional collect_actors_err"
272            )
273            .into()));
274
275            // Then stop this actor if asked
276            if barrier.is_stop(id) {
277                debug!(actor_id = id, epoch = ?barrier.epoch, "stop at barrier");
278                break Ok(barrier);
279            }
280
281            current_epoch.set(barrier.epoch.curr as i64);
282
283            // Collect barriers to local barrier manager
284            self.barrier_manager.collect(id, &barrier);
285
286            // Tracing related work
287            last_epoch = Some(barrier.epoch);
288            span = barrier.tracing_context().attach(new_span(last_epoch));
289        };
290
291        spawn_blocking_drop_stream(stream).await;
292
293        let result = result.map(|stop_barrier| {
294            // Collect the stop barrier after the stream has been dropped to ensure that all resources
295            self.barrier_manager.collect(id, &stop_barrier);
296        });
297
298        tracing::debug!(actor_id = id, ok = result.is_ok(), "actor exit");
299        result
300    }
301}
302
303/// Drop the stream in a blocking task to avoid interfering with other actors.
304///
305/// Logically the actor is dropped after we send the barrier with `Drop` mutation to the
306/// downstream, thus making the `drop`'s progress asynchronous. However, there might be a
307/// considerable amount of data in the executors' in-memory cache, dropping these structures might
308/// be a CPU-intensive task. This may lead to the runtime being unable to schedule other actors if
309/// the `drop` is called on the current thread.
310pub async fn spawn_blocking_drop_stream<T: Send + 'static>(stream: T) {
311    let _ = tokio::task::spawn_blocking(move || drop(stream))
312        .instrument_await("drop_stream")
313        .await;
314}