Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

refactor(agg): separate semantics of AggKind and AggType #18640

Merged
merged 6 commits into from
Sep 23, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
10 changes: 5 additions & 5 deletions proto/expr.proto
Original file line number Diff line number Diff line change
Expand Up @@ -422,7 +422,7 @@

// Aggregate Function Calls for Aggregation
message AggCall {
enum Type {
enum Kind {
UNSPECIFIED = 0;
SUM = 1;
MIN = 2;
Expand Down Expand Up @@ -458,16 +458,16 @@
// wraps a scalar function that takes a list as input as an aggregate function.
WRAP_SCALAR = 101;
}
Type type = 1;
Kind kind = 1;

Check failure on line 461 in proto/expr.proto

View workflow job for this annotation

GitHub Actions / Check breaking changes in Protobuf files

Field "1" with name "kind" on message "AggCall" changed option "json_name" from "type" to "kind".

Check failure on line 461 in proto/expr.proto

View workflow job for this annotation

GitHub Actions / Check breaking changes in Protobuf files

Field "1" with name "kind" on message "AggCall" changed type from "expr.AggCall.Type" to "expr.AggCall.Kind".

Check failure on line 461 in proto/expr.proto

View workflow job for this annotation

GitHub Actions / Check breaking changes in Protobuf files

Field "1" on message "AggCall" changed name from "type" to "kind".
repeated InputRef args = 2;
data.DataType return_type = 3;
bool distinct = 4;
repeated common.ColumnOrder order_by = 5;
ExprNode filter = 6;
repeated Constant direct_args = 7;
// optional. only used when the type is USER_DEFINED.
// optional. only used when the kind is USER_DEFINED.
UserDefinedFunctionMetadata udf = 8;
// optional. only used when the type is WRAP_SCALAR.
// optional. only used when the kind is WRAP_SCALAR.
ExprNode scalar = 9;
}

Expand Down Expand Up @@ -562,7 +562,7 @@

oneof type {
GeneralType general = 1;
AggCall.Type aggregate = 2;
AggCall.Kind aggregate = 2;

Check failure on line 565 in proto/expr.proto

View workflow job for this annotation

GitHub Actions / Check breaking changes in Protobuf files

Field "2" with name "aggregate" on message "WindowFunction" changed type from "expr.AggCall.Type" to "expr.AggCall.Kind".
}
repeated InputRef args = 3;
data.DataType return_type = 4;
Expand Down
14 changes: 7 additions & 7 deletions src/batch/benches/hash_agg.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ use risingwave_common::catalog::{Field, Schema};
use risingwave_common::memory::MemoryContext;
use risingwave_common::types::DataType;
use risingwave_common::{enable_jemalloc, hash};
use risingwave_expr::aggregate::{AggCall, AggKind, PbAggKind};
use risingwave_expr::aggregate::{AggCall, AggType, PbAggKind};
use risingwave_pb::expr::{PbAggCall, PbInputRef};
use tokio::runtime::Runtime;
use utils::{create_input, execute_executor};
Expand All @@ -34,12 +34,12 @@ enable_jemalloc!();

fn create_agg_call(
input_schema: &Schema,
agg_kind: AggKind,
agg_type: AggType,
args: Vec<usize>,
return_type: DataType,
) -> PbAggCall {
PbAggCall {
r#type: agg_kind.to_protobuf() as i32,
kind: agg_type.to_protobuf_simple() as i32,
args: args
.into_iter()
.map(|col_idx| PbInputRef {
Expand All @@ -59,7 +59,7 @@ fn create_agg_call(

fn create_hash_agg_executor(
group_key_columns: Vec<usize>,
agg_kind: AggKind,
agg_type: AggType,
arg_columns: Vec<usize>,
return_type: DataType,
chunk_size: usize,
Expand All @@ -75,7 +75,7 @@ fn create_hash_agg_executor(

let agg_calls = vec![create_agg_call(
input_schema,
agg_kind,
agg_type,
arg_columns,
return_type,
)];
Expand Down Expand Up @@ -131,7 +131,7 @@ fn bench_hash_agg(c: &mut Criterion) {
(vec![0, 2], PbAggKind::Min, vec![1], DataType::Int64),
];

for (group_key_columns, agg_kind, arg_columns, return_type) in bench_variants {
for (group_key_columns, agg_type, arg_columns, return_type) in bench_variants {
for chunk_size in &[32, 128, 512, 1024, 2048, 4096] {
c.bench_with_input(
BenchmarkId::new("HashAggExecutor", chunk_size),
Expand All @@ -142,7 +142,7 @@ fn bench_hash_agg(c: &mut Criterion) {
|| {
create_hash_agg_executor(
group_key_columns.clone(),
agg_kind.into(),
agg_type.into(),
arg_columns.clone(),
return_type.clone(),
chunk_size,
Expand Down
10 changes: 5 additions & 5 deletions src/batch/src/executor/hash_agg.rs
Original file line number Diff line number Diff line change
Expand Up @@ -755,7 +755,7 @@ mod tests {
use risingwave_common::util::sort_util::{ColumnOrder, OrderType};
use risingwave_pb::data::data_type::TypeName;
use risingwave_pb::data::PbDataType;
use risingwave_pb::expr::agg_call::Type;
use risingwave_pb::expr::agg_call::PbKind as PbAggKind;
use risingwave_pb::expr::{AggCall, InputRef};

use super::*;
Expand Down Expand Up @@ -788,7 +788,7 @@ mod tests {
));

let agg_call = AggCall {
r#type: Type::Sum as i32,
kind: PbAggKind::Sum as i32,
args: vec![InputRef {
index: 2,
r#type: Some(PbDataType {
Expand Down Expand Up @@ -873,7 +873,7 @@ mod tests {
);

let agg_call = AggCall {
r#type: Type::Count as i32,
kind: PbAggKind::Count as i32,
args: vec![],
return_type: Some(PbDataType {
type_name: TypeName::Int64 as i32,
Expand Down Expand Up @@ -985,7 +985,7 @@ mod tests {
);

let agg_call = AggCall {
r#type: Type::Sum as i32,
kind: PbAggKind::Sum as i32,
args: vec![InputRef {
index: 2,
r#type: Some(PbDataType {
Expand Down Expand Up @@ -1078,7 +1078,7 @@ mod tests {
));

let agg_call = AggCall {
r#type: Type::Sum as i32,
kind: PbAggKind::Sum as i32,
args: vec![InputRef {
index: 2,
r#type: Some(PbDataType {
Expand Down
Loading
Loading