risingwave_expr/aggregate/
scalar_wrapper.rs1use risingwave_common::array::{Array, ArrayBuilderImpl, DataChunk, ListArray};
16use risingwave_common::types::ListValue;
17
18use super::*;
19use crate::expr::{BoxedExpression, Expression};
20
21#[derive(Debug)]
23pub struct ScalarWrapper {
24    arg_type: DataType,
25    scalar: BoxedExpression,
26}
27
28impl ScalarWrapper {
29    pub fn new(arg_type: DataType, scalar: BoxedExpression) -> Self {
31        Self { arg_type, scalar }
32    }
33}
34
35#[async_trait::async_trait]
36impl AggregateFunction for ScalarWrapper {
37    fn return_type(&self) -> DataType {
38        self.scalar.return_type()
39    }
40
41    fn create_state(&self) -> Result<AggregateState> {
43        Ok(AggregateState::Any(Box::new(State(
44            self.arg_type.create_array_builder(0),
45        ))))
46    }
47
48    async fn update(&self, state: &mut AggregateState, input: &StreamChunk) -> Result<()> {
52        let state = &mut state.downcast_mut::<State>().0;
53        let column = input.column_at(0);
54        for i in input.visibility().iter_ones() {
55            state.append(column.datum_at(i));
56        }
57        Ok(())
58    }
59
60    async fn update_range(
62        &self,
63        state: &mut AggregateState,
64        input: &StreamChunk,
65        range: Range<usize>,
66    ) -> Result<()> {
67        let state = &mut state.downcast_mut::<State>().0;
68        let column = input.column_at(0);
69        for i in input.visibility().iter_ones() {
70            if i < range.start {
71                continue;
72            } else if i >= range.end {
73                break;
74            }
75            state.append(column.datum_at(i));
76        }
77        Ok(())
78    }
79
80    async fn get_result(&self, state: &AggregateState) -> Result<Datum> {
82        let state = &state.downcast_ref::<State>().0;
83        let list = ListValue::new(state.clone().finish());
85        let chunk = DataChunk::new(vec![ListArray::from_iter([list]).into_ref()], 1);
86        let output = self.scalar.eval(&chunk).await?;
87        Ok(output.to_datum())
88    }
89
90    fn encode_state(&self, _state: &AggregateState) -> Result<Datum> {
92        panic!("should not store state in state table")
93    }
94
95    fn decode_state(&self, _datum: Datum) -> Result<AggregateState> {
97        panic!("should not store state in state table")
98    }
99}
100
101#[derive(Debug)]
103struct State(ArrayBuilderImpl);
104
105impl EstimateSize for State {
106    fn estimated_heap_size(&self) -> usize {
107        self.0.estimated_heap_size()
108    }
109}
110
111impl AggStateDyn for State {}