risingwave_stream/executor/
hop_window.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::num::NonZeroUsize;

use itertools::Itertools;
use risingwave_common::array::{DataChunk, Op};
use risingwave_common::types::Interval;
use risingwave_expr::expr::NonStrictExpression;
use risingwave_expr::ExprError;

use crate::executor::prelude::*;

pub struct HopWindowExecutor {
    _ctx: ActorContextRef,
    pub input: Executor,
    pub time_col_idx: usize,
    pub window_slide: Interval,
    pub window_size: Interval,
    window_start_exprs: Vec<NonStrictExpression>,
    window_end_exprs: Vec<NonStrictExpression>,
    pub output_indices: Vec<usize>,
    chunk_size: usize,
}

impl HopWindowExecutor {
    #[allow(clippy::too_many_arguments)]
    pub fn new(
        ctx: ActorContextRef,
        input: Executor,
        time_col_idx: usize,
        window_slide: Interval,
        window_size: Interval,
        window_start_exprs: Vec<NonStrictExpression>,
        window_end_exprs: Vec<NonStrictExpression>,
        output_indices: Vec<usize>,
        chunk_size: usize,
    ) -> Self {
        HopWindowExecutor {
            _ctx: ctx,
            input,
            time_col_idx,
            window_slide,
            window_size,
            window_start_exprs,
            window_end_exprs,
            output_indices,
            chunk_size,
        }
    }
}

impl Execute for HopWindowExecutor {
    fn execute(self: Box<Self>) -> super::BoxedMessageStream {
        self.execute_inner().boxed()
    }
}

impl HopWindowExecutor {
    #[try_stream(ok = Message, error = StreamExecutorError)]
    async fn execute_inner(self: Box<Self>) {
        let Self {
            input,

            window_slide,
            window_size,
            output_indices,
            time_col_idx,

            chunk_size,
            ..
        } = *self;
        let units = window_size
            .exact_div(&window_slide)
            .and_then(|x| NonZeroUsize::new(usize::try_from(x).ok()?))
            .ok_or_else(|| ExprError::InvalidParam {
                name: "window",
                reason: format!(
                    "window_size {} cannot be divided by window_slide {}",
                    window_size, window_slide
                )
                .into(),
            })?
            .get();

        // The following indices are the output indices as if the downstream needs all input + hop
        // window columns.
        let logical_window_start_col_idx = input.schema().len();
        let logical_window_end_col_idx = input.schema().len() + 1;

        // The following indices are the real output column indices. `None` means we don't need to
        // output that column.
        let out_window_start_col_idx = output_indices
            .iter()
            .position(|&idx| idx == logical_window_start_col_idx);
        let out_window_end_col_idx = output_indices
            .iter()
            .position(|&idx| idx == logical_window_end_col_idx);

        #[for_await]
        for msg in input.execute() {
            let msg = msg?;
            match msg {
                Message::Chunk(chunk) => {
                    if units == 0 {
                        continue;
                    }

                    // TODO: compact may be not necessary here.
                    let chunk = chunk.compact();
                    let (data_chunk, ops) = chunk.into_parts();
                    // SAFETY: Already compacted.
                    assert!(data_chunk.is_compacted());
                    let len = data_chunk.cardinality();

                    // Collect each window's data into a chunk.
                    let mut chunks = Vec::with_capacity(units);

                    for i in 0..units {
                        let window_start_col = if out_window_start_col_idx.is_some() {
                            Some(
                                self.window_start_exprs[i]
                                    .eval_infallible(&data_chunk)
                                    .await,
                            )
                        } else {
                            None
                        };
                        let window_end_col = if out_window_end_col_idx.is_some() {
                            Some(self.window_end_exprs[i].eval_infallible(&data_chunk).await)
                        } else {
                            None
                        };
                        let new_cols = output_indices
                            .iter()
                            .filter_map(|&idx| {
                                if idx < logical_window_start_col_idx {
                                    Some(data_chunk.column_at(idx).clone())
                                } else if idx == logical_window_start_col_idx {
                                    Some(window_start_col.clone().unwrap())
                                } else if idx == logical_window_end_col_idx {
                                    Some(window_end_col.clone().unwrap())
                                } else {
                                    None
                                }
                            })
                            .collect();

                        chunks.push(DataChunk::new(new_cols, len));
                    }

                    // Reorganize the output rows from the same input row together.
                    let mut row_iters = chunks.iter().map(|c| c.rows()).collect_vec();

                    let data_types = chunks[0].data_types();
                    let mut chunk_builder = StreamChunkBuilder::new(chunk_size, data_types);

                    for &op in &*ops {
                        // Since there could be multiple rows for the same input row, we need to
                        // transform the `U-`/`U+` into `-`/`+` and then duplicate it.
                        let op = match op {
                            Op::Insert | Op::UpdateInsert => Op::Insert,
                            Op::Delete | Op::UpdateDelete => Op::Delete,
                        };
                        for row_iter in &mut row_iters {
                            if let Some(chunk) =
                                chunk_builder.append_row(op, row_iter.next().unwrap())
                            {
                                yield Message::Chunk(chunk);
                            }
                        }
                    }

                    if let Some(chunk) = chunk_builder.take() {
                        yield Message::Chunk(chunk);
                    }

                    // Rows should be exhausted.
                    debug_assert!(row_iters.into_iter().all(|mut it| it.next().is_none()));
                }
                Message::Barrier(b) => {
                    yield Message::Barrier(b);
                }
                Message::Watermark(w) => {
                    if w.col_idx == time_col_idx {
                        if let (Some(out_start_idx), Some(start_expr)) =
                            (out_window_start_col_idx, self.window_start_exprs.first())
                        {
                            let w = w
                                .clone()
                                .transform_with_expr(start_expr, out_start_idx)
                                .await;
                            if let Some(w) = w {
                                yield Message::Watermark(w);
                            }
                        }
                        if let (Some(out_end_idx), Some(end_expr)) =
                            (out_window_end_col_idx, self.window_end_exprs.first())
                        {
                            let w = w.transform_with_expr(end_expr, out_end_idx).await;
                            if let Some(w) = w {
                                yield Message::Watermark(w);
                            }
                        }
                    } else if let Some(out_idx) =
                        output_indices.iter().position(|&idx| idx == w.col_idx)
                    {
                        yield Message::Watermark(w.with_idx(out_idx));
                    }
                }
            };
        }
    }
}

#[cfg(test)]
mod tests {
    use risingwave_common::array::stream_chunk::StreamChunkTestExt;
    use risingwave_common::catalog::Field;
    use risingwave_common::types::test_utils::IntervalTestExt;
    use risingwave_expr::expr::test_utils::make_hop_window_expression;

    use super::*;
    use crate::executor::test_utils::MockSource;

    const CHUNK_SIZE: usize = 256;

    fn create_executor(output_indices: Vec<usize>) -> Box<dyn Execute> {
        let field1 = Field::unnamed(DataType::Int64);
        let field2 = Field::unnamed(DataType::Int64);
        let field3 = Field::with_name(DataType::Timestamp, "created_at");
        let schema = Schema::new(vec![field1, field2, field3]);
        let pk_indices = vec![0];

        let chunk = StreamChunk::from_pretty(
            &"I I TS
            + 1 1 ^10:00:00
            + 2 3 ^10:05:00
            - 3 2 ^10:14:00
            + 4 1 ^10:22:00
           U- 5 2 ^10:33:00
           U+ 6 2 ^10:42:00
            - 7 1 ^10:51:00
            + 8 3 ^11:02:00"
                .replace('^', "2022-02-02T"),
        );
        let input =
            MockSource::with_chunks(vec![chunk]).into_executor(schema.clone(), pk_indices.clone());
        let window_slide = Interval::from_minutes(15);
        let window_size = Interval::from_minutes(30);
        let window_offset = Interval::from_minutes(0);
        let (window_start_exprs, window_end_exprs) = make_hop_window_expression(
            DataType::Timestamp,
            2,
            window_size,
            window_slide,
            window_offset,
        )
        .unwrap();

        HopWindowExecutor::new(
            ActorContext::for_test(123),
            input,
            2,
            window_slide,
            window_size,
            window_start_exprs
                .into_iter()
                .map(NonStrictExpression::for_test)
                .collect(),
            window_end_exprs
                .into_iter()
                .map(NonStrictExpression::for_test)
                .collect(),
            output_indices,
            CHUNK_SIZE,
        )
        .boxed()
    }

    #[tokio::test]
    async fn test_execute() {
        let default_indices: Vec<_> = (0..5).collect();
        let executor = create_executor(default_indices);

        let mut stream = executor.execute();
        // TODO: add more test infra to reduce the duplicated codes below.

        let chunk = stream.next().await.unwrap().unwrap().into_chunk().unwrap();
        assert_eq!(
            chunk,
            StreamChunk::from_pretty(
                &"I I TS        TS        TS
                + 1 1 ^10:00:00 ^09:45:00 ^10:15:00
                + 1 1 ^10:00:00 ^10:00:00 ^10:30:00
                + 2 3 ^10:05:00 ^09:45:00 ^10:15:00
                + 2 3 ^10:05:00 ^10:00:00 ^10:30:00
                - 3 2 ^10:14:00 ^09:45:00 ^10:15:00
                - 3 2 ^10:14:00 ^10:00:00 ^10:30:00
                + 4 1 ^10:22:00 ^10:00:00 ^10:30:00
                + 4 1 ^10:22:00 ^10:15:00 ^10:45:00
                - 5 2 ^10:33:00 ^10:15:00 ^10:45:00
                - 5 2 ^10:33:00 ^10:30:00 ^11:00:00
                + 6 2 ^10:42:00 ^10:15:00 ^10:45:00
                + 6 2 ^10:42:00 ^10:30:00 ^11:00:00
                - 7 1 ^10:51:00 ^10:30:00 ^11:00:00
                - 7 1 ^10:51:00 ^10:45:00 ^11:15:00
                + 8 3 ^11:02:00 ^10:45:00 ^11:15:00
                + 8 3 ^11:02:00 ^11:00:00 ^11:30:00"
                    .replace('^', "2022-02-02T"),
            )
        );
    }

    #[tokio::test]
    async fn test_output_indices() {
        let executor = create_executor(vec![4, 1, 0, 2]);

        let mut stream = executor.execute();
        // TODO: add more test infra to reduce the duplicated codes below.

        let chunk = stream.next().await.unwrap().unwrap().into_chunk().unwrap();
        assert_eq!(
            chunk,
            StreamChunk::from_pretty(
                &"TS        I I TS
                + ^10:15:00 1 1 ^10:00:00
                + ^10:30:00 1 1 ^10:00:00
                + ^10:15:00 3 2 ^10:05:00
                + ^10:30:00 3 2 ^10:05:00
                - ^10:15:00 2 3 ^10:14:00
                - ^10:30:00 2 3 ^10:14:00
                + ^10:30:00 1 4 ^10:22:00
                + ^10:45:00 1 4 ^10:22:00
                - ^10:45:00 2 5 ^10:33:00
                - ^11:00:00 2 5 ^10:33:00
                + ^10:45:00 2 6 ^10:42:00
                + ^11:00:00 2 6 ^10:42:00
                - ^11:00:00 1 7 ^10:51:00
                - ^11:15:00 1 7 ^10:51:00
                + ^11:15:00 3 8 ^11:02:00
                + ^11:30:00 3 8 ^11:02:00"
                    .replace('^', "2022-02-02T"),
            )
        );
    }
}