Skip to content

Commit 5992017

Browse files
authored
refactor(hash agg): directly use AggGroup instead of Box<AggGroup> (risingwavelabs#8745)
Signed-off-by: Richard Chien <[email protected]>
1 parent 5f41727 commit 5992017

File tree

1 file changed

+16
-19
lines changed

1 file changed

+16
-19
lines changed

src/stream/src/executor/hash_agg.rs

Lines changed: 16 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -41,14 +41,14 @@ use super::{
4141
use crate::cache::{cache_may_stale, new_with_hasher, ExecutorCache};
4242
use crate::common::table::state_table::StateTable;
4343
use crate::error::StreamResult;
44-
use crate::executor::aggregation::{generate_agg_schema, AggCall, AggGroup};
44+
use crate::executor::aggregation::{generate_agg_schema, AggCall, AggGroup as GenericAggGroup};
4545
use crate::executor::error::StreamExecutorError;
4646
use crate::executor::monitor::StreamingMetrics;
4747
use crate::executor::{BoxedMessageStream, Message};
4848
use crate::task::AtomicU64Ref;
4949

50-
type BoxedAggGroup<S> = Box<AggGroup<S, OnlyOutputIfHasInput>>;
51-
type AggGroupCache<K, S> = ExecutorCache<K, BoxedAggGroup<S>, PrecomputedBuildHasher>;
50+
type AggGroup<S> = GenericAggGroup<S, OnlyOutputIfHasInput>;
51+
type AggGroupCache<K, S> = ExecutorCache<K, AggGroup<S>, PrecomputedBuildHasher>;
5252

5353
/// [`HashAggExecutor`] could process large amounts of data using a state backend. It works as
5454
/// follows:
@@ -255,19 +255,17 @@ impl<K: HashKey, S: StateStore> HashAggExecutor<K, S> {
255255
Some(async {
256256
// Create `AggGroup` for the current group if not exists. This will
257257
// fetch previous agg result from the result table.
258-
let agg_group = Box::new(
259-
AggGroup::create(
260-
Some(key.deserialize(group_key_types)?),
261-
&this.agg_calls,
262-
&this.storages,
263-
&this.result_table,
264-
&this.input_pk_indices,
265-
this.row_count_index,
266-
this.extreme_cache_size,
267-
&this.input_schema,
268-
)
269-
.await?,
270-
);
258+
let agg_group = AggGroup::create(
259+
Some(key.deserialize(group_key_types)?),
260+
&this.agg_calls,
261+
&this.storages,
262+
&this.result_table,
263+
&this.input_pk_indices,
264+
this.row_count_index,
265+
this.extreme_cache_size,
266+
&this.input_schema,
267+
)
268+
.await?;
271269
Ok::<_, StreamExecutorError>((key.clone(), agg_group))
272270
})
273271
}
@@ -345,7 +343,7 @@ impl<K: HashKey, S: StateStore> HashAggExecutor<K, S> {
345343

346344
// Apply chunk to each of the state (per agg_call), for each group.
347345
for (key, visibility) in group_visibilities {
348-
let agg_group = vars.agg_group_cache.get_mut(&key).unwrap().as_mut();
346+
let agg_group = vars.agg_group_cache.get_mut(&key).unwrap();
349347
let visibilities = call_visibilities
350348
.iter()
351349
.map(Option::as_ref)
@@ -421,8 +419,7 @@ impl<K: HashKey, S: StateStore> HashAggExecutor<K, S> {
421419
let agg_group = vars
422420
.agg_group_cache
423421
.get_mut(key)
424-
.expect("changed group must have corresponding AggGroup")
425-
.as_mut();
422+
.expect("changed group must have corresponding AggGroup");
426423
agg_group.flush_state_if_needed(&mut this.storages).await?;
427424
}
428425

0 commit comments

Comments
 (0)