Skip to content

Commit 9abe5dc

Browse files
authored
perf(hash agg): use get_mut instead of pop+put pattern (#8691)
Signed-off-by: Richard Chien <[email protected]>
1 parent bf95473 commit 9abe5dc

File tree

1 file changed

+13
-9
lines changed

1 file changed

+13
-9
lines changed

src/stream/src/executor/hash_agg.rs

Lines changed: 13 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -14,6 +14,7 @@
1414

1515
use std::collections::{HashMap, HashSet};
1616
use std::marker::PhantomData;
17+
use std::ptr::NonNull;
1718
use std::sync::Arc;
1819

1920
use futures::{stream, StreamExt, TryStreamExt};
@@ -436,11 +437,17 @@ impl<K: HashKey, S: StateStore> HashAggExecutor<K, S> {
436437

437438
// Calculate current outputs, concurrently.
438439
let futs = keys_in_batch.into_iter().map(|key| {
439-
// Pop out the agg group temporarily.
440-
let mut agg_group = vars
441-
.agg_group_cache
442-
.pop(&key)
443-
.expect("changed group must have corresponding AggGroup");
440+
// Get agg group of the key.
441+
let agg_group = {
442+
let mut ptr: NonNull<_> = vars
443+
.agg_group_cache
444+
.get_mut(&key)
445+
.expect("changed group must have corresponding AggGroup")
446+
.into();
447+
// SAFETY: `key`s in `keys_in_batch` are unique by nature, because they're
448+
// from `group_change_set` which is a set.
449+
unsafe { ptr.as_mut() }
450+
};
444451
async {
445452
let curr_outputs = agg_group.get_outputs(&this.storages).await?;
446453
Ok::<_, StreamExecutorError>((key, agg_group, curr_outputs))
@@ -452,7 +459,7 @@ impl<K: HashKey, S: StateStore> HashAggExecutor<K, S> {
452459
.try_collect()
453460
.await?;
454461

455-
for (key, mut agg_group, curr_outputs) in outputs_in_batch {
462+
for (key, agg_group, curr_outputs) in outputs_in_batch {
456463
let AggChangesInfo {
457464
n_appended_ops,
458465
result_row,
@@ -477,9 +484,6 @@ impl<K: HashKey, S: StateStore> HashAggExecutor<K, S> {
477484
this.result_table.insert(result_row);
478485
}
479486
}
480-
481-
// Put the agg group back into the agg group cache.
482-
vars.agg_group_cache.put(key, agg_group);
483487
}
484488

485489
let columns = builders

0 commit comments

Comments
 (0)