risingwave_stream/from_proto/
values.rs1use itertools::Itertools;
16use risingwave_expr::expr::build_non_strict_from_prost;
17use risingwave_pb::stream_plan::ValuesNode;
18use risingwave_storage::StateStore;
19
20use super::ExecutorBuilder;
21use crate::error::StreamResult;
22use crate::executor::{Executor, ValuesExecutor};
23use crate::task::ExecutorParams;
24
25pub struct ValuesExecutorBuilder;
28
29impl_stream_node_body!(Values(ValuesNode) => ValuesExecutorBuilder);
30
31impl ExecutorBuilder for ValuesExecutorBuilder {
32 type Node = ValuesNode;
33
34 async fn new_boxed_executor(
35 params: ExecutorParams,
36 node: &ValuesNode,
37 _store: impl StateStore,
38 ) -> StreamResult<Executor> {
39 let barrier_receiver = params
40 .local_barrier_manager
41 .subscribe_barrier(params.actor_context.id);
42 let progress = params
43 .local_barrier_manager
44 .register_create_mview_progress(¶ms.actor_context);
45 let rows = node
46 .get_tuples()
47 .iter()
48 .map(|tuple| {
49 tuple
50 .get_cells()
51 .iter()
52 .map(|node| {
53 build_non_strict_from_prost(node, params.eval_error_report.clone()).unwrap()
54 })
55 .collect_vec()
56 })
57 .collect_vec();
58 let exec = ValuesExecutor::new(
59 params.actor_context,
60 params.info.schema.clone(),
61 progress,
62 rows,
63 barrier_receiver,
64 );
65 Ok((params.info, exec).into())
66 }
67}