risingwave_stream/executor/
filter.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
// 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 risingwave_common::array::{Array, ArrayImpl, Op};
use risingwave_common::bitmap::BitmapBuilder;
use risingwave_common::util::iter_util::ZipEqFast;
use risingwave_expr::expr::NonStrictExpression;

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

/// `FilterExecutor` filters data with the `expr`. The `expr` takes a chunk of data,
/// and returns a boolean array on whether each item should be retained. And then,
/// `FilterExecutor` will insert, delete or update element into next executor according
/// to the result of the expression.
pub struct FilterExecutor {
    _ctx: ActorContextRef,
    input: Executor,

    /// Expression of the current filter, note that the filter must always have the same output for
    /// the same input.
    expr: NonStrictExpression,
}

impl FilterExecutor {
    pub fn new(ctx: ActorContextRef, input: Executor, expr: NonStrictExpression) -> Self {
        Self {
            _ctx: ctx,
            input,
            expr,
        }
    }

    pub(super) fn filter(
        chunk: StreamChunk,
        filter: Arc<ArrayImpl>,
    ) -> StreamExecutorResult<Option<StreamChunk>> {
        let (data_chunk, ops) = chunk.into_parts();

        let (columns, vis) = data_chunk.into_parts();

        let n = ops.len();

        // TODO: Can we update ops and visibility inplace?
        let mut new_ops = Vec::with_capacity(n);
        let mut new_visibility = BitmapBuilder::with_capacity(n);
        let mut last_res = false;

        assert_eq!(vis.len(), n);

        let ArrayImpl::Bool(bool_array) = &*filter else {
            panic!("unmatched type: filter expr returns a non-null array");
        };
        for (&op, res) in ops.iter().zip_eq_fast(bool_array.iter()) {
            // SAFETY: ops.len() == pred_output.len() == visibility.len()
            let res = res.unwrap_or(false);
            match op {
                Op::Insert | Op::Delete => {
                    new_ops.push(op);
                    if res {
                        new_visibility.append(true);
                    } else {
                        new_visibility.append(false);
                    }
                }
                Op::UpdateDelete => {
                    last_res = res;
                }
                Op::UpdateInsert => match (last_res, res) {
                    (true, false) => {
                        new_ops.push(Op::Delete);
                        new_ops.push(Op::UpdateInsert);
                        new_visibility.append(true);
                        new_visibility.append(false);
                    }
                    (false, true) => {
                        new_ops.push(Op::UpdateDelete);
                        new_ops.push(Op::Insert);
                        new_visibility.append(false);
                        new_visibility.append(true);
                    }
                    (true, true) => {
                        new_ops.push(Op::UpdateDelete);
                        new_ops.push(Op::UpdateInsert);
                        new_visibility.append(true);
                        new_visibility.append(true);
                    }
                    (false, false) => {
                        new_ops.push(Op::UpdateDelete);
                        new_ops.push(Op::UpdateInsert);
                        new_visibility.append(false);
                        new_visibility.append(false);
                    }
                },
            }
        }

        let new_visibility = new_visibility.finish();

        Ok(if new_visibility.count_ones() > 0 {
            let new_chunk = StreamChunk::with_visibility(new_ops, columns, new_visibility);
            Some(new_chunk)
        } else {
            None
        })
    }
}

impl Debug for FilterExecutor {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("FilterExecutor")
            .field("expr", &self.expr)
            .finish()
    }
}

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

impl FilterExecutor {
    #[try_stream(ok = Message, error = StreamExecutorError)]
    async fn execute_inner(self) {
        let input = self.input.execute();
        #[for_await]
        for msg in input {
            let msg = msg?;
            match msg {
                Message::Watermark(w) => yield Message::Watermark(w),
                Message::Chunk(chunk) => {
                    let chunk = chunk.compact();

                    let pred_output = self.expr.eval_infallible(chunk.data_chunk()).await;

                    match Self::filter(chunk, pred_output)? {
                        Some(new_chunk) => yield Message::Chunk(new_chunk),
                        None => continue,
                    }
                }
                m => yield m,
            }
        }
    }
}

#[cfg(test)]
mod tests {
    use risingwave_common::array::stream_chunk::StreamChunkTestExt;
    use risingwave_common::catalog::Field;

    use super::super::test_utils::expr::build_from_pretty;
    use super::super::test_utils::MockSource;
    use super::super::*;
    use super::*;

    #[tokio::test]
    async fn test_filter() {
        let chunk1 = StreamChunk::from_pretty(
            " I I
            + 1 4
            + 5 2
            + 6 6
            - 7 5",
        );
        let chunk2 = StreamChunk::from_pretty(
            "  I I
            U- 5 3  // true -> true
            U+ 7 5  // expect UpdateDelete, UpdateInsert
            U- 5 3  // true -> false
            U+ 3 5  // expect Delete
            U- 3 5  // false -> true
            U+ 5 3  // expect Insert
            U- 3 5  // false -> false
            U+ 4 6  // expect nothing",
        );
        let schema = Schema {
            fields: vec![
                Field::unnamed(DataType::Int64),
                Field::unnamed(DataType::Int64),
            ],
        };
        let pk_indices = PkIndices::new();
        let source = MockSource::with_chunks(vec![chunk1, chunk2])
            .into_executor(schema.clone(), pk_indices.clone());

        let test_expr = build_from_pretty("(greater_than:boolean $0:int8 $1:int8)");

        let mut filter = FilterExecutor::new(ActorContext::for_test(123), source, test_expr)
            .boxed()
            .execute();

        let chunk = filter.next().await.unwrap().unwrap().into_chunk().unwrap();
        assert_eq!(
            chunk,
            StreamChunk::from_pretty(
                " I I
                + 1 4 D
                + 5 2
                + 6 6 D
                - 7 5",
            )
        );

        let chunk = filter.next().await.unwrap().unwrap().into_chunk().unwrap();
        assert_eq!(
            chunk,
            StreamChunk::from_pretty(
                "  I I
                U- 5 3
                U+ 7 5
                -  5 3
                U+ 3 5 D
                U- 3 5 D
                +  5 3
                U- 3 5 D
                U+ 4 6 D",
            )
        );

        assert!(filter.next().await.unwrap().unwrap().is_stop());
    }
}