@@ -25,7 +25,7 @@ use super::error::StreamExecutorError;
25
25
use super :: {
26
26
expect_first_barrier, Barrier , BoxedExecutor , Executor , ExecutorInfo , Message , MessageStream ,
27
27
} ;
28
- use crate :: executor:: PkIndices ;
28
+ use crate :: executor:: { BoxedMessageStream , PkIndices , Watermark } ;
29
29
use crate :: task:: { ActorId , CreateMviewProgress } ;
30
30
31
31
/// `ChainExecutor` is an executor that enables synchronization between the existing stream and
@@ -49,35 +49,47 @@ pub struct RearrangedChainExecutor {
49
49
info : ExecutorInfo ,
50
50
}
51
51
52
- fn mapping ( upstream_indices : & [ usize ] , msg : Message ) -> Message {
52
+ fn mapping ( upstream_indices : & [ usize ] , msg : Message ) -> Option < Message > {
53
53
match msg {
54
- Message :: Watermark ( _ ) => {
55
- todo ! ( "https://github.com/risingwavelabs/risingwave/issues/6042" )
54
+ Message :: Watermark ( watermark ) => {
55
+ mapping_watermark ( watermark , upstream_indices ) . map ( Message :: Watermark )
56
56
}
57
-
58
57
Message :: Chunk ( chunk) => {
59
58
let ( ops, columns, visibility) = chunk. into_inner ( ) ;
60
59
let mapped_columns = upstream_indices
61
60
. iter ( )
62
61
. map ( |& i| columns[ i] . clone ( ) )
63
62
. collect ( ) ;
64
- Message :: Chunk ( StreamChunk :: new ( ops, mapped_columns, visibility) )
63
+ Some ( Message :: Chunk ( StreamChunk :: new (
64
+ ops,
65
+ mapped_columns,
66
+ visibility,
67
+ ) ) )
65
68
}
66
- _ => msg,
69
+ Message :: Barrier ( _ ) => Some ( msg) ,
67
70
}
68
71
}
69
72
73
+ fn mapping_watermark ( watermark : Watermark , upstream_indices : & [ usize ] ) -> Option < Watermark > {
74
+ upstream_indices
75
+ . iter ( )
76
+ . position ( |& idx| idx == watermark. col_idx )
77
+ . map ( |idx| watermark. with_idx ( idx) )
78
+ }
79
+
70
80
#[ derive( Debug ) ]
71
81
enum RearrangedMessage {
72
82
RearrangedBarrier ( Barrier ) ,
73
83
PhantomBarrier ( Barrier ) ,
74
84
Chunk ( StreamChunk ) ,
85
+ // This watermark is just a place holder.
86
+ Watermark ,
75
87
}
76
88
77
89
impl RearrangedMessage {
78
90
fn phantom_into ( self ) -> Option < Message > {
79
91
match self {
80
- RearrangedMessage :: RearrangedBarrier ( _) => None ,
92
+ RearrangedMessage :: RearrangedBarrier ( _) | RearrangedMessage :: Watermark => None ,
81
93
RearrangedMessage :: PhantomBarrier ( barrier) => Message :: Barrier ( barrier) . into ( ) ,
82
94
RearrangedMessage :: Chunk ( chunk) => Message :: Chunk ( chunk) . into ( ) ,
83
95
}
@@ -87,21 +99,15 @@ impl RearrangedMessage {
87
99
impl RearrangedMessage {
88
100
fn rearranged_from ( msg : Message ) -> Self {
89
101
match msg {
90
- Message :: Watermark ( _) => {
91
- todo ! ( "https://github.com/risingwavelabs/risingwave/issues/6042" )
92
- }
93
-
102
+ Message :: Watermark ( _) => RearrangedMessage :: Watermark ,
94
103
Message :: Chunk ( chunk) => RearrangedMessage :: Chunk ( chunk) ,
95
104
Message :: Barrier ( barrier) => RearrangedMessage :: RearrangedBarrier ( barrier) ,
96
105
}
97
106
}
98
107
99
108
fn phantom_from ( msg : Message ) -> Self {
100
109
match msg {
101
- Message :: Watermark ( _) => {
102
- todo ! ( "https://github.com/risingwavelabs/risingwave/issues/6042" )
103
- }
104
-
110
+ Message :: Watermark ( _) => RearrangedMessage :: Watermark ,
105
111
Message :: Chunk ( chunk) => RearrangedMessage :: Chunk ( chunk) ,
106
112
Message :: Barrier ( barrier) => RearrangedMessage :: PhantomBarrier ( barrier) ,
107
113
}
@@ -135,10 +141,11 @@ impl RearrangedChainExecutor {
135
141
async fn execute_inner ( mut self ) {
136
142
// 0. Project the upstream with `upstream_indices`.
137
143
let upstream_indices = self . upstream_indices . clone ( ) ;
138
- let mut upstream = self
139
- . upstream
140
- . execute ( )
141
- . map ( move |result| result. map ( |msg| mapping ( & upstream_indices, msg) ) ) ;
144
+
145
+ let mut upstream = Box :: pin ( Self :: mapping_stream (
146
+ self . upstream . execute ( ) ,
147
+ & upstream_indices,
148
+ ) ) ;
142
149
143
150
// 1. Poll the upstream to get the first barrier.
144
151
let first_barrier = expect_first_barrier ( & mut upstream) . await ?;
@@ -219,6 +226,9 @@ impl RearrangedChainExecutor {
219
226
yield Message :: Barrier ( barrier) ;
220
227
}
221
228
RearrangedMessage :: Chunk ( chunk) => yield Message :: Chunk ( chunk) ,
229
+ RearrangedMessage :: Watermark => {
230
+ // Ignore watermark during snapshot consumption.
231
+ }
222
232
}
223
233
}
224
234
@@ -310,6 +320,18 @@ impl RearrangedChainExecutor {
310
320
}
311
321
}
312
322
}
323
+
324
+ #[ expect( clippy:: needless_lifetimes, reason = "code generated by try_stream" ) ]
325
+ #[ try_stream( ok = Message , error = StreamExecutorError ) ]
326
+ async fn mapping_stream ( stream : BoxedMessageStream , upstream_indices : & [ usize ] ) {
327
+ #[ for_await]
328
+ for msg in stream {
329
+ match mapping ( upstream_indices, msg?) {
330
+ Some ( msg) => yield msg,
331
+ None => continue ,
332
+ }
333
+ }
334
+ }
313
335
}
314
336
315
337
impl Executor for RearrangedChainExecutor {
0 commit comments