risingwave_stream/executor/
values.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
// 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::vec;

use risingwave_common::array::{DataChunk, Op};
use risingwave_common::ensure;
use risingwave_common::util::iter_util::ZipEqFast;
use risingwave_expr::expr::NonStrictExpression;
use tokio::sync::mpsc::UnboundedReceiver;

use crate::executor::prelude::*;
use crate::task::CreateMviewProgressReporter;

const DEFAULT_CHUNK_SIZE: usize = 1024;

/// Execute `values` in stream. As is a leaf, current workaround holds a `barrier_executor`.
/// May refractor with `BarrierRecvExecutor` in the near future.
pub struct ValuesExecutor {
    ctx: ActorContextRef,

    schema: Schema,
    // Receiver of barrier channel.
    barrier_receiver: UnboundedReceiver<Barrier>,
    progress: CreateMviewProgressReporter,

    rows: vec::IntoIter<Vec<NonStrictExpression>>,
}

impl ValuesExecutor {
    /// Currently hard-code the `pk_indices` as the last column.
    pub fn new(
        ctx: ActorContextRef,
        schema: Schema,
        progress: CreateMviewProgressReporter,
        rows: Vec<Vec<NonStrictExpression>>,
        barrier_receiver: UnboundedReceiver<Barrier>,
    ) -> Self {
        Self {
            ctx,
            schema,
            progress,
            barrier_receiver,
            rows: rows.into_iter(),
        }
    }

    #[try_stream(ok = Message, error = StreamExecutorError)]
    async fn execute_inner(self) {
        let Self {
            schema,
            mut progress,
            mut barrier_receiver,
            mut rows,
            ..
        } = self;
        let barrier = barrier_receiver
            .recv()
            .instrument_await("values_executor_recv_first_barrier")
            .await
            .unwrap();

        let emit = barrier.is_newly_added(self.ctx.id);
        let paused_on_startup = barrier.is_pause_on_startup();

        yield Message::Barrier(barrier);

        // If it's failover, do not evaluate rows (assume they have been yielded)
        if emit {
            if paused_on_startup {
                // Wait for the data stream to be resumed before yielding the chunks.
                while let Some(barrier) = barrier_receiver.recv().await {
                    let is_resume = barrier.is_resume();
                    yield Message::Barrier(barrier);
                    if is_resume {
                        break;
                    }
                }
            }

            let cardinality = schema.len();
            ensure!(cardinality > 0);
            while !rows.is_empty() {
                // We need a one row chunk rather than an empty chunk because constant
                // expression's eval result is same size as input chunk
                // cardinality.
                let one_row_chunk = DataChunk::new_dummy(1);

                let chunk_size = DEFAULT_CHUNK_SIZE.min(rows.len());
                let mut array_builders = schema.create_array_builders(chunk_size);
                for row in rows.by_ref().take(chunk_size) {
                    for (expr, builder) in row.into_iter().zip_eq_fast(&mut array_builders) {
                        let out = expr.eval_infallible(&one_row_chunk).await;
                        builder.append_array(&out);
                    }
                }

                let columns: Vec<_> = array_builders
                    .into_iter()
                    .map(|b| b.finish().into())
                    .collect();

                let chunk = DataChunk::new(columns, chunk_size);
                let ops = vec![Op::Insert; chunk_size];

                let stream_chunk = StreamChunk::from_parts(ops, chunk);
                yield Message::Chunk(stream_chunk);
            }
        }

        while let Some(barrier) = barrier_receiver.recv().await {
            if emit {
                progress.finish(barrier.epoch, 0);
            }
            yield Message::Barrier(barrier);
        }
    }
}

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

#[cfg(test)]
mod tests {

    use futures::StreamExt;
    use risingwave_common::array::{
        Array, ArrayImpl, I16Array, I32Array, I64Array, StructArray, StructValue,
    };
    use risingwave_common::catalog::{Field, Schema};
    use risingwave_common::types::{DataType, ScalarImpl, StructType};
    use risingwave_common::util::epoch::test_epoch;
    use risingwave_expr::expr::{BoxedExpression, LiteralExpression, NonStrictExpression};
    use tokio::sync::mpsc::unbounded_channel;

    use super::ValuesExecutor;
    use crate::executor::test_utils::StreamExecutorTestExt;
    use crate::executor::{ActorContext, AddMutation, Barrier, Execute, Mutation};
    use crate::task::{CreateMviewProgressReporter, LocalBarrierManager};

    #[tokio::test]
    async fn test_values() {
        let barrier_manager = LocalBarrierManager::for_test();
        let progress = CreateMviewProgressReporter::for_test(barrier_manager);
        let actor_id = progress.actor_id();
        let (tx, barrier_receiver) = unbounded_channel();
        let value = StructValue::new(vec![Some(1.into()), Some(2.into()), Some(3.into())]);
        let exprs = vec![
            Box::new(LiteralExpression::new(
                DataType::Int16,
                Some(ScalarImpl::Int16(1)),
            )) as BoxedExpression,
            Box::new(LiteralExpression::new(
                DataType::Int32,
                Some(ScalarImpl::Int32(2)),
            )),
            Box::new(LiteralExpression::new(
                DataType::Int64,
                Some(ScalarImpl::Int64(3)),
            )),
            Box::new(LiteralExpression::new(
                DataType::new_struct(
                    vec![DataType::Int32, DataType::Int32, DataType::Int32],
                    vec![],
                ),
                Some(ScalarImpl::Struct(value)),
            )),
            Box::new(LiteralExpression::new(
                DataType::Int64,
                Some(ScalarImpl::Int64(0)),
            )),
        ];
        let schema = exprs
            .iter() // for each column
            .map(|col| Field::unnamed(col.return_type()))
            .collect::<Schema>();
        let values_executor_struct = ValuesExecutor::new(
            ActorContext::for_test(actor_id),
            schema,
            progress,
            vec![exprs
                .into_iter()
                .map(NonStrictExpression::for_test)
                .collect()],
            barrier_receiver,
        );
        let mut values_executor = Box::new(values_executor_struct).execute();

        // Init barrier
        let first_message =
            Barrier::new_test_barrier(test_epoch(1)).with_mutation(Mutation::Add(AddMutation {
                adds: Default::default(),
                added_actors: maplit::hashset! {actor_id},
                splits: Default::default(),
                pause: false,
                subscriptions_to_add: vec![],
            }));
        tx.send(first_message).unwrap();

        assert!(matches!(
            values_executor.next_unwrap_ready_barrier().unwrap(),
            Barrier { .. }
        ));

        // Consume the barrier
        let values_msg = values_executor.next().await.unwrap().unwrap();

        let result = values_msg
            .into_chunk()
            .unwrap()
            .compact()
            .data_chunk()
            .to_owned();

        let array: ArrayImpl = StructArray::new(
            StructType::unnamed(vec![DataType::Int32, DataType::Int32, DataType::Int32]),
            vec![
                I32Array::from_iter([1]).into_ref(),
                I32Array::from_iter([2]).into_ref(),
                I32Array::from_iter([3]).into_ref(),
            ],
            [true].into_iter().collect(),
        )
        .into();

        assert_eq!(*result.column_at(0), I16Array::from_iter([1]).into_ref());
        assert_eq!(*result.column_at(1), I32Array::from_iter([2]).into_ref());
        assert_eq!(*result.column_at(2), I64Array::from_iter([3]).into_ref());
        assert_eq!(*result.column_at(3), array.into());
        assert_eq!(*result.column_at(4), I64Array::from_iter([0]).into_ref());

        // ValueExecutor should simply forward following barriers
        tx.send(Barrier::new_test_barrier(test_epoch(2))).unwrap();

        assert!(matches!(
            values_executor.next_unwrap_ready_barrier().unwrap(),
            Barrier { .. }
        ));

        tx.send(Barrier::new_test_barrier(test_epoch(3))).unwrap();

        assert!(matches!(
            values_executor.next_unwrap_ready_barrier().unwrap(),
            Barrier { .. }
        ));
    }
}