12
12
// See the License for the specific language governing permissions and
13
13
// limitations under the License.
14
14
15
- use std:: collections:: hash_map:: Entry :: Vacant ;
16
- use std:: collections:: { HashMap , HashSet } ;
15
+ use std:: collections:: HashSet ;
17
16
use std:: sync:: Arc ;
18
17
19
18
use async_trait:: async_trait;
@@ -30,7 +29,7 @@ use risingwave_storage::StateStore;
30
29
use super :: top_n_cache:: TopNCacheTrait ;
31
30
use super :: utils:: * ;
32
31
use super :: TopNCache ;
33
- use crate :: cache:: cache_may_stale;
32
+ use crate :: cache:: { cache_may_stale, EvictableHashMap , ExecutorCache , LruManagerRef } ;
34
33
use crate :: common:: table:: state_table:: StateTable ;
35
34
use crate :: error:: StreamResult ;
36
35
use crate :: executor:: error:: StreamExecutorResult ;
@@ -52,6 +51,8 @@ impl<S: StateStore> GroupTopNExecutor<S, false> {
52
51
executor_id : u64 ,
53
52
group_by : Vec < usize > ,
54
53
state_table : StateTable < S > ,
54
+ lru_manager : Option < LruManagerRef > ,
55
+ cache_size : usize ,
55
56
) -> StreamResult < Self > {
56
57
let info = input. info ( ) ;
57
58
let schema = input. schema ( ) . clone ( ) ;
@@ -68,6 +69,8 @@ impl<S: StateStore> GroupTopNExecutor<S, false> {
68
69
executor_id,
69
70
group_by,
70
71
state_table,
72
+ lru_manager,
73
+ cache_size,
71
74
) ?,
72
75
} )
73
76
}
@@ -85,6 +88,8 @@ impl<S: StateStore> GroupTopNExecutor<S, true> {
85
88
executor_id : u64 ,
86
89
group_by : Vec < usize > ,
87
90
state_table : StateTable < S > ,
91
+ lru_manager : Option < LruManagerRef > ,
92
+ cache_size : usize ,
88
93
) -> StreamResult < Self > {
89
94
let info = input. info ( ) ;
90
95
let schema = input. schema ( ) . clone ( ) ;
@@ -102,6 +107,8 @@ impl<S: StateStore> GroupTopNExecutor<S, true> {
102
107
executor_id,
103
108
group_by,
104
109
state_table,
110
+ lru_manager,
111
+ cache_size,
105
112
) ?,
106
113
} )
107
114
}
@@ -132,7 +139,7 @@ pub struct InnerGroupTopNExecutorNew<S: StateStore, const WITH_TIES: bool> {
132
139
group_by : Vec < usize > ,
133
140
134
141
/// group key -> cache for this group
135
- caches : HashMap < Vec < Datum > , TopNCache < WITH_TIES > > ,
142
+ caches : GroupTopNCache < WITH_TIES > ,
136
143
137
144
/// The number of fields of the ORDER BY clause, and will be used to split key into `CacheKey`.
138
145
order_by_len : usize ,
@@ -153,6 +160,8 @@ impl<S: StateStore, const WITH_TIES: bool> InnerGroupTopNExecutorNew<S, WITH_TIE
153
160
executor_id : u64 ,
154
161
group_by : Vec < usize > ,
155
162
state_table : StateTable < S > ,
163
+ lru_manager : Option < LruManagerRef > ,
164
+ cache_size : usize ,
156
165
) -> StreamResult < Self > {
157
166
// order_pairs is superset of pk
158
167
assert ! ( order_pairs
@@ -196,13 +205,47 @@ impl<S: StateStore, const WITH_TIES: bool> InnerGroupTopNExecutorNew<S, WITH_TIE
196
205
pk_indices,
197
206
internal_key_indices,
198
207
group_by,
199
- caches : HashMap :: new ( ) ,
208
+ caches : GroupTopNCache :: new ( lru_manager , cache_size ) ,
200
209
order_by_len,
201
210
cache_key_serde,
202
211
} )
203
212
}
204
213
}
205
214
215
+ pub struct GroupTopNCache < const WITH_TIES : bool > {
216
+ data : ExecutorCache < Vec < Datum > , TopNCache < WITH_TIES > > ,
217
+ }
218
+
219
+ impl < const WITH_TIES : bool > GroupTopNCache < WITH_TIES > {
220
+ pub fn new ( lru_manager : Option < LruManagerRef > , cache_size : usize ) -> Self {
221
+ let cache = if let Some ( lru_manager) = lru_manager {
222
+ ExecutorCache :: Managed ( lru_manager. create_cache ( ) )
223
+ } else {
224
+ ExecutorCache :: Local ( EvictableHashMap :: new ( cache_size) )
225
+ } ;
226
+ Self { data : cache }
227
+ }
228
+
229
+ fn clear ( & mut self ) {
230
+ self . data . clear ( )
231
+ }
232
+
233
+ fn get_mut ( & mut self , key : & [ Datum ] ) -> Option < & mut TopNCache < WITH_TIES > > {
234
+ self . data . get_mut ( key)
235
+ }
236
+
237
+ fn contains ( & mut self , key : & [ Datum ] ) -> bool {
238
+ self . data . contains ( key)
239
+ }
240
+
241
+ fn insert ( & mut self , key : Vec < Datum > , value : TopNCache < WITH_TIES > ) {
242
+ self . data . push ( key, value) ;
243
+ }
244
+
245
+ fn evict ( & mut self ) {
246
+ self . data . evict ( )
247
+ }
248
+ }
206
249
#[ async_trait]
207
250
impl < S : StateStore , const WITH_TIES : bool > TopNExecutorBase
208
251
for InnerGroupTopNExecutorNew < S , WITH_TIES >
@@ -229,12 +272,12 @@ where
229
272
230
273
// If 'self.caches' does not already have a cache for the current group, create a new
231
274
// cache for it and insert it into `self.caches`
232
- if let Vacant ( entry ) = self . caches . entry ( group_key) {
275
+ if ! self . caches . contains ( & group_key) {
233
276
let mut topn_cache = TopNCache :: new ( self . offset , self . limit , self . order_by_len ) ;
234
277
self . managed_state
235
278
. init_topn_cache ( Some ( & pk_prefix) , & mut topn_cache, self . order_by_len )
236
279
. await ?;
237
- entry . insert ( topn_cache) ;
280
+ self . caches . insert ( group_key , topn_cache) ;
238
281
}
239
282
let cache = self . caches . get_mut ( & pk_prefix. 0 ) . unwrap ( ) ;
240
283
@@ -291,6 +334,10 @@ where
291
334
}
292
335
}
293
336
337
+ fn evict ( & mut self ) {
338
+ self . caches . evict ( )
339
+ }
340
+
294
341
async fn init ( & mut self , epoch : EpochPair ) -> StreamExecutorResult < ( ) > {
295
342
self . managed_state . state_table . init_epoch ( epoch) ;
296
343
Ok ( ( ) )
@@ -406,6 +453,8 @@ mod tests {
406
453
1 ,
407
454
vec ! [ 1 ] ,
408
455
state_table,
456
+ None ,
457
+ 0 ,
409
458
)
410
459
. unwrap ( ) ,
411
460
) ;
@@ -504,6 +553,8 @@ mod tests {
504
553
1 ,
505
554
vec ! [ 1 ] ,
506
555
state_table,
556
+ None ,
557
+ 0 ,
507
558
)
508
559
. unwrap ( ) ,
509
560
) ;
@@ -594,6 +645,8 @@ mod tests {
594
645
1 ,
595
646
vec ! [ 1 , 2 ] ,
596
647
state_table,
648
+ None ,
649
+ 0 ,
597
650
)
598
651
. unwrap ( ) ,
599
652
) ;
0 commit comments