risingwave_stream/executor/
mod.rs

1// Copyright 2022 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
15mod prelude;
16
17use std::collections::{BTreeMap, HashMap, HashSet, VecDeque};
18use std::fmt::Debug;
19use std::future::pending;
20use std::hash::Hash;
21use std::pin::Pin;
22use std::sync::Arc;
23use std::task::Poll;
24use std::vec;
25
26use await_tree::InstrumentAwait;
27use enum_as_inner::EnumAsInner;
28use futures::future::try_join_all;
29use futures::stream::{BoxStream, FusedStream, FuturesUnordered, StreamFuture};
30use futures::{FutureExt, Stream, StreamExt, TryStreamExt};
31use itertools::Itertools;
32use prometheus::core::{AtomicU64, GenericCounter};
33use risingwave_common::array::StreamChunk;
34use risingwave_common::bitmap::Bitmap;
35use risingwave_common::catalog::{Schema, TableId};
36use risingwave_common::config::StreamingConfig;
37use risingwave_common::metrics::LabelGuardedMetric;
38use risingwave_common::row::OwnedRow;
39use risingwave_common::types::{DataType, Datum, DefaultOrd, ScalarImpl};
40use risingwave_common::util::epoch::{Epoch, EpochPair};
41use risingwave_common::util::tracing::TracingContext;
42use risingwave_common::util::value_encoding::{DatumFromProtoExt, DatumToProtoExt};
43use risingwave_connector::source::SplitImpl;
44use risingwave_expr::expr::{Expression, NonStrictExpression};
45use risingwave_pb::data::PbEpoch;
46use risingwave_pb::expr::PbInputRef;
47use risingwave_pb::stream_plan::add_mutation::PbNewUpstreamSink;
48use risingwave_pb::stream_plan::barrier::BarrierKind;
49use risingwave_pb::stream_plan::barrier_mutation::Mutation as PbMutation;
50use risingwave_pb::stream_plan::stream_node::PbStreamKind;
51use risingwave_pb::stream_plan::throttle_mutation::ThrottleConfig;
52use risingwave_pb::stream_plan::update_mutation::{DispatcherUpdate, MergeUpdate};
53use risingwave_pb::stream_plan::{
54    PbBarrier, PbBarrierMutation, PbDispatcher, PbSinkSchemaChange, PbStreamMessageBatch,
55    PbWatermark, SubscriptionUpstreamInfo,
56};
57use smallvec::SmallVec;
58use tokio::sync::mpsc;
59use tokio::time::{Duration, Instant};
60
61use crate::error::StreamResult;
62use crate::executor::exchange::input::{
63    BoxedActorInput, BoxedInput, assert_equal_dispatcher_barrier, new_input,
64};
65use crate::executor::monitor::ActorInputMetrics;
66use crate::executor::prelude::StreamingMetrics;
67use crate::executor::watermark::BufferedWatermarks;
68use crate::task::{ActorId, FragmentId, LocalBarrierManager};
69
70mod actor;
71mod barrier_align;
72pub mod exchange;
73pub mod monitor;
74
75pub mod aggregate;
76pub mod asof_join;
77mod backfill;
78mod barrier_recv;
79mod batch_query;
80mod chain;
81mod changelog;
82mod dedup;
83mod dispatch;
84pub mod dml;
85mod dynamic_filter;
86pub mod eowc;
87pub mod error;
88mod expand;
89mod filter;
90mod gap_fill;
91pub mod hash_join;
92mod hop_window;
93mod join;
94pub mod locality_provider;
95mod lookup;
96mod lookup_union;
97mod merge;
98mod mview;
99mod nested_loop_temporal_join;
100mod no_op;
101mod now;
102mod over_window;
103pub mod project;
104mod rearranged_chain;
105mod receiver;
106pub mod row_id_gen;
107mod sink;
108pub mod source;
109mod stream_reader;
110pub mod subtask;
111mod temporal_join;
112mod top_n;
113mod troublemaker;
114mod union;
115mod upstream_sink_union;
116mod values;
117mod watermark;
118mod watermark_filter;
119mod wrapper;
120
121mod approx_percentile;
122
123mod row_merge;
124
125#[cfg(test)]
126mod integration_tests;
127mod sync_kv_log_store;
128#[cfg(any(test, feature = "test"))]
129pub mod test_utils;
130mod utils;
131mod vector;
132
133pub use actor::{Actor, ActorContext, ActorContextRef};
134use anyhow::Context;
135pub use approx_percentile::global::GlobalApproxPercentileExecutor;
136pub use approx_percentile::local::LocalApproxPercentileExecutor;
137pub use backfill::arrangement_backfill::*;
138pub use backfill::cdc::{
139    CdcBackfillExecutor, ExternalStorageTable, ParallelizedCdcBackfillExecutor,
140};
141pub use backfill::no_shuffle_backfill::*;
142pub use backfill::snapshot_backfill::*;
143pub use barrier_recv::BarrierRecvExecutor;
144pub use batch_query::BatchQueryExecutor;
145pub use chain::ChainExecutor;
146pub use changelog::ChangeLogExecutor;
147pub use dedup::AppendOnlyDedupExecutor;
148pub use dispatch::DispatchExecutor;
149pub use dynamic_filter::DynamicFilterExecutor;
150pub use error::{StreamExecutorError, StreamExecutorResult};
151pub use expand::ExpandExecutor;
152pub use filter::{FilterExecutor, UpsertFilterExecutor};
153pub use gap_fill::{GapFillExecutor, GapFillExecutorArgs};
154pub use hash_join::*;
155pub use hop_window::HopWindowExecutor;
156pub use join::asof_join::{AsOfCpuEncoding, AsOfMemoryEncoding};
157pub use join::row::{CachedJoinRow, CpuEncoding, JoinEncoding, MemoryEncoding};
158pub use join::{AsOfDesc, AsOfJoinType, JoinType};
159pub use lookup::*;
160pub use lookup_union::LookupUnionExecutor;
161pub use merge::MergeExecutor;
162pub(crate) use merge::{MergeExecutorInput, MergeExecutorUpstream};
163pub use mview::{MaterializeExecutor, RefreshableMaterializeArgs};
164pub use nested_loop_temporal_join::NestedLoopTemporalJoinExecutor;
165pub use no_op::NoOpExecutor;
166pub use now::*;
167pub use over_window::*;
168pub use rearranged_chain::RearrangedChainExecutor;
169pub use receiver::ReceiverExecutor;
170use risingwave_common::id::SourceId;
171pub use row_merge::RowMergeExecutor;
172pub use sink::SinkExecutor;
173pub use sync_kv_log_store::SyncedKvLogStoreExecutor;
174pub use sync_kv_log_store::metrics::SyncedKvLogStoreMetrics;
175pub use temporal_join::TemporalJoinExecutor;
176pub use top_n::{
177    AppendOnlyGroupTopNExecutor, AppendOnlyTopNExecutor, GroupTopNExecutor, TopNExecutor,
178};
179pub use troublemaker::TroublemakerExecutor;
180pub use union::UnionExecutor;
181pub use upstream_sink_union::{UpstreamFragmentInfo, UpstreamSinkUnionExecutor};
182pub use utils::DummyExecutor;
183pub use values::ValuesExecutor;
184pub use vector::*;
185pub use watermark_filter::{UpsertWatermarkFilterExecutor, WatermarkFilterExecutor};
186pub use wrapper::WrapperExecutor;
187
188use self::barrier_align::AlignedMessageStream;
189
190pub type MessageStreamItemInner<M> = StreamExecutorResult<MessageInner<M>>;
191pub type MessageStreamItem = MessageStreamItemInner<BarrierMutationType>;
192pub type DispatcherMessageStreamItem = StreamExecutorResult<DispatcherMessage>;
193pub type BoxedMessageStream = BoxStream<'static, MessageStreamItem>;
194
195pub use risingwave_common::util::epoch::task_local::{curr_epoch, epoch, prev_epoch};
196use risingwave_connector::sink::catalog::SinkId;
197use risingwave_connector::source::cdc::{
198    CdcTableSnapshotSplitAssignmentWithGeneration,
199    build_actor_cdc_table_snapshot_splits_with_generation,
200};
201use risingwave_pb::id::{ExecutorId, SubscriberId};
202use risingwave_pb::stream_plan::stream_message_batch::{BarrierBatch, StreamMessageBatch};
203
204pub trait MessageStreamInner<M> = Stream<Item = MessageStreamItemInner<M>> + Send;
205pub trait MessageStream = Stream<Item = MessageStreamItem> + Send;
206pub trait DispatcherMessageStream = Stream<Item = DispatcherMessageStreamItem> + Send;
207
208/// Static information of an executor.
209#[derive(Debug, Default, Clone)]
210pub struct ExecutorInfo {
211    /// The schema of the OUTPUT of the executor.
212    pub schema: Schema,
213
214    /// The stream key indices of the OUTPUT of the executor.
215    pub stream_key: StreamKey,
216
217    /// The stream kind of the OUTPUT of the executor.
218    pub stream_kind: PbStreamKind,
219
220    /// Identity of the executor.
221    pub identity: String,
222
223    /// The executor id of the executor.
224    pub id: ExecutorId,
225}
226
227impl ExecutorInfo {
228    pub fn for_test(schema: Schema, stream_key: StreamKey, identity: String, id: u64) -> Self {
229        Self {
230            schema,
231            stream_key,
232            stream_kind: PbStreamKind::Retract, // dummy value for test
233            identity,
234            id: id.into(),
235        }
236    }
237}
238
239/// [`Execute`] describes the methods an executor should implement to handle control messages.
240pub trait Execute: Send + 'static {
241    fn execute(self: Box<Self>) -> BoxedMessageStream;
242
243    fn execute_with_epoch(self: Box<Self>, _epoch: u64) -> BoxedMessageStream {
244        self.execute()
245    }
246
247    fn boxed(self) -> Box<dyn Execute>
248    where
249        Self: Sized + Send + 'static,
250    {
251        Box::new(self)
252    }
253}
254
255/// [`Executor`] combines the static information ([`ExecutorInfo`]) and the executable object to
256/// handle messages ([`Execute`]).
257pub struct Executor {
258    info: ExecutorInfo,
259    execute: Box<dyn Execute>,
260}
261
262impl Executor {
263    pub fn new(info: ExecutorInfo, execute: Box<dyn Execute>) -> Self {
264        Self { info, execute }
265    }
266
267    pub fn info(&self) -> &ExecutorInfo {
268        &self.info
269    }
270
271    pub fn schema(&self) -> &Schema {
272        &self.info.schema
273    }
274
275    pub fn stream_key(&self) -> StreamKeyRef<'_> {
276        &self.info.stream_key
277    }
278
279    pub fn stream_kind(&self) -> PbStreamKind {
280        self.info.stream_kind
281    }
282
283    pub fn identity(&self) -> &str {
284        &self.info.identity
285    }
286
287    pub fn execute(self) -> BoxedMessageStream {
288        self.execute.execute()
289    }
290
291    pub fn execute_with_epoch(self, epoch: u64) -> BoxedMessageStream {
292        self.execute.execute_with_epoch(epoch)
293    }
294}
295
296impl std::fmt::Debug for Executor {
297    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
298        f.write_str(self.identity())
299    }
300}
301
302impl From<(ExecutorInfo, Box<dyn Execute>)> for Executor {
303    fn from((info, execute): (ExecutorInfo, Box<dyn Execute>)) -> Self {
304        Self::new(info, execute)
305    }
306}
307
308impl<E> From<(ExecutorInfo, E)> for Executor
309where
310    E: Execute,
311{
312    fn from((info, execute): (ExecutorInfo, E)) -> Self {
313        Self::new(info, execute.boxed())
314    }
315}
316
317pub const INVALID_EPOCH: u64 = 0;
318
319type UpstreamFragmentId = FragmentId;
320type SplitAssignments = HashMap<ActorId, Vec<SplitImpl>>;
321
322#[derive(Debug, Clone)]
323#[cfg_attr(any(test, feature = "test"), derive(Default, PartialEq))]
324pub struct UpdateMutation {
325    pub dispatchers: HashMap<ActorId, Vec<DispatcherUpdate>>,
326    pub merges: HashMap<(ActorId, UpstreamFragmentId), MergeUpdate>,
327    pub vnode_bitmaps: HashMap<ActorId, Arc<Bitmap>>,
328    pub dropped_actors: HashSet<ActorId>,
329    pub actor_splits: SplitAssignments,
330    pub actor_new_dispatchers: HashMap<ActorId, Vec<PbDispatcher>>,
331    pub actor_cdc_table_snapshot_splits: CdcTableSnapshotSplitAssignmentWithGeneration,
332    pub sink_schema_change: HashMap<SinkId, PbSinkSchemaChange>,
333    pub subscriptions_to_drop: Vec<SubscriptionUpstreamInfo>,
334}
335
336#[derive(Debug, Clone)]
337#[cfg_attr(any(test, feature = "test"), derive(Default, PartialEq))]
338pub struct AddMutation {
339    pub adds: HashMap<ActorId, Vec<PbDispatcher>>,
340    pub added_actors: HashSet<ActorId>,
341    // TODO: remove this and use `SourceChangesSplit` after we support multiple mutations.
342    pub splits: SplitAssignments,
343    pub pause: bool,
344    /// (`upstream_mv_table_id`,  `subscriber_id`)
345    pub subscriptions_to_add: Vec<(TableId, SubscriberId)>,
346    /// nodes which should start backfill
347    pub backfill_nodes_to_pause: HashSet<FragmentId>,
348    pub actor_cdc_table_snapshot_splits: CdcTableSnapshotSplitAssignmentWithGeneration,
349    pub new_upstream_sinks: HashMap<FragmentId, PbNewUpstreamSink>,
350}
351
352#[derive(Debug, Clone)]
353#[cfg_attr(any(test, feature = "test"), derive(Default, PartialEq))]
354pub struct StopMutation {
355    pub dropped_actors: HashSet<ActorId>,
356    pub dropped_sink_fragments: HashSet<FragmentId>,
357}
358
359/// See [`PbMutation`] for the semantics of each mutation.
360#[cfg_attr(any(test, feature = "test"), derive(PartialEq))]
361#[derive(Debug, Clone)]
362pub enum Mutation {
363    Stop(StopMutation),
364    Update(UpdateMutation),
365    Add(AddMutation),
366    SourceChangeSplit(SplitAssignments),
367    Pause,
368    Resume,
369    Throttle(HashMap<FragmentId, ThrottleConfig>),
370    ConnectorPropsChange(HashMap<u32, HashMap<String, String>>),
371    DropSubscriptions {
372        /// `subscriber` -> `upstream_mv_table_id`
373        subscriptions_to_drop: Vec<SubscriptionUpstreamInfo>,
374    },
375    StartFragmentBackfill {
376        fragment_ids: HashSet<FragmentId>,
377    },
378    RefreshStart {
379        table_id: TableId,
380        associated_source_id: SourceId,
381    },
382    ListFinish {
383        associated_source_id: SourceId,
384    },
385    LoadFinish {
386        associated_source_id: SourceId,
387    },
388    ResetSource {
389        source_id: SourceId,
390    },
391    InjectSourceOffsets {
392        source_id: SourceId,
393        /// Split ID -> offset (JSON-encoded based on connector type)
394        split_offsets: HashMap<String, String>,
395    },
396}
397
398/// The generic type `M` is the mutation type of the barrier.
399///
400/// For barrier of in the dispatcher, `M` is `()`, which means the mutation is erased.
401/// For barrier flowing within the streaming actor, `M` is the normal `BarrierMutationType`.
402#[derive(Debug, Clone)]
403pub struct BarrierInner<M> {
404    pub epoch: EpochPair,
405    pub mutation: M,
406    pub kind: BarrierKind,
407
408    /// Tracing context for the **current** epoch of this barrier.
409    pub tracing_context: TracingContext,
410}
411
412pub type BarrierMutationType = Option<Arc<Mutation>>;
413pub type Barrier = BarrierInner<BarrierMutationType>;
414pub type DispatcherBarrier = BarrierInner<()>;
415
416impl<M: Default> BarrierInner<M> {
417    /// Create a plain barrier.
418    pub fn new_test_barrier(epoch: u64) -> Self {
419        Self {
420            epoch: EpochPair::new_test_epoch(epoch),
421            kind: BarrierKind::Checkpoint,
422            tracing_context: TracingContext::none(),
423            mutation: Default::default(),
424        }
425    }
426
427    pub fn with_prev_epoch_for_test(epoch: u64, prev_epoch: u64) -> Self {
428        Self {
429            epoch: EpochPair::new(epoch, prev_epoch),
430            kind: BarrierKind::Checkpoint,
431            tracing_context: TracingContext::none(),
432            mutation: Default::default(),
433        }
434    }
435}
436
437impl Barrier {
438    pub fn into_dispatcher(self) -> DispatcherBarrier {
439        DispatcherBarrier {
440            epoch: self.epoch,
441            mutation: (),
442            kind: self.kind,
443            tracing_context: self.tracing_context,
444        }
445    }
446
447    #[must_use]
448    pub fn with_mutation(self, mutation: Mutation) -> Self {
449        Self {
450            mutation: Some(Arc::new(mutation)),
451            ..self
452        }
453    }
454
455    #[must_use]
456    pub fn with_stop(self) -> Self {
457        self.with_mutation(Mutation::Stop(StopMutation {
458            dropped_actors: Default::default(),
459            dropped_sink_fragments: Default::default(),
460        }))
461    }
462
463    /// Whether this barrier carries stop mutation.
464    pub fn is_with_stop_mutation(&self) -> bool {
465        matches!(self.mutation.as_deref(), Some(Mutation::Stop(_)))
466    }
467
468    /// Whether this barrier is to stop the actor with `actor_id`.
469    pub fn is_stop(&self, actor_id: ActorId) -> bool {
470        self.all_stop_actors()
471            .is_some_and(|actors| actors.contains(&actor_id))
472    }
473
474    pub fn is_checkpoint(&self) -> bool {
475        self.kind == BarrierKind::Checkpoint
476    }
477
478    /// Get the initial split assignments for the actor with `actor_id`.
479    ///
480    /// This should only be called on the initial barrier received by the executor. It must be
481    ///
482    /// - `Add` mutation when it's a new streaming job, or recovery.
483    /// - `Update` mutation when it's created for scaling.
484    ///
485    /// Note that `SourceChangeSplit` is **not** included, because it's only used for changing splits
486    /// of existing executors.
487    pub fn initial_split_assignment(&self, actor_id: ActorId) -> Option<&[SplitImpl]> {
488        match self.mutation.as_deref()? {
489            Mutation::Update(UpdateMutation { actor_splits, .. })
490            | Mutation::Add(AddMutation {
491                splits: actor_splits,
492                ..
493            }) => actor_splits.get(&actor_id),
494
495            _ => {
496                if cfg!(debug_assertions) {
497                    panic!(
498                        "the initial mutation of the barrier should not be {:?}",
499                        self.mutation
500                    );
501                }
502                None
503            }
504        }
505        .map(|s| s.as_slice())
506    }
507
508    /// Get all actors that to be stopped (dropped) by this barrier.
509    pub fn all_stop_actors(&self) -> Option<&HashSet<ActorId>> {
510        match self.mutation.as_deref() {
511            Some(Mutation::Stop(StopMutation { dropped_actors, .. })) => Some(dropped_actors),
512            Some(Mutation::Update(UpdateMutation { dropped_actors, .. })) => Some(dropped_actors),
513            _ => None,
514        }
515    }
516
517    /// Whether this barrier is to newly add the actor with `actor_id`. This is used for `Chain` and
518    /// `Values` to decide whether to output the existing (historical) data.
519    ///
520    /// By "newly", we mean the actor belongs to a subgraph of a new streaming job. That is, actors
521    /// added for scaling are not included.
522    pub fn is_newly_added(&self, actor_id: ActorId) -> bool {
523        match self.mutation.as_deref() {
524            Some(Mutation::Add(AddMutation { added_actors, .. })) => {
525                added_actors.contains(&actor_id)
526            }
527            _ => false,
528        }
529    }
530
531    pub fn should_start_fragment_backfill(&self, fragment_id: FragmentId) -> bool {
532        if let Some(Mutation::StartFragmentBackfill { fragment_ids }) = self.mutation.as_deref() {
533            fragment_ids.contains(&fragment_id)
534        } else {
535            false
536        }
537    }
538
539    /// Whether this barrier adds new downstream fragment for the actor with `upstream_actor_id`.
540    ///
541    /// # Use case
542    /// Some optimizations are applied when an actor doesn't have any downstreams ("standalone" actors).
543    /// * Pause a standalone shared `SourceExecutor`.
544    /// * Disable a standalone `MaterializeExecutor`'s conflict check.
545    ///
546    /// This is implemented by checking `actor_context.initial_dispatch_num` on startup, and
547    /// check `has_more_downstream_fragments` on barrier to see whether the optimization
548    /// needs to be turned off.
549    ///
550    /// ## Some special cases not included
551    ///
552    /// Note that this is not `has_new_downstream_actor/fragment`. For our use case, we only
553    /// care about **number of downstream fragments** (more precisely, existence).
554    /// - When scaling, the number of downstream actors is changed, and they are "new", but downstream fragments is not changed.
555    /// - When `ALTER TABLE sink_into_table`, the fragment is replaced with a "new" one, but the number is not changed.
556    pub fn has_more_downstream_fragments(&self, upstream_actor_id: ActorId) -> bool {
557        let Some(mutation) = self.mutation.as_deref() else {
558            return false;
559        };
560        match mutation {
561            // Add is for mv, index and sink creation.
562            Mutation::Add(AddMutation { adds, .. }) => adds.get(&upstream_actor_id).is_some(),
563            Mutation::Update(_)
564            | Mutation::Stop(_)
565            | Mutation::Pause
566            | Mutation::Resume
567            | Mutation::SourceChangeSplit(_)
568            | Mutation::Throttle { .. }
569            | Mutation::DropSubscriptions { .. }
570            | Mutation::ConnectorPropsChange(_)
571            | Mutation::StartFragmentBackfill { .. }
572            | Mutation::RefreshStart { .. }
573            | Mutation::ListFinish { .. }
574            | Mutation::LoadFinish { .. }
575            | Mutation::ResetSource { .. }
576            | Mutation::InjectSourceOffsets { .. } => false,
577        }
578    }
579
580    /// Whether this barrier requires the executor to pause its data stream on startup.
581    pub fn is_pause_on_startup(&self) -> bool {
582        match self.mutation.as_deref() {
583            Some(Mutation::Add(AddMutation { pause, .. })) => *pause,
584            _ => false,
585        }
586    }
587
588    pub fn is_backfill_pause_on_startup(&self, backfill_fragment_id: FragmentId) -> bool {
589        match self.mutation.as_deref() {
590            Some(Mutation::Add(AddMutation {
591                backfill_nodes_to_pause,
592                ..
593            })) => backfill_nodes_to_pause.contains(&backfill_fragment_id),
594            Some(Mutation::Update(_)) => false,
595            _ => {
596                tracing::warn!(
597                    "expected an AddMutation or UpdateMutation on Startup, instead got {:?}",
598                    self
599                );
600                false
601            }
602        }
603    }
604
605    /// Whether this barrier is for resume.
606    pub fn is_resume(&self) -> bool {
607        matches!(self.mutation.as_deref(), Some(Mutation::Resume))
608    }
609
610    /// Returns the [`MergeUpdate`] if this barrier is to update the merge executors for the actor
611    /// with `actor_id`.
612    pub fn as_update_merge(
613        &self,
614        actor_id: ActorId,
615        upstream_fragment_id: UpstreamFragmentId,
616    ) -> Option<&MergeUpdate> {
617        self.mutation
618            .as_deref()
619            .and_then(|mutation| match mutation {
620                Mutation::Update(UpdateMutation { merges, .. }) => {
621                    merges.get(&(actor_id, upstream_fragment_id))
622                }
623                _ => None,
624            })
625    }
626
627    /// Returns the new upstream sink information if this barrier is to add a new upstream sink for
628    /// the specified downstream fragment.
629    pub fn as_new_upstream_sink(&self, fragment_id: FragmentId) -> Option<&PbNewUpstreamSink> {
630        self.mutation
631            .as_deref()
632            .and_then(|mutation| match mutation {
633                Mutation::Add(AddMutation {
634                    new_upstream_sinks, ..
635                }) => new_upstream_sinks.get(&fragment_id),
636                _ => None,
637            })
638    }
639
640    /// Returns the dropped upstream sink-fragment if this barrier is to drop any sink.
641    pub fn as_dropped_upstream_sinks(&self) -> Option<&HashSet<FragmentId>> {
642        self.mutation
643            .as_deref()
644            .and_then(|mutation| match mutation {
645                Mutation::Stop(StopMutation {
646                    dropped_sink_fragments,
647                    ..
648                }) => Some(dropped_sink_fragments),
649                _ => None,
650            })
651    }
652
653    /// Returns the new vnode bitmap if this barrier is to update the vnode bitmap for the actor
654    /// with `actor_id`.
655    ///
656    /// Actually, this vnode bitmap update is only useful for the record accessing validation for
657    /// distributed executors, since the read/write pattern will never be across multiple vnodes.
658    pub fn as_update_vnode_bitmap(&self, actor_id: ActorId) -> Option<Arc<Bitmap>> {
659        self.mutation
660            .as_deref()
661            .and_then(|mutation| match mutation {
662                Mutation::Update(UpdateMutation { vnode_bitmaps, .. }) => {
663                    vnode_bitmaps.get(&actor_id).cloned()
664                }
665                _ => None,
666            })
667    }
668
669    pub fn as_sink_schema_change(&self, sink_id: SinkId) -> Option<PbSinkSchemaChange> {
670        self.mutation
671            .as_deref()
672            .and_then(|mutation| match mutation {
673                Mutation::Update(UpdateMutation {
674                    sink_schema_change, ..
675                }) => sink_schema_change.get(&sink_id).cloned(),
676                _ => None,
677            })
678    }
679
680    pub fn as_subscriptions_to_drop(&self) -> Option<&[SubscriptionUpstreamInfo]> {
681        match self.mutation.as_deref() {
682            Some(Mutation::DropSubscriptions {
683                subscriptions_to_drop,
684            })
685            | Some(Mutation::Update(UpdateMutation {
686                subscriptions_to_drop,
687                ..
688            })) => Some(subscriptions_to_drop.as_slice()),
689            _ => None,
690        }
691    }
692
693    pub fn get_curr_epoch(&self) -> Epoch {
694        Epoch(self.epoch.curr)
695    }
696
697    /// Retrieve the tracing context for the **current** epoch of this barrier.
698    pub fn tracing_context(&self) -> &TracingContext {
699        &self.tracing_context
700    }
701
702    pub fn added_subscriber_on_mv_table(
703        &self,
704        mv_table_id: TableId,
705    ) -> impl Iterator<Item = SubscriberId> + '_ {
706        if let Some(Mutation::Add(add)) = self.mutation.as_deref() {
707            Some(add)
708        } else {
709            None
710        }
711        .into_iter()
712        .flat_map(move |add| {
713            add.subscriptions_to_add.iter().filter_map(
714                move |(upstream_mv_table_id, subscriber_id)| {
715                    if *upstream_mv_table_id == mv_table_id {
716                        Some(*subscriber_id)
717                    } else {
718                        None
719                    }
720                },
721            )
722        })
723    }
724}
725
726impl<M: PartialEq> PartialEq for BarrierInner<M> {
727    fn eq(&self, other: &Self) -> bool {
728        self.epoch == other.epoch && self.mutation == other.mutation
729    }
730}
731
732impl Mutation {
733    /// Return true if the mutation is stop.
734    ///
735    /// Note that this does not mean we will stop the current actor.
736    #[cfg(test)]
737    pub fn is_stop(&self) -> bool {
738        matches!(self, Mutation::Stop(_))
739    }
740
741    #[cfg(test)]
742    fn to_protobuf(&self) -> PbMutation {
743        use risingwave_pb::source::{
744            ConnectorSplit, ConnectorSplits, PbCdcTableSnapshotSplitsWithGeneration,
745        };
746        use risingwave_pb::stream_plan::connector_props_change_mutation::ConnectorPropsInfo;
747        use risingwave_pb::stream_plan::{
748            PbAddMutation, PbConnectorPropsChangeMutation, PbDispatchers,
749            PbDropSubscriptionsMutation, PbPauseMutation, PbResumeMutation,
750            PbSourceChangeSplitMutation, PbStartFragmentBackfillMutation, PbStopMutation,
751            PbThrottleMutation, PbUpdateMutation,
752        };
753        let actor_splits_to_protobuf = |actor_splits: &SplitAssignments| {
754            actor_splits
755                .iter()
756                .map(|(&actor_id, splits)| {
757                    (
758                        actor_id,
759                        ConnectorSplits {
760                            splits: splits.clone().iter().map(ConnectorSplit::from).collect(),
761                        },
762                    )
763                })
764                .collect::<HashMap<_, _>>()
765        };
766
767        match self {
768            Mutation::Stop(StopMutation {
769                dropped_actors,
770                dropped_sink_fragments,
771            }) => PbMutation::Stop(PbStopMutation {
772                actors: dropped_actors.iter().copied().collect(),
773                dropped_sink_fragments: dropped_sink_fragments.iter().copied().collect(),
774            }),
775            Mutation::Update(UpdateMutation {
776                dispatchers,
777                merges,
778                vnode_bitmaps,
779                dropped_actors,
780                actor_splits,
781                actor_new_dispatchers,
782                actor_cdc_table_snapshot_splits,
783                sink_schema_change,
784                subscriptions_to_drop,
785            }) => PbMutation::Update(PbUpdateMutation {
786                dispatcher_update: dispatchers.values().flatten().cloned().collect(),
787                merge_update: merges.values().cloned().collect(),
788                actor_vnode_bitmap_update: vnode_bitmaps
789                    .iter()
790                    .map(|(&actor_id, bitmap)| (actor_id, bitmap.to_protobuf()))
791                    .collect(),
792                dropped_actors: dropped_actors.iter().copied().collect(),
793                actor_splits: actor_splits_to_protobuf(actor_splits),
794                actor_new_dispatchers: actor_new_dispatchers
795                    .iter()
796                    .map(|(&actor_id, dispatchers)| {
797                        (
798                            actor_id,
799                            PbDispatchers {
800                                dispatchers: dispatchers.clone(),
801                            },
802                        )
803                    })
804                    .collect(),
805                actor_cdc_table_snapshot_splits: Some(PbCdcTableSnapshotSplitsWithGeneration {
806                    splits:actor_cdc_table_snapshot_splits.splits.iter().map(|(actor_id,(splits, generation))| {
807                        (*actor_id, risingwave_pb::source::PbCdcTableSnapshotSplits {
808                            splits: splits.iter().map(risingwave_connector::source::cdc::build_cdc_table_snapshot_split).collect(),
809                            generation: *generation,
810                        })
811                    }).collect()
812                }),
813                sink_schema_change: sink_schema_change
814                    .iter()
815                    .map(|(sink_id, change)| ((*sink_id).as_raw_id(), change.clone()))
816                    .collect(),
817                subscriptions_to_drop: subscriptions_to_drop.clone(),
818            }),
819            Mutation::Add(AddMutation {
820                adds,
821                added_actors,
822                splits,
823                pause,
824                subscriptions_to_add,
825                backfill_nodes_to_pause,
826                actor_cdc_table_snapshot_splits,
827                new_upstream_sinks,
828            }) => PbMutation::Add(PbAddMutation {
829                actor_dispatchers: adds
830                    .iter()
831                    .map(|(&actor_id, dispatchers)| {
832                        (
833                            actor_id,
834                            PbDispatchers {
835                                dispatchers: dispatchers.clone(),
836                            },
837                        )
838                    })
839                    .collect(),
840                added_actors: added_actors.iter().copied().collect(),
841                actor_splits: actor_splits_to_protobuf(splits),
842                pause: *pause,
843                subscriptions_to_add: subscriptions_to_add
844                    .iter()
845                    .map(|(table_id, subscriber_id)| SubscriptionUpstreamInfo {
846                        subscriber_id: *subscriber_id,
847                        upstream_mv_table_id: *table_id,
848                    })
849                    .collect(),
850                backfill_nodes_to_pause: backfill_nodes_to_pause.iter().copied().collect(),
851                actor_cdc_table_snapshot_splits:
852                Some(PbCdcTableSnapshotSplitsWithGeneration {
853                    splits:actor_cdc_table_snapshot_splits.splits.iter().map(|(actor_id,(splits, generation))| {
854                        (*actor_id, risingwave_pb::source::PbCdcTableSnapshotSplits {
855                            splits: splits.iter().map(risingwave_connector::source::cdc::build_cdc_table_snapshot_split).collect(),
856                            generation: *generation,
857                        })
858                    }).collect()
859                }),
860                new_upstream_sinks: new_upstream_sinks
861                    .iter()
862                    .map(|(k, v)| (*k, v.clone()))
863                    .collect(),
864            }),
865            Mutation::SourceChangeSplit(changes) => {
866                PbMutation::Splits(PbSourceChangeSplitMutation {
867                    actor_splits: changes
868                        .iter()
869                        .map(|(&actor_id, splits)| {
870                            (
871                                actor_id,
872                                ConnectorSplits {
873                                    splits: splits
874                                        .clone()
875                                        .iter()
876                                        .map(ConnectorSplit::from)
877                                        .collect(),
878                                },
879                            )
880                        })
881                        .collect(),
882                })
883            }
884            Mutation::Pause => PbMutation::Pause(PbPauseMutation {}),
885            Mutation::Resume => PbMutation::Resume(PbResumeMutation {}),
886            Mutation::Throttle (changes) => PbMutation::Throttle(PbThrottleMutation {
887                fragment_throttle: changes.clone(),
888            }),
889            Mutation::DropSubscriptions {
890                subscriptions_to_drop,
891            } => PbMutation::DropSubscriptions(PbDropSubscriptionsMutation {
892                info: subscriptions_to_drop.clone(),
893            }),
894            Mutation::ConnectorPropsChange(map) => {
895                PbMutation::ConnectorPropsChange(PbConnectorPropsChangeMutation {
896                    connector_props_infos: map
897                        .iter()
898                        .map(|(actor_id, options)| {
899                            (
900                                *actor_id,
901                                ConnectorPropsInfo {
902                                    connector_props_info: options
903                                        .iter()
904                                        .map(|(k, v)| (k.clone(), v.clone()))
905                                        .collect(),
906                                },
907                            )
908                        })
909                        .collect(),
910                })
911            }
912            Mutation::StartFragmentBackfill { fragment_ids } => {
913                PbMutation::StartFragmentBackfill(PbStartFragmentBackfillMutation {
914                    fragment_ids: fragment_ids.iter().copied().collect(),
915                })
916            }
917            Mutation::RefreshStart {
918                table_id,
919                associated_source_id,
920            } => PbMutation::RefreshStart(risingwave_pb::stream_plan::RefreshStartMutation {
921                table_id: *table_id,
922                associated_source_id: *associated_source_id,
923            }),
924            Mutation::ListFinish {
925                associated_source_id,
926            } => PbMutation::ListFinish(risingwave_pb::stream_plan::ListFinishMutation {
927                associated_source_id: *associated_source_id,
928            }),
929            Mutation::LoadFinish {
930                associated_source_id,
931            } => PbMutation::LoadFinish(risingwave_pb::stream_plan::LoadFinishMutation {
932                associated_source_id: *associated_source_id,
933            }),
934            Mutation::ResetSource { source_id } => {
935                PbMutation::ResetSource(risingwave_pb::stream_plan::ResetSourceMutation {
936                    source_id: source_id.as_raw_id(),
937                })
938            }
939            Mutation::InjectSourceOffsets {
940                source_id,
941                split_offsets,
942            } => PbMutation::InjectSourceOffsets(
943                risingwave_pb::stream_plan::InjectSourceOffsetsMutation {
944                    source_id: source_id.as_raw_id(),
945                    split_offsets: split_offsets.clone(),
946                },
947            ),
948        }
949    }
950
951    fn from_protobuf(prost: &PbMutation) -> StreamExecutorResult<Self> {
952        let mutation = match prost {
953            PbMutation::Stop(stop) => Mutation::Stop(StopMutation {
954                dropped_actors: stop.actors.iter().copied().collect(),
955                dropped_sink_fragments: stop.dropped_sink_fragments.iter().copied().collect(),
956            }),
957
958            PbMutation::Update(update) => Mutation::Update(UpdateMutation {
959                dispatchers: update
960                    .dispatcher_update
961                    .iter()
962                    .map(|u| (u.actor_id, u.clone()))
963                    .into_group_map(),
964                merges: update
965                    .merge_update
966                    .iter()
967                    .map(|u| ((u.actor_id, u.upstream_fragment_id), u.clone()))
968                    .collect(),
969                vnode_bitmaps: update
970                    .actor_vnode_bitmap_update
971                    .iter()
972                    .map(|(&actor_id, bitmap)| (actor_id, Arc::new(bitmap.into())))
973                    .collect(),
974                dropped_actors: update.dropped_actors.iter().copied().collect(),
975                actor_splits: update
976                    .actor_splits
977                    .iter()
978                    .map(|(&actor_id, splits)| {
979                        (
980                            actor_id,
981                            splits
982                                .splits
983                                .iter()
984                                .map(|split| split.try_into().unwrap())
985                                .collect(),
986                        )
987                    })
988                    .collect(),
989                actor_new_dispatchers: update
990                    .actor_new_dispatchers
991                    .iter()
992                    .map(|(&actor_id, dispatchers)| (actor_id, dispatchers.dispatchers.clone()))
993                    .collect(),
994                actor_cdc_table_snapshot_splits:
995                    build_actor_cdc_table_snapshot_splits_with_generation(
996                        update
997                            .actor_cdc_table_snapshot_splits
998                            .clone()
999                            .unwrap_or_default(),
1000                    ),
1001                sink_schema_change: update
1002                    .sink_schema_change
1003                    .iter()
1004                    .map(|(sink_id, change)| (SinkId::from(*sink_id), change.clone()))
1005                    .collect(),
1006                subscriptions_to_drop: update.subscriptions_to_drop.clone(),
1007            }),
1008
1009            PbMutation::Add(add) => Mutation::Add(AddMutation {
1010                adds: add
1011                    .actor_dispatchers
1012                    .iter()
1013                    .map(|(&actor_id, dispatchers)| (actor_id, dispatchers.dispatchers.clone()))
1014                    .collect(),
1015                added_actors: add.added_actors.iter().copied().collect(),
1016                // TODO: remove this and use `SourceChangesSplit` after we support multiple
1017                // mutations.
1018                splits: add
1019                    .actor_splits
1020                    .iter()
1021                    .map(|(&actor_id, splits)| {
1022                        (
1023                            actor_id,
1024                            splits
1025                                .splits
1026                                .iter()
1027                                .map(|split| split.try_into().unwrap())
1028                                .collect(),
1029                        )
1030                    })
1031                    .collect(),
1032                pause: add.pause,
1033                subscriptions_to_add: add
1034                    .subscriptions_to_add
1035                    .iter()
1036                    .map(
1037                        |SubscriptionUpstreamInfo {
1038                             subscriber_id,
1039                             upstream_mv_table_id,
1040                         }| { (*upstream_mv_table_id, *subscriber_id) },
1041                    )
1042                    .collect(),
1043                backfill_nodes_to_pause: add.backfill_nodes_to_pause.iter().copied().collect(),
1044                actor_cdc_table_snapshot_splits:
1045                    build_actor_cdc_table_snapshot_splits_with_generation(
1046                        add.actor_cdc_table_snapshot_splits
1047                            .clone()
1048                            .unwrap_or_default(),
1049                    ),
1050                new_upstream_sinks: add
1051                    .new_upstream_sinks
1052                    .iter()
1053                    .map(|(k, v)| (*k, v.clone()))
1054                    .collect(),
1055            }),
1056
1057            PbMutation::Splits(s) => {
1058                let mut change_splits: Vec<(ActorId, Vec<SplitImpl>)> =
1059                    Vec::with_capacity(s.actor_splits.len());
1060                for (&actor_id, splits) in &s.actor_splits {
1061                    if !splits.splits.is_empty() {
1062                        change_splits.push((
1063                            actor_id,
1064                            splits
1065                                .splits
1066                                .iter()
1067                                .map(SplitImpl::try_from)
1068                                .try_collect()?,
1069                        ));
1070                    }
1071                }
1072                Mutation::SourceChangeSplit(change_splits.into_iter().collect())
1073            }
1074            PbMutation::Pause(_) => Mutation::Pause,
1075            PbMutation::Resume(_) => Mutation::Resume,
1076            PbMutation::Throttle(changes) => Mutation::Throttle(changes.fragment_throttle.clone()),
1077            PbMutation::DropSubscriptions(drop) => Mutation::DropSubscriptions {
1078                subscriptions_to_drop: drop.info.clone(),
1079            },
1080            PbMutation::ConnectorPropsChange(alter_connector_props) => {
1081                Mutation::ConnectorPropsChange(
1082                    alter_connector_props
1083                        .connector_props_infos
1084                        .iter()
1085                        .map(|(connector_id, options)| {
1086                            (
1087                                *connector_id,
1088                                options
1089                                    .connector_props_info
1090                                    .iter()
1091                                    .map(|(k, v)| (k.clone(), v.clone()))
1092                                    .collect(),
1093                            )
1094                        })
1095                        .collect(),
1096                )
1097            }
1098            PbMutation::StartFragmentBackfill(start_fragment_backfill) => {
1099                Mutation::StartFragmentBackfill {
1100                    fragment_ids: start_fragment_backfill
1101                        .fragment_ids
1102                        .iter()
1103                        .copied()
1104                        .collect(),
1105                }
1106            }
1107            PbMutation::RefreshStart(refresh_start) => Mutation::RefreshStart {
1108                table_id: refresh_start.table_id,
1109                associated_source_id: refresh_start.associated_source_id,
1110            },
1111            PbMutation::ListFinish(list_finish) => Mutation::ListFinish {
1112                associated_source_id: list_finish.associated_source_id,
1113            },
1114            PbMutation::LoadFinish(load_finish) => Mutation::LoadFinish {
1115                associated_source_id: load_finish.associated_source_id,
1116            },
1117            PbMutation::ResetSource(reset_source) => Mutation::ResetSource {
1118                source_id: SourceId::from(reset_source.source_id),
1119            },
1120            PbMutation::InjectSourceOffsets(inject) => Mutation::InjectSourceOffsets {
1121                source_id: SourceId::from(inject.source_id),
1122                split_offsets: inject.split_offsets.clone(),
1123            },
1124        };
1125        Ok(mutation)
1126    }
1127}
1128
1129impl<M> BarrierInner<M> {
1130    fn to_protobuf_inner(&self, barrier_fn: impl FnOnce(&M) -> Option<PbMutation>) -> PbBarrier {
1131        let Self {
1132            epoch,
1133            mutation,
1134            kind,
1135            tracing_context,
1136            ..
1137        } = self;
1138
1139        PbBarrier {
1140            epoch: Some(PbEpoch {
1141                curr: epoch.curr,
1142                prev: epoch.prev,
1143            }),
1144            mutation: barrier_fn(mutation).map(|mutation| PbBarrierMutation {
1145                mutation: Some(mutation),
1146            }),
1147            tracing_context: tracing_context.to_protobuf(),
1148            kind: *kind as _,
1149        }
1150    }
1151
1152    fn from_protobuf_inner(
1153        prost: &PbBarrier,
1154        mutation_from_pb: impl FnOnce(Option<&PbMutation>) -> StreamExecutorResult<M>,
1155    ) -> StreamExecutorResult<Self> {
1156        let epoch = prost.get_epoch()?;
1157
1158        Ok(Self {
1159            kind: prost.kind(),
1160            epoch: EpochPair::new(epoch.curr, epoch.prev),
1161            mutation: mutation_from_pb(
1162                (prost.mutation.as_ref()).and_then(|mutation| mutation.mutation.as_ref()),
1163            )?,
1164            tracing_context: TracingContext::from_protobuf(&prost.tracing_context),
1165        })
1166    }
1167
1168    pub fn map_mutation<M2>(self, f: impl FnOnce(M) -> M2) -> BarrierInner<M2> {
1169        BarrierInner {
1170            epoch: self.epoch,
1171            mutation: f(self.mutation),
1172            kind: self.kind,
1173            tracing_context: self.tracing_context,
1174        }
1175    }
1176}
1177
1178impl DispatcherBarrier {
1179    pub fn to_protobuf(&self) -> PbBarrier {
1180        self.to_protobuf_inner(|_| None)
1181    }
1182}
1183
1184impl Barrier {
1185    #[cfg(test)]
1186    pub fn to_protobuf(&self) -> PbBarrier {
1187        self.to_protobuf_inner(|mutation| mutation.as_ref().map(|mutation| mutation.to_protobuf()))
1188    }
1189
1190    pub fn from_protobuf(prost: &PbBarrier) -> StreamExecutorResult<Self> {
1191        Self::from_protobuf_inner(prost, |mutation| {
1192            mutation
1193                .map(|m| Mutation::from_protobuf(m).map(Arc::new))
1194                .transpose()
1195        })
1196    }
1197}
1198
1199#[derive(Debug, PartialEq, Eq, Clone)]
1200pub struct Watermark {
1201    pub col_idx: usize,
1202    pub data_type: DataType,
1203    pub val: ScalarImpl,
1204}
1205
1206impl PartialOrd for Watermark {
1207    fn partial_cmp(&self, other: &Self) -> Option<std::cmp::Ordering> {
1208        Some(self.cmp(other))
1209    }
1210}
1211
1212impl Ord for Watermark {
1213    fn cmp(&self, other: &Self) -> std::cmp::Ordering {
1214        self.val.default_cmp(&other.val)
1215    }
1216}
1217
1218impl Watermark {
1219    pub fn new(col_idx: usize, data_type: DataType, val: ScalarImpl) -> Self {
1220        Self {
1221            col_idx,
1222            data_type,
1223            val,
1224        }
1225    }
1226
1227    pub async fn transform_with_expr(
1228        self,
1229        expr: &NonStrictExpression<impl Expression>,
1230        new_col_idx: usize,
1231    ) -> Option<Self> {
1232        let Self { col_idx, val, .. } = self;
1233        let row = {
1234            let mut row = vec![None; col_idx + 1];
1235            row[col_idx] = Some(val);
1236            OwnedRow::new(row)
1237        };
1238        let val = expr.eval_row_infallible(&row).await?;
1239        Some(Self::new(new_col_idx, expr.inner().return_type(), val))
1240    }
1241
1242    /// Transform the watermark with the given output indices. If this watermark is not in the
1243    /// output, return `None`.
1244    pub fn transform_with_indices(self, output_indices: &[usize]) -> Option<Self> {
1245        output_indices
1246            .iter()
1247            .position(|p| *p == self.col_idx)
1248            .map(|new_col_idx| self.with_idx(new_col_idx))
1249    }
1250
1251    pub fn to_protobuf(&self) -> PbWatermark {
1252        PbWatermark {
1253            column: Some(PbInputRef {
1254                index: self.col_idx as _,
1255                r#type: Some(self.data_type.to_protobuf()),
1256            }),
1257            val: Some(&self.val).to_protobuf().into(),
1258        }
1259    }
1260
1261    pub fn from_protobuf(prost: &PbWatermark) -> StreamExecutorResult<Self> {
1262        let col_ref = prost.get_column()?;
1263        let data_type = DataType::from(col_ref.get_type()?);
1264        let val = Datum::from_protobuf(prost.get_val()?, &data_type)?
1265            .expect("watermark value cannot be null");
1266        Ok(Self::new(col_ref.get_index() as _, data_type, val))
1267    }
1268
1269    pub fn with_idx(self, idx: usize) -> Self {
1270        Self::new(idx, self.data_type, self.val)
1271    }
1272}
1273
1274#[cfg_attr(any(test, feature = "test"), derive(PartialEq))]
1275#[derive(Debug, EnumAsInner, Clone)]
1276pub enum MessageInner<M> {
1277    Chunk(StreamChunk),
1278    Barrier(BarrierInner<M>),
1279    Watermark(Watermark),
1280}
1281
1282impl<M> MessageInner<M> {
1283    pub fn map_mutation<M2>(self, f: impl FnOnce(M) -> M2) -> MessageInner<M2> {
1284        match self {
1285            MessageInner::Chunk(chunk) => MessageInner::Chunk(chunk),
1286            MessageInner::Barrier(barrier) => MessageInner::Barrier(barrier.map_mutation(f)),
1287            MessageInner::Watermark(watermark) => MessageInner::Watermark(watermark),
1288        }
1289    }
1290}
1291
1292pub type Message = MessageInner<BarrierMutationType>;
1293pub type DispatcherMessage = MessageInner<()>;
1294
1295/// `MessageBatchInner` is used exclusively by `Dispatcher` and the `Merger`/`Receiver` for exchanging messages between them.
1296/// It shares the same message type as the fundamental `MessageInner`, but batches multiple barriers into a single message.
1297#[derive(Debug, EnumAsInner, Clone)]
1298pub enum MessageBatchInner<M> {
1299    Chunk(StreamChunk),
1300    BarrierBatch(Vec<BarrierInner<M>>),
1301    Watermark(Watermark),
1302}
1303pub type MessageBatch = MessageBatchInner<BarrierMutationType>;
1304pub type DispatcherBarriers = Vec<DispatcherBarrier>;
1305pub type DispatcherMessageBatch = MessageBatchInner<()>;
1306
1307impl From<DispatcherMessage> for DispatcherMessageBatch {
1308    fn from(m: DispatcherMessage) -> Self {
1309        match m {
1310            DispatcherMessage::Chunk(c) => Self::Chunk(c),
1311            DispatcherMessage::Barrier(b) => Self::BarrierBatch(vec![b]),
1312            DispatcherMessage::Watermark(w) => Self::Watermark(w),
1313        }
1314    }
1315}
1316
1317impl From<StreamChunk> for Message {
1318    fn from(chunk: StreamChunk) -> Self {
1319        Message::Chunk(chunk)
1320    }
1321}
1322
1323impl<'a> TryFrom<&'a Message> for &'a Barrier {
1324    type Error = ();
1325
1326    fn try_from(m: &'a Message) -> std::result::Result<Self, Self::Error> {
1327        match m {
1328            Message::Chunk(_) => Err(()),
1329            Message::Barrier(b) => Ok(b),
1330            Message::Watermark(_) => Err(()),
1331        }
1332    }
1333}
1334
1335impl Message {
1336    /// Return true if the message is a stop barrier, meaning the stream
1337    /// will not continue, false otherwise.
1338    ///
1339    /// Note that this does not mean we will stop the current actor.
1340    #[cfg(test)]
1341    pub fn is_stop(&self) -> bool {
1342        matches!(
1343            self,
1344            Message::Barrier(Barrier {
1345                mutation,
1346                ..
1347            }) if mutation.as_ref().unwrap().is_stop()
1348        )
1349    }
1350}
1351
1352impl DispatcherMessageBatch {
1353    pub fn to_protobuf(&self) -> PbStreamMessageBatch {
1354        let prost = match self {
1355            Self::Chunk(stream_chunk) => {
1356                let prost_stream_chunk = stream_chunk.to_protobuf();
1357                StreamMessageBatch::StreamChunk(prost_stream_chunk)
1358            }
1359            Self::BarrierBatch(barrier_batch) => StreamMessageBatch::BarrierBatch(BarrierBatch {
1360                barriers: barrier_batch.iter().map(|b| b.to_protobuf()).collect(),
1361            }),
1362            Self::Watermark(watermark) => StreamMessageBatch::Watermark(watermark.to_protobuf()),
1363        };
1364        PbStreamMessageBatch {
1365            stream_message_batch: Some(prost),
1366        }
1367    }
1368
1369    pub fn from_protobuf(prost: &PbStreamMessageBatch) -> StreamExecutorResult<Self> {
1370        let res = match prost.get_stream_message_batch()? {
1371            StreamMessageBatch::StreamChunk(chunk) => {
1372                Self::Chunk(StreamChunk::from_protobuf(chunk)?)
1373            }
1374            StreamMessageBatch::BarrierBatch(barrier_batch) => {
1375                let barriers = barrier_batch
1376                    .barriers
1377                    .iter()
1378                    .map(|barrier| {
1379                        DispatcherBarrier::from_protobuf_inner(barrier, |mutation| {
1380                            if mutation.is_some() {
1381                                if cfg!(debug_assertions) {
1382                                    panic!("should not receive message of barrier with mutation");
1383                                } else {
1384                                    warn!(?barrier, "receive message of barrier with mutation");
1385                                }
1386                            }
1387                            Ok(())
1388                        })
1389                    })
1390                    .try_collect()?;
1391                Self::BarrierBatch(barriers)
1392            }
1393            StreamMessageBatch::Watermark(watermark) => {
1394                Self::Watermark(Watermark::from_protobuf(watermark)?)
1395            }
1396        };
1397        Ok(res)
1398    }
1399
1400    pub fn get_encoded_len(msg: &impl ::prost::Message) -> usize {
1401        ::prost::Message::encoded_len(msg)
1402    }
1403}
1404
1405pub type StreamKey = Vec<usize>;
1406pub type StreamKeyRef<'a> = &'a [usize];
1407pub type StreamKeyDataTypes = SmallVec<[DataType; 1]>;
1408
1409/// Expect the first message of the given `stream` as a barrier.
1410pub async fn expect_first_barrier<M: Debug>(
1411    stream: &mut (impl MessageStreamInner<M> + Unpin),
1412) -> StreamExecutorResult<BarrierInner<M>> {
1413    let message = stream
1414        .next()
1415        .instrument_await("expect_first_barrier")
1416        .await
1417        .context("failed to extract the first message: stream closed unexpectedly")??;
1418    let barrier = message
1419        .into_barrier()
1420        .expect("the first message must be a barrier");
1421    // TODO: Is this check correct?
1422    assert!(matches!(
1423        barrier.kind,
1424        BarrierKind::Checkpoint | BarrierKind::Initial
1425    ));
1426    Ok(barrier)
1427}
1428
1429/// Expect the first message of the given `stream` as a barrier.
1430pub async fn expect_first_barrier_from_aligned_stream(
1431    stream: &mut (impl AlignedMessageStream + Unpin),
1432) -> StreamExecutorResult<Barrier> {
1433    let message = stream
1434        .next()
1435        .instrument_await("expect_first_barrier")
1436        .await
1437        .context("failed to extract the first message: stream closed unexpectedly")??;
1438    let barrier = message
1439        .into_barrier()
1440        .expect("the first message must be a barrier");
1441    Ok(barrier)
1442}
1443
1444/// `StreamConsumer` is the last step in an actor.
1445pub trait StreamConsumer: Send + 'static {
1446    type BarrierStream: Stream<Item = StreamResult<Barrier>> + Send;
1447
1448    fn execute(self: Box<Self>) -> Self::BarrierStream;
1449}
1450
1451type BoxedMessageInput<InputId, M> = BoxedInput<InputId, MessageStreamItemInner<M>>;
1452
1453/// A stream for merging messages from multiple upstreams.
1454/// Can dynamically add and delete upstream streams.
1455/// For the meaning of the generic parameter `M` used, refer to `BarrierInner<M>`.
1456pub struct DynamicReceivers<InputId, M> {
1457    /// The barrier we're aligning to. If this is `None`, then `blocked_upstreams` is empty.
1458    barrier: Option<BarrierInner<M>>,
1459    /// The start timestamp of the current barrier. Used for measuring the alignment duration.
1460    start_ts: Option<Instant>,
1461    /// The upstreams that're blocked by the `barrier`.
1462    blocked: Vec<BoxedMessageInput<InputId, M>>,
1463    /// The upstreams that're not blocked and can be polled.
1464    active: FuturesUnordered<StreamFuture<BoxedMessageInput<InputId, M>>>,
1465    /// watermark column index -> `BufferedWatermarks`
1466    buffered_watermarks: BTreeMap<usize, BufferedWatermarks<InputId>>,
1467    /// Currently only used for union.
1468    barrier_align_duration: Option<LabelGuardedMetric<GenericCounter<AtomicU64>>>,
1469    /// Only for merge. If None, then we don't take `Instant::now()` and `observe` during `poll_next`
1470    merge_barrier_align_duration: Option<LabelGuardedMetric<GenericCounter<AtomicU64>>>,
1471}
1472
1473impl<InputId: Clone + Ord + Hash + std::fmt::Debug + Unpin, M: Clone + Unpin> Stream
1474    for DynamicReceivers<InputId, M>
1475{
1476    type Item = MessageStreamItemInner<M>;
1477
1478    fn poll_next(
1479        mut self: Pin<&mut Self>,
1480        cx: &mut std::task::Context<'_>,
1481    ) -> Poll<Option<Self::Item>> {
1482        if self.is_empty() {
1483            return Poll::Ready(None);
1484        }
1485
1486        loop {
1487            match futures::ready!(self.active.poll_next_unpin(cx)) {
1488                // Directly forward the error.
1489                Some((Some(Err(e)), _)) => {
1490                    return Poll::Ready(Some(Err(e)));
1491                }
1492                // Handle the message from some upstream.
1493                Some((Some(Ok(message)), remaining)) => {
1494                    let input_id = remaining.id();
1495                    match message {
1496                        MessageInner::Chunk(chunk) => {
1497                            // Continue polling this upstream by pushing it back to `active`.
1498                            self.active.push(remaining.into_future());
1499                            return Poll::Ready(Some(Ok(MessageInner::Chunk(chunk))));
1500                        }
1501                        MessageInner::Watermark(watermark) => {
1502                            // Continue polling this upstream by pushing it back to `active`.
1503                            self.active.push(remaining.into_future());
1504                            if let Some(watermark) = self.handle_watermark(input_id, watermark) {
1505                                return Poll::Ready(Some(Ok(MessageInner::Watermark(watermark))));
1506                            }
1507                        }
1508                        MessageInner::Barrier(barrier) => {
1509                            // Block this upstream by pushing it to `blocked`.
1510                            if self.blocked.is_empty() {
1511                                self.start_ts = Some(Instant::now());
1512                            }
1513                            self.blocked.push(remaining);
1514                            if let Some(current_barrier) = self.barrier.as_ref() {
1515                                if current_barrier.epoch != barrier.epoch {
1516                                    return Poll::Ready(Some(Err(
1517                                        StreamExecutorError::align_barrier(
1518                                            current_barrier.clone().map_mutation(|_| None),
1519                                            barrier.map_mutation(|_| None),
1520                                        ),
1521                                    )));
1522                                }
1523                            } else {
1524                                self.barrier = Some(barrier);
1525                            }
1526                        }
1527                    }
1528                }
1529                // We use barrier as the control message of the stream. That is, we always stop the
1530                // actors actively when we receive a `Stop` mutation, instead of relying on the stream
1531                // termination.
1532                //
1533                // Besides, in abnormal cases when the other side of the `Input` closes unexpectedly,
1534                // we also yield an `Err(ExchangeChannelClosed)`, which will hit the `Err` arm above.
1535                // So this branch will never be reached in all cases.
1536                Some((None, remaining)) => {
1537                    return Poll::Ready(Some(Err(StreamExecutorError::channel_closed(format!(
1538                        "upstream input {:?} unexpectedly closed",
1539                        remaining.id()
1540                    )))));
1541                }
1542                // There's no active upstreams. Process the barrier and resume the blocked ones.
1543                None => {
1544                    assert!(!self.blocked.is_empty());
1545
1546                    let start_ts = self
1547                        .start_ts
1548                        .take()
1549                        .expect("should have received at least one barrier");
1550                    if let Some(barrier_align_duration) = &self.barrier_align_duration {
1551                        barrier_align_duration.inc_by(start_ts.elapsed().as_nanos() as u64);
1552                    }
1553                    if let Some(merge_barrier_align_duration) = &self.merge_barrier_align_duration {
1554                        merge_barrier_align_duration.inc_by(start_ts.elapsed().as_nanos() as u64);
1555                    }
1556
1557                    break;
1558                }
1559            }
1560        }
1561
1562        assert!(self.active.is_terminated());
1563
1564        let barrier = self.barrier.take().unwrap();
1565
1566        let upstreams = std::mem::take(&mut self.blocked);
1567        self.extend_active(upstreams);
1568        assert!(!self.active.is_terminated());
1569
1570        Poll::Ready(Some(Ok(MessageInner::Barrier(barrier))))
1571    }
1572}
1573
1574impl<InputId: Clone + Ord + Hash + std::fmt::Debug, M> DynamicReceivers<InputId, M> {
1575    pub fn new(
1576        upstreams: Vec<BoxedMessageInput<InputId, M>>,
1577        barrier_align_duration: Option<LabelGuardedMetric<GenericCounter<AtomicU64>>>,
1578        merge_barrier_align_duration: Option<LabelGuardedMetric<GenericCounter<AtomicU64>>>,
1579    ) -> Self {
1580        let mut this = Self {
1581            barrier: None,
1582            start_ts: None,
1583            blocked: Vec::with_capacity(upstreams.len()),
1584            active: Default::default(),
1585            buffered_watermarks: Default::default(),
1586            merge_barrier_align_duration,
1587            barrier_align_duration,
1588        };
1589        this.extend_active(upstreams);
1590        this
1591    }
1592
1593    /// Extend the active upstreams with the given upstreams. The current stream must be at the
1594    /// clean state right after a barrier.
1595    pub fn extend_active(
1596        &mut self,
1597        upstreams: impl IntoIterator<Item = BoxedMessageInput<InputId, M>>,
1598    ) {
1599        assert!(self.blocked.is_empty() && self.barrier.is_none());
1600
1601        self.active
1602            .extend(upstreams.into_iter().map(|s| s.into_future()));
1603    }
1604
1605    /// Handle a new watermark message. Optionally returns the watermark message to emit.
1606    pub fn handle_watermark(
1607        &mut self,
1608        input_id: InputId,
1609        watermark: Watermark,
1610    ) -> Option<Watermark> {
1611        let col_idx = watermark.col_idx;
1612        // Insert a buffer watermarks when first received from a column.
1613        let upstream_ids: Vec<_> = self.upstream_input_ids().collect();
1614        let watermarks = self
1615            .buffered_watermarks
1616            .entry(col_idx)
1617            .or_insert_with(|| BufferedWatermarks::with_ids(upstream_ids));
1618        watermarks.handle_watermark(input_id, watermark)
1619    }
1620
1621    /// Consume `other` and add its upstreams to `self`. The two streams must be at the clean state
1622    /// right after a barrier.
1623    pub fn add_upstreams_from(
1624        &mut self,
1625        new_inputs: impl IntoIterator<Item = BoxedMessageInput<InputId, M>>,
1626    ) {
1627        assert!(self.blocked.is_empty() && self.barrier.is_none());
1628
1629        let new_inputs: Vec<_> = new_inputs.into_iter().collect();
1630        let input_ids = new_inputs.iter().map(|input| input.id());
1631        self.buffered_watermarks.values_mut().for_each(|buffers| {
1632            // Add buffers to the buffered watermarks for all cols
1633            buffers.add_buffers(input_ids.clone());
1634        });
1635        self.active
1636            .extend(new_inputs.into_iter().map(|s| s.into_future()));
1637    }
1638
1639    /// Remove upstreams from `self` in `upstream_input_ids`. The current stream must be at the
1640    /// clean state right after a barrier.
1641    /// The current container does not necessarily contain all the input ids passed in.
1642    pub fn remove_upstreams(&mut self, upstream_input_ids: &HashSet<InputId>) {
1643        assert!(self.blocked.is_empty() && self.barrier.is_none());
1644
1645        let new_upstreams = std::mem::take(&mut self.active)
1646            .into_iter()
1647            .map(|s| s.into_inner().unwrap())
1648            .filter(|u| !upstream_input_ids.contains(&u.id()));
1649        self.extend_active(new_upstreams);
1650        self.buffered_watermarks.values_mut().for_each(|buffers| {
1651            // Call `check_heap` in case the only upstream(s) that does not have
1652            // watermark in heap is removed
1653            buffers.remove_buffer(upstream_input_ids.clone());
1654        });
1655    }
1656
1657    pub fn merge_barrier_align_duration(
1658        &self,
1659    ) -> Option<LabelGuardedMetric<GenericCounter<AtomicU64>>> {
1660        self.merge_barrier_align_duration.clone()
1661    }
1662
1663    pub fn flush_buffered_watermarks(&mut self) {
1664        self.buffered_watermarks
1665            .values_mut()
1666            .for_each(|buffers| buffers.clear());
1667    }
1668
1669    pub fn upstream_input_ids(&self) -> impl Iterator<Item = InputId> + '_ {
1670        self.blocked
1671            .iter()
1672            .map(|s| s.id())
1673            .chain(self.active.iter().map(|s| s.get_ref().unwrap().id()))
1674    }
1675
1676    pub fn is_empty(&self) -> bool {
1677        self.blocked.is_empty() && self.active.is_empty()
1678    }
1679}
1680
1681// Explanation of why we need `DispatchBarrierBuffer`:
1682//
1683// When we need to create or replace an upstream fragment for the current fragment, the `Merge` operator must
1684// add some new upstream actor inputs. However, the `Merge` operator may still have old upstreams. We must wait
1685// for these old upstreams to completely process their barriers and align before we can safely update the
1686// `upstream-input-set`.
1687//
1688// Meanwhile, the creation of a new upstream actor can only succeed after the channel to the downstream `Merge`
1689// operator has been established. This creates a potential dependency chain: [new_actor_creation ->
1690// downstream_merge_update -> old_actor_processing]
1691//
1692// To address this, we split the application of a barrier's `Mutation` into two steps:
1693// 1. Parse the `Mutation`. If there is an addition on the upstream-set, establish a channel with the upstream
1694//    and cache it.
1695// 2. When the upstream barrier actually arrives, apply the cached upstream changes to the upstream-set
1696//
1697// Additionally, since receiving a barrier from current upstream input and from the `barrier_rx` are
1698// asynchronous, we cannot determine which will arrive first. Therefore, when a barrier is received from an
1699// upstream: if a cached mutation is present, we apply it. Otherwise, we must fetch a new barrier from
1700// `barrier_rx`.
1701pub(crate) struct DispatchBarrierBuffer {
1702    buffer: VecDeque<(Barrier, Option<Vec<BoxedActorInput>>)>,
1703    barrier_rx: mpsc::UnboundedReceiver<Barrier>,
1704    recv_state: BarrierReceiverState,
1705    curr_upstream_fragment_id: FragmentId,
1706    actor_id: ActorId,
1707    // read-only context for building new inputs
1708    build_input_ctx: Arc<BuildInputContext>,
1709}
1710
1711struct BuildInputContext {
1712    pub actor_id: ActorId,
1713    pub local_barrier_manager: LocalBarrierManager,
1714    pub metrics: Arc<StreamingMetrics>,
1715    pub fragment_id: FragmentId,
1716    pub actor_config: Arc<StreamingConfig>,
1717}
1718
1719type BoxedNewInputsFuture =
1720    Pin<Box<dyn Future<Output = StreamExecutorResult<Vec<BoxedActorInput>>> + Send>>;
1721
1722enum BarrierReceiverState {
1723    ReceivingBarrier,
1724    CreatingNewInput(Barrier, BoxedNewInputsFuture),
1725}
1726
1727impl DispatchBarrierBuffer {
1728    pub fn new(
1729        barrier_rx: mpsc::UnboundedReceiver<Barrier>,
1730        actor_id: ActorId,
1731        curr_upstream_fragment_id: FragmentId,
1732        local_barrier_manager: LocalBarrierManager,
1733        metrics: Arc<StreamingMetrics>,
1734        fragment_id: FragmentId,
1735        actor_config: Arc<StreamingConfig>,
1736    ) -> Self {
1737        Self {
1738            buffer: VecDeque::new(),
1739            barrier_rx,
1740            recv_state: BarrierReceiverState::ReceivingBarrier,
1741            curr_upstream_fragment_id,
1742            actor_id,
1743            build_input_ctx: Arc::new(BuildInputContext {
1744                actor_id,
1745                local_barrier_manager,
1746                metrics,
1747                fragment_id,
1748                actor_config,
1749            }),
1750        }
1751    }
1752
1753    pub async fn await_next_message(
1754        &mut self,
1755        stream: &mut (impl Stream<Item = StreamExecutorResult<DispatcherMessage>> + Unpin),
1756        metrics: &ActorInputMetrics,
1757    ) -> StreamExecutorResult<DispatcherMessage> {
1758        let mut start_time = Instant::now();
1759        let interval_duration = Duration::from_secs(15);
1760        let mut interval =
1761            tokio::time::interval_at(start_time + interval_duration, interval_duration);
1762
1763        loop {
1764            tokio::select! {
1765                biased;
1766                msg = stream.try_next() => {
1767                    metrics
1768                        .actor_input_buffer_blocking_duration_ns
1769                        .inc_by(start_time.elapsed().as_nanos() as u64);
1770                    return msg?.ok_or_else(
1771                        || StreamExecutorError::channel_closed("upstream executor closed unexpectedly")
1772                    );
1773                }
1774
1775                e = self.continuously_fetch_barrier_rx() => {
1776                    return Err(e);
1777                }
1778
1779                _ = interval.tick() => {
1780                    start_time = Instant::now();
1781                    metrics.actor_input_buffer_blocking_duration_ns.inc_by(interval_duration.as_nanos() as u64);
1782                    continue;
1783                }
1784            }
1785        }
1786    }
1787
1788    pub async fn pop_barrier_with_inputs(
1789        &mut self,
1790        barrier: DispatcherBarrier,
1791    ) -> StreamExecutorResult<(Barrier, Option<Vec<BoxedActorInput>>)> {
1792        while self.buffer.is_empty() {
1793            self.try_fetch_barrier_rx(false).await?;
1794        }
1795        let (recv_barrier, inputs) = self.buffer.pop_front().unwrap();
1796        assert_equal_dispatcher_barrier(&recv_barrier, &barrier);
1797
1798        Ok((recv_barrier, inputs))
1799    }
1800
1801    async fn continuously_fetch_barrier_rx(&mut self) -> StreamExecutorError {
1802        loop {
1803            if let Err(e) = self.try_fetch_barrier_rx(true).await {
1804                return e;
1805            }
1806        }
1807    }
1808
1809    async fn try_fetch_barrier_rx(&mut self, pending_on_end: bool) -> StreamExecutorResult<()> {
1810        match &mut self.recv_state {
1811            BarrierReceiverState::ReceivingBarrier => {
1812                let Some(barrier) = self.barrier_rx.recv().await else {
1813                    if pending_on_end {
1814                        return pending().await;
1815                    } else {
1816                        return Err(StreamExecutorError::channel_closed(
1817                            "barrier channel closed unexpectedly",
1818                        ));
1819                    }
1820                };
1821                if let Some(fut) = self.pre_apply_barrier(&barrier) {
1822                    self.recv_state = BarrierReceiverState::CreatingNewInput(barrier, fut);
1823                } else {
1824                    self.buffer.push_back((barrier, None));
1825                }
1826            }
1827            BarrierReceiverState::CreatingNewInput(barrier, fut) => {
1828                let new_inputs = fut.await?;
1829                self.buffer.push_back((barrier.clone(), Some(new_inputs)));
1830                self.recv_state = BarrierReceiverState::ReceivingBarrier;
1831            }
1832        }
1833        Ok(())
1834    }
1835
1836    fn pre_apply_barrier(&mut self, barrier: &Barrier) -> Option<BoxedNewInputsFuture> {
1837        if let Some(update) = barrier.as_update_merge(self.actor_id, self.curr_upstream_fragment_id)
1838            && !update.added_upstream_actors.is_empty()
1839        {
1840            // When update upstream fragment, added_actors will not be empty.
1841            let upstream_fragment_id =
1842                if let Some(new_upstream_fragment_id) = update.new_upstream_fragment_id {
1843                    self.curr_upstream_fragment_id = new_upstream_fragment_id;
1844                    new_upstream_fragment_id
1845                } else {
1846                    self.curr_upstream_fragment_id
1847                };
1848            let ctx = self.build_input_ctx.clone();
1849            let added_upstream_actors = update.added_upstream_actors.clone();
1850            let barrier = barrier.clone();
1851            let fut = async move {
1852                try_join_all(added_upstream_actors.iter().map(|upstream_actor| async {
1853                    let mut new_input = new_input(
1854                        &ctx.local_barrier_manager,
1855                        ctx.metrics.clone(),
1856                        ctx.actor_id,
1857                        ctx.fragment_id,
1858                        upstream_actor,
1859                        upstream_fragment_id,
1860                        ctx.actor_config.clone(),
1861                    )
1862                    .await?;
1863
1864                    // Poll the first barrier from the new upstreams. It must be the same as the one we polled from
1865                    // original upstreams.
1866                    let first_barrier = expect_first_barrier(&mut new_input).await?;
1867                    assert_equal_dispatcher_barrier(&barrier, &first_barrier);
1868
1869                    StreamExecutorResult::Ok(new_input)
1870                }))
1871                .await
1872            }
1873            .boxed();
1874
1875            Some(fut)
1876        } else {
1877            None
1878        }
1879    }
1880}