risingwave_batch/executor/aggregation/
distinct.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
// 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::collections::HashSet;
use std::ops::Range;

use risingwave_common::array::StreamChunk;
use risingwave_common::bitmap::BitmapBuilder;
use risingwave_common::row::{OwnedRow, Row};
use risingwave_common::types::{DataType, Datum};
use risingwave_common_estimate_size::EstimateSize;
use risingwave_expr::aggregate::{
    AggStateDyn, AggregateFunction, AggregateState, BoxedAggregateFunction,
};
use risingwave_expr::Result;

/// `Distinct` is a wrapper of `Aggregator` that only keeps distinct rows.
pub struct Distinct {
    inner: BoxedAggregateFunction,
}

/// The intermediate state for distinct aggregation.
#[derive(Debug)]
struct State {
    /// Inner aggregate function state.
    inner: AggregateState,
    /// The set of distinct rows.
    exists: HashSet<OwnedRow>, // TODO: optimize for small rows
    exists_estimated_heap_size: usize,
}

impl EstimateSize for State {
    fn estimated_heap_size(&self) -> usize {
        self.inner.estimated_size()
            + self.exists.capacity() * std::mem::size_of::<OwnedRow>()
            + self.exists_estimated_heap_size
    }
}

impl AggStateDyn for State {}

impl Distinct {
    pub fn new(inner: BoxedAggregateFunction) -> Self {
        Self { inner }
    }
}

#[async_trait::async_trait]
impl AggregateFunction for Distinct {
    fn return_type(&self) -> DataType {
        self.inner.return_type()
    }

    fn create_state(&self) -> Result<AggregateState> {
        Ok(AggregateState::Any(Box::new(State {
            inner: self.inner.create_state()?,
            exists: HashSet::new(),
            exists_estimated_heap_size: 0,
        })))
    }

    async fn update(&self, state: &mut AggregateState, input: &StreamChunk) -> Result<()> {
        self.update_range(state, input, 0..input.capacity()).await
    }

    async fn update_range(
        &self,
        state: &mut AggregateState,
        input: &StreamChunk,
        range: Range<usize>,
    ) -> Result<()> {
        let state = state.downcast_mut::<State>();

        let mut bitmap_builder = BitmapBuilder::with_capacity(input.capacity());
        bitmap_builder.append_bitmap(input.data_chunk().visibility());
        for row_id in range.clone() {
            let (row_ref, vis) = input.data_chunk().row_at(row_id);
            let row = row_ref.to_owned_row();
            let row_size = row.estimated_heap_size();
            let b = vis && state.exists.insert(row);
            if b {
                state.exists_estimated_heap_size += row_size;
            }
            bitmap_builder.set(row_id, b);
        }
        let input = input.clone_with_vis(bitmap_builder.finish());
        self.inner
            .update_range(&mut state.inner, &input, range)
            .await
    }

    async fn get_result(&self, state: &AggregateState) -> Result<Datum> {
        let state = state.downcast_ref::<State>();
        self.inner.get_result(&state.inner).await
    }
}

#[cfg(test)]
mod tests {
    use futures_util::FutureExt;
    use risingwave_common::array::StreamChunk;
    use risingwave_common::test_prelude::StreamChunkTestExt;
    use risingwave_common::types::{Datum, Decimal};
    use risingwave_expr::aggregate::AggCall;

    use super::super::build;

    #[test]
    fn distinct_sum_int32() {
        let input = StreamChunk::from_pretty(
            " i
            + 1
            + 1
            + 3",
        );
        test_agg("(sum:int8 $0:int4 distinct)", input, Some(4i64.into()));
    }

    #[test]
    fn distinct_sum_int64() {
        let input = StreamChunk::from_pretty(
            " I
            + 1
            + 1
            + 3",
        );
        test_agg(
            "(sum:decimal $0:int8 distinct)",
            input,
            Some(Decimal::from(4).into()),
        );
    }

    #[test]
    fn distinct_min_float32() {
        let input = StreamChunk::from_pretty(
            " f
            + 1.0
            + 2.0
            + 3.0",
        );
        test_agg(
            "(min:float4 $0:float4 distinct)",
            input,
            Some(1.0f32.into()),
        );
    }

    #[test]
    fn distinct_min_char() {
        let input = StreamChunk::from_pretty(
            " T
            + b
            + aa",
        );
        test_agg(
            "(min:varchar $0:varchar distinct)",
            input,
            Some("aa".into()),
        );
    }

    #[test]
    fn distinct_max_char() {
        let input = StreamChunk::from_pretty(
            " T
            + b
            + aa",
        );
        test_agg("(max:varchar $0:varchar distinct)", input, Some("b".into()));
    }

    #[test]
    fn distinct_count_int32() {
        let input = StreamChunk::from_pretty(
            " i
            + 1
            + 1
            + 3",
        );
        test_agg("(count:int8 $0:int4 distinct)", input, Some(2i64.into()));

        let input = StreamChunk::from_pretty("i");
        test_agg("(count:int8 $0:int4 distinct)", input, Some(0i64.into()));

        let input = StreamChunk::from_pretty(
            " i
            + .",
        );
        test_agg("(count:int8 $0:int4 distinct)", input, Some(0i64.into()));
    }

    fn test_agg(pretty: &str, input: StreamChunk, expected: Datum) {
        let agg = build(&AggCall::from_pretty(pretty)).unwrap();
        let mut state = agg.create_state().unwrap();
        agg.update(&mut state, &input)
            .now_or_never()
            .unwrap()
            .unwrap();
        let actual = agg.get_result(&state).now_or_never().unwrap().unwrap();
        assert_eq!(actual, expected);
    }
}