risingwave_stream/executor/aggregation/
agg_state_cache.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
// 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.

//! Object-safe version of [`StateCache`] for aggregation.

use risingwave_common::array::StreamChunk;
use risingwave_common::row::Row;
use risingwave_common::types::{DataType, Datum, ToOwnedDatum};
use risingwave_common::util::chunk_coalesce::DataChunkBuilder;
use risingwave_common::util::memcmp_encoding::MemcmpEncoded;
use risingwave_common::util::row_serde::OrderedRowSerde;
use risingwave_common_estimate_size::EstimateSize;
use smallvec::SmallVec;

use crate::common::state_cache::{StateCache, StateCacheFiller};

/// Cache key type.
type CacheKey = MemcmpEncoded;

#[derive(Debug)]
pub struct CacheValue(SmallVec<[Datum; 2]>);

/// Trait that defines the interface of state table cache for stateful streaming agg.
pub trait AggStateCache: EstimateSize {
    /// Check if the cache is synced with state table.
    fn is_synced(&self) -> bool;

    /// Apply a batch of updates to the cache.
    fn apply_batch(
        &mut self,
        chunk: &StreamChunk,
        cache_key_serializer: &OrderedRowSerde,
        arg_col_indices: &[usize],
        order_col_indices: &[usize],
    );

    /// Begin syncing the cache with state table.
    fn begin_syncing(&mut self) -> Box<dyn AggStateCacheFiller + Send + Sync + '_>;

    /// Output batches from the cache.
    fn output_batches(&self, chunk_size: usize) -> Box<dyn Iterator<Item = StreamChunk> + '_>;

    /// Output the first value.
    fn output_first(&self) -> Datum;
}

/// Trait that defines agg state cache syncing interface.
pub trait AggStateCacheFiller {
    /// Get the capacity of the cache to be filled. `None` means unlimited.
    fn capacity(&self) -> Option<usize>;

    /// Insert an entry to the cache without checking row count, capacity, key order, etc.
    /// Just insert into the inner cache structure, e.g. `BTreeMap`.
    fn append(&mut self, key: CacheKey, value: CacheValue);

    /// Mark the cache as synced.
    fn finish(self: Box<Self>);
}

/// A wrapper over generic [`StateCache`] that implements [`AggStateCache`].
#[derive(EstimateSize)]
pub struct GenericAggStateCache<C>
where
    C: StateCache<Key = CacheKey, Value = CacheValue>,
{
    state_cache: C,
    input_types: Vec<DataType>,
}

impl<C> GenericAggStateCache<C>
where
    C: StateCache<Key = CacheKey, Value = CacheValue>,
{
    pub fn new(state_cache: C, input_types: &[DataType]) -> Self {
        Self {
            state_cache,
            input_types: input_types.to_vec(),
        }
    }
}

impl<C> AggStateCache for GenericAggStateCache<C>
where
    C: StateCache<Key = CacheKey, Value = CacheValue>,
    for<'a> C::Filler<'a>: Send + Sync,
{
    fn is_synced(&self) -> bool {
        self.state_cache.is_synced()
    }

    fn apply_batch(
        &mut self,
        chunk: &StreamChunk,
        cache_key_serializer: &OrderedRowSerde,
        arg_col_indices: &[usize],
        order_col_indices: &[usize],
    ) {
        let rows = chunk.rows().map(|(op, row)| {
            let key = {
                let mut key = Vec::new();
                cache_key_serializer.serialize_datums(
                    order_col_indices
                        .iter()
                        .map(|col_idx| row.datum_at(*col_idx)),
                    &mut key,
                );
                key.into()
            };
            let value = CacheValue(
                arg_col_indices
                    .iter()
                    .map(|col_idx| row.datum_at(*col_idx).to_owned_datum())
                    .collect(),
            );
            (op, key, value)
        });
        self.state_cache.apply_batch(rows);
    }

    fn begin_syncing(&mut self) -> Box<dyn AggStateCacheFiller + Send + Sync + '_> {
        Box::new(GenericAggStateCacheFiller::<'_, C> {
            cache_filler: self.state_cache.begin_syncing(),
        })
    }

    fn output_batches(&self, chunk_size: usize) -> Box<dyn Iterator<Item = StreamChunk> + '_> {
        let mut values = self.state_cache.values();
        Box::new(std::iter::from_fn(move || {
            // build data chunk from rows
            let mut builder = DataChunkBuilder::new(self.input_types.clone(), chunk_size);
            for row in &mut values {
                if let Some(chunk) = builder.append_one_row(row.0.as_slice()) {
                    return Some(chunk.into());
                }
            }
            builder.consume_all().map(|chunk| chunk.into())
        }))
    }

    fn output_first(&self) -> Datum {
        let value = self.state_cache.values().next()?;
        value.0[0].clone()
    }
}

pub struct GenericAggStateCacheFiller<'filler, C>
where
    C: StateCache<Key = CacheKey, Value = CacheValue> + 'filler,
{
    cache_filler: C::Filler<'filler>,
}

impl<C> AggStateCacheFiller for GenericAggStateCacheFiller<'_, C>
where
    C: StateCache<Key = CacheKey, Value = CacheValue>,
{
    fn capacity(&self) -> Option<usize> {
        self.cache_filler.capacity()
    }

    fn append(&mut self, key: CacheKey, value: CacheValue) {
        self.cache_filler.insert_unchecked(key, value);
    }

    fn finish(self: Box<Self>) {
        self.cache_filler.finish()
    }
}

impl FromIterator<Datum> for CacheValue {
    fn from_iter<T: IntoIterator<Item = Datum>>(iter: T) -> Self {
        Self(iter.into_iter().collect())
    }
}

impl EstimateSize for CacheValue {
    fn estimated_heap_size(&self) -> usize {
        let data_heap_size: usize = self.0.iter().map(|datum| datum.estimated_heap_size()).sum();
        self.0.len() * std::mem::size_of::<Datum>() + data_heap_size
    }
}