From d4af542a34f28db6f6303cd09da10bb5efb8bc30 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 24 Jun 2024 12:16:08 +0800 Subject: [PATCH 01/33] add ApproxPercentile AggKind --- proto/expr.proto | 2 ++ src/expr/core/src/aggregate/def.rs | 3 +++ src/frontend/src/binder/expr/function.rs | 6 +++++- src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs | 1 + 4 files changed, 11 insertions(+), 1 deletion(-) diff --git a/proto/expr.proto b/proto/expr.proto index 9babde3fecaf..da466d690f82 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -436,6 +436,8 @@ message AggCall { LAST_VALUE = 25; GROUPING = 26; INTERNAL_LAST_SEEN_VALUE = 27; + APPROX_PERCENTILE = 28; + // user defined aggregate function USER_DEFINED = 100; // wraps a scalar function that takes a list as input as an aggregate function. diff --git a/src/expr/core/src/aggregate/def.rs b/src/expr/core/src/aggregate/def.rs index cd735a1df49f..855d0dde1a9a 100644 --- a/src/expr/core/src/aggregate/def.rs +++ b/src/expr/core/src/aggregate/def.rs @@ -314,6 +314,9 @@ pub mod agg_kinds { | PbAggKind::VarPop | PbAggKind::VarSamp | PbAggKind::Grouping, + // ApproxPercentile always uses custom agg executors, + // rather than an aggregation operator + | PbAggKind::ApproxPercentile ) }; } diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 1d08fc3e19cb..65f81406b5f1 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -568,7 +568,11 @@ impl Binder { ); if f.distinct { - if matches!(kind, AggKind::Builtin(PbAggKind::ApproxCountDistinct)) { + if matches!( + kind, + AggKind::Builtin(PbAggKind::ApproxCountDistinct) + | AggKind::Builtin(PbAggKind::ApproxPercentile) + ) { return Err(ErrorCode::InvalidInputSyntax(format!( "DISTINCT is not allowed for approximate aggregation `{}`", kind diff --git a/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs index eab534b1b9a3..33bb59e59bf1 100644 --- a/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs +++ b/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs @@ -172,6 +172,7 @@ impl Rule for ApplyAggTransposeRule { | PbAggKind::LastValue | PbAggKind::InternalLastSeenValue // All statistical aggregates only consider non-null inputs. + | PbAggKind::ApproxPercentile | PbAggKind::VarPop | PbAggKind::VarSamp | PbAggKind::StddevPop From 28afa59c19c3c7af02e3870e8dac5ca1b48b89b6 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 18 Jul 2024 14:46:07 +0800 Subject: [PATCH 02/33] add plan node for two-phase simple agg --- .../src/optimizer/plan_node/logical_agg.rs | 75 ++++++++++++++-- src/frontend/src/optimizer/plan_node/mod.rs | 12 +++ .../stream_global_approx_percentile.rs | 78 ++++++++++++++++ .../optimizer/plan_node/stream_keyed_merge.rs | 88 +++++++++++++++++++ .../stream_local_approx_percentile.rs | 77 ++++++++++++++++ 5 files changed, 325 insertions(+), 5 deletions(-) create mode 100644 src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs create mode 100644 src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs create mode 100644 src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index ed2b4d308815..b272aca20ee0 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -22,9 +22,9 @@ use risingwave_expr::aggregate::{agg_kinds, AggKind, PbAggKind}; use super::generic::{self, Agg, GenericPlanRef, PlanAggCall, ProjectBuilder}; use super::utils::impl_distill_by_unit; use super::{ - BatchHashAgg, BatchSimpleAgg, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, - PlanTreeNodeUnary, PredicatePushdown, StreamHashAgg, StreamProject, StreamSimpleAgg, - StreamStatelessSimpleAgg, ToBatch, ToStream, + BatchHashAgg, BatchSimpleAgg, ColPrunable, ExprRewritable, Logical, LogicalShare, PlanBase, + PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHashAgg, StreamProject, StreamShare, + StreamSimpleAgg, StreamStatelessSimpleAgg, ToBatch, ToStream, }; use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{ @@ -33,6 +33,9 @@ use crate::expr::{ }; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::GenericPlanNode; +use crate::optimizer::plan_node::stream_global_approx_percentile::StreamGlobalApproxPercentile; +use crate::optimizer::plan_node::stream_keyed_merge::StreamKeyedMerge; +use crate::optimizer::plan_node::stream_local_approx_percentile::StreamLocalApproxPercentile; use crate::optimizer::plan_node::{ gen_filter_and_pushdown, BatchSortAgg, ColumnPruningContext, LogicalDedup, LogicalProject, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, @@ -167,7 +170,11 @@ impl LogicalAgg { } /// Generates distributed stream plan. - fn gen_dist_stream_agg_plan(&self, stream_input: PlanRef) -> Result { + fn gen_dist_stream_agg_plan( + &self, + stream_input: PlanRef, + ctx: &mut ToStreamContext, + ) -> Result { use super::stream::prelude::*; let input_dist = stream_input.distribution(); @@ -192,6 +199,15 @@ impl LogicalAgg { self.core.can_two_phase_agg() }); + // Handle 2-phase approx_percentile aggregation. + if self + .agg_calls() + .iter() + .any(|agg_call| agg_call.agg_kind == AggKind::ApproxPercentile) + { + return self.gen_approx_percentile_plan(stream_input, ctx); + } + // Stateless 2-phase simple agg // can be applied on stateless simple agg calls, // with input distributed by [`Distribution::AnyShard`] @@ -242,6 +258,55 @@ impl LogicalAgg { } } + /// ------- REWRITES ------- + /// + /// SimpleAgg { agg_calls: [count(*), approx_percentile(quantile, relative_error, v1), sum(v1)] } + /// -> Input + /// + /// -------- INTO ---------- + /// + /// KeyedMerge { agg_calls: [count(*), approx_percentile(quantile, relative_error, v1), sum(v1)] } + /// -> SimpleAgg { agg_calls: [count(*)] } + /// -> StreamShare { id: 0 } + /// -> Input + /// -> GlobalApproxPercentile { args: [bucket_id, count] } + /// -> LocalApproxPercentile { args: [relative_error, ] } + /// -> StreamShare { id: 0 } + /// -> Input + // NOTE(kwannoel): We can't split LocalApproxPercentile into Project + Agg, because + // for the agg step, we need to count by bucket id, and not just dispatch a single record + // downstream. So we need a custom executor for this logic. + fn gen_approx_percentile_plan( + &self, + stream_input: PlanRef, + ctx: &mut ToStreamContext, + ) -> Result { + if !self.group_key().is_empty() { + return Err(ErrorCode::NotSupported( + "ApproxPercentile aggregation with GROUP BY is not supported".into(), + "try to remove GROUP BY".into(), + ) + .into()); + } + + let shared_input = LogicalShare::new(stream_input).to_stream(ctx)?; + let (rewritten_simple_agg, lhs_mapping, rhs_mapping) = + Self::rewrite_simple_agg_for_approx_percentile(shared_input.clone())?; + let rewritten_approx_percentile = StreamLocalApproxPercentile::new(shared_input); + let global_approx_percentile = + StreamGlobalApproxPercentile::new(rewritten_approx_percentile.into()); + let agg_merge = + StreamKeyedMerge::new(rewritten_simple_agg, global_approx_percentile.into()); + Ok(agg_merge.into()) + } + + /// Returns rewritten simple agg + Mapping of the lhs and rhs indices for keyed merge. + fn rewrite_simple_agg_for_approx_percentile( + input: PlanRef, + ) -> Result<(PlanRef, ColIndexMapping, ColIndexMapping)> { + todo!() + } + pub fn core(&self) -> &Agg { &self.core } @@ -1110,7 +1175,7 @@ impl ToStream for LogicalAgg { return logical_dedup.to_stream(ctx); } - let plan = self.gen_dist_stream_agg_plan(stream_input)?; + let plan = self.gen_dist_stream_agg_plan(stream_input, ctx)?; let (plan, n_final_agg_calls) = if let Some(final_agg) = plan.as_stream_simple_agg() { if eowc { diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index ee2b16265e7a..2cf7e67dd2b6 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -893,10 +893,13 @@ mod stream_exchange; mod stream_expand; mod stream_filter; mod stream_fs_fetch; +mod stream_global_approx_percentile; mod stream_group_topn; mod stream_hash_agg; mod stream_hash_join; mod stream_hop_window; +mod stream_keyed_merge; +mod stream_local_approx_percentile; mod stream_materialize; mod stream_now; mod stream_over_window; @@ -1002,10 +1005,13 @@ pub use stream_exchange::StreamExchange; pub use stream_expand::StreamExpand; pub use stream_filter::StreamFilter; pub use stream_fs_fetch::StreamFsFetch; +pub use stream_global_approx_percentile::StreamGlobalApproxPercentile; pub use stream_group_topn::StreamGroupTopN; pub use stream_hash_agg::StreamHashAgg; pub use stream_hash_join::StreamHashJoin; pub use stream_hop_window::StreamHopWindow; +pub use stream_keyed_merge::StreamKeyedMerge; +pub use stream_local_approx_percentile::StreamLocalApproxPercentile; pub use stream_materialize::StreamMaterialize; pub use stream_now::StreamNow; pub use stream_over_window::StreamOverWindow; @@ -1150,6 +1156,9 @@ macro_rules! for_all_plan_nodes { , { Stream, OverWindow } , { Stream, FsFetch } , { Stream, ChangeLog } + , { Stream, GlobalApproxPercentile } + , { Stream, LocalApproxPercentile } + , { Stream, KeyedMerge } } }; } @@ -1276,6 +1285,9 @@ macro_rules! for_stream_plan_nodes { , { Stream, OverWindow } , { Stream, FsFetch } , { Stream, ChangeLog } + , { Stream, GlobalApproxPercentile } + , { Stream, LocalApproxPercentile } + , { Stream, KeyedMerge } } }; } diff --git a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs new file mode 100644 index 000000000000..9a0f4661e05e --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs @@ -0,0 +1,78 @@ +// 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. + +use pretty_xmlish::XmlNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; + +use crate::expr::{ExprRewriter, ExprVisitor}; +use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::plan_node::generic::GenericPlanRef; +use crate::optimizer::plan_node::utils::{childless_record, watermark_pretty, Distill}; +use crate::optimizer::plan_node::{ + ExprRewritable, PlanBase, PlanTreeNodeUnary, Stream, StreamHopWindow, StreamKeyedMerge, + StreamNode, +}; +use crate::stream_fragmenter::BuildFragmentGraphState; +use crate::PlanRef; + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct StreamGlobalApproxPercentile { + pub base: PlanBase, +} + +impl StreamGlobalApproxPercentile { + pub fn new(input: PlanRef) -> Self { + Self { base: todo!() } + } +} + +impl Distill for StreamGlobalApproxPercentile { + fn distill<'a>(&self) -> XmlNode<'a> { + todo!() + } +} + +impl PlanTreeNodeUnary for StreamGlobalApproxPercentile { + fn input(&self) -> PlanRef { + todo!() + } + + fn clone_with_input(&self, input: PlanRef) -> Self { + todo!() + } +} + +impl_plan_tree_node_for_unary! {StreamGlobalApproxPercentile} + +impl StreamNode for StreamGlobalApproxPercentile { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { + todo!() + } +} + +impl ExprRewritable for StreamGlobalApproxPercentile { + fn has_rewritable_expr(&self) -> bool { + todo!() + } + + fn rewrite_exprs(&self, _rewriter: &mut dyn ExprRewriter) -> PlanRef { + todo!() + } +} + +impl ExprVisitable for StreamGlobalApproxPercentile { + fn visit_exprs(&self, v: &mut dyn ExprVisitor) { + todo!() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs new file mode 100644 index 000000000000..ed726dda3bef --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs @@ -0,0 +1,88 @@ +// 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. + +use pretty_xmlish::XmlNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; +use risingwave_pb::stream_plan::HopWindowNode; + +use crate::expr::{ExprRewriter, ExprVisitor}; +use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::plan_node::utils::Distill; +use crate::optimizer::plan_node::{ + ExprRewritable, PlanBase, PlanTreeNodeBinary, Stream, StreamHopWindow, + StreamLocalApproxPercentile, StreamNode, +}; +use crate::stream_fragmenter::BuildFragmentGraphState; +use crate::PlanRef; + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct StreamKeyedMerge { + pub lhs_input: PlanRef, + pub rhs_input: PlanRef, + pub base: PlanBase, +} + +impl StreamKeyedMerge { + pub fn new(lhs_input: PlanRef, rhs_input: PlanRef) -> Self { + Self { + lhs_input, + rhs_input, + base: todo!(), + } + } +} + +impl Distill for StreamKeyedMerge { + fn distill<'a>(&self) -> XmlNode<'a> { + todo!() + } +} + +impl PlanTreeNodeBinary for StreamKeyedMerge { + fn left(&self) -> PlanRef { + todo!() + } + + fn right(&self) -> PlanRef { + todo!() + } + + fn clone_with_left_right(&self, left: PlanRef, right: PlanRef) -> Self { + todo!() + } +} + +impl_plan_tree_node_for_binary! { StreamKeyedMerge } + +impl StreamNode for StreamKeyedMerge { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { + todo!() + } +} + +impl ExprRewritable for StreamKeyedMerge { + fn has_rewritable_expr(&self) -> bool { + todo!() + } + + fn rewrite_exprs(&self, _rewriter: &mut dyn ExprRewriter) -> PlanRef { + todo!() + } +} + +impl ExprVisitable for StreamKeyedMerge { + fn visit_exprs(&self, v: &mut dyn ExprVisitor) { + todo!() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs new file mode 100644 index 000000000000..24959eddbed4 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs @@ -0,0 +1,77 @@ +// 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. + +use pretty_xmlish::XmlNode; +use risingwave_pb::stream_plan::stream_node::PbNodeBody; + +use crate::expr::{ExprRewriter, ExprVisitor}; +use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::plan_node::utils::Distill; +use crate::optimizer::plan_node::{ + ExprRewritable, PlanBase, PlanNode, PlanTreeNodeUnary, Stream, StreamGlobalApproxPercentile, + StreamHopWindow, StreamNode, +}; +use crate::stream_fragmenter::BuildFragmentGraphState; +use crate::PlanRef; + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct StreamLocalApproxPercentile { + pub base: PlanBase, +} + +impl StreamLocalApproxPercentile { + pub fn new(input: PlanRef) -> Self { + Self { base: todo!() } + } +} + +impl Distill for StreamLocalApproxPercentile { + fn distill<'a>(&self) -> XmlNode<'a> { + todo!() + } +} + +impl PlanTreeNodeUnary for StreamLocalApproxPercentile { + fn input(&self) -> PlanRef { + todo!() + } + + fn clone_with_input(&self, input: PlanRef) -> Self { + todo!() + } +} + +impl_plan_tree_node_for_unary! {StreamLocalApproxPercentile} + +impl StreamNode for StreamLocalApproxPercentile { + fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody { + todo!() + } +} + +impl ExprRewritable for StreamLocalApproxPercentile { + fn has_rewritable_expr(&self) -> bool { + todo!() + } + + fn rewrite_exprs(&self, _rewriter: &mut dyn ExprRewriter) -> PlanRef { + todo!() + } +} + +impl ExprVisitable for StreamLocalApproxPercentile { + fn visit_exprs(&self, v: &mut dyn ExprVisitor) { + todo!() + } +} From c90b89ab3db914125d80e08c03857e56cda9072e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 18 Jul 2024 17:03:46 +0800 Subject: [PATCH 03/33] pass binder test --- src/expr/core/src/aggregate/def.rs | 4 +- .../impl/src/aggregate/approx_percentile.rs | 139 +++++++++++ src/expr/impl/src/aggregate/mod.rs | 1 + src/frontend/src/binder/expr/function.rs | 1 + src/frontend/src/binder/mod.rs | 217 ++++++++++++++++++ 5 files changed, 361 insertions(+), 1 deletion(-) create mode 100644 src/expr/impl/src/aggregate/approx_percentile.rs diff --git a/src/expr/core/src/aggregate/def.rs b/src/expr/core/src/aggregate/def.rs index 855d0dde1a9a..1fb985585cdf 100644 --- a/src/expr/core/src/aggregate/def.rs +++ b/src/expr/core/src/aggregate/def.rs @@ -446,11 +446,13 @@ pub mod agg_kinds { macro_rules! ordered_set { () => { AggKind::Builtin( - PbAggKind::PercentileCont | PbAggKind::PercentileDisc | PbAggKind::Mode, + PbAggKind::PercentileCont | PbAggKind::PercentileDisc | PbAggKind::Mode | PbAggKind::ApproxPercentile ) }; } pub use ordered_set; + + use crate::aggregate::{AggKind, PbAggKind}; } impl AggKind { diff --git a/src/expr/impl/src/aggregate/approx_percentile.rs b/src/expr/impl/src/aggregate/approx_percentile.rs new file mode 100644 index 000000000000..9b5b0c9e4ac0 --- /dev/null +++ b/src/expr/impl/src/aggregate/approx_percentile.rs @@ -0,0 +1,139 @@ +// 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. + +use std::ops::Range; + +use risingwave_common::array::*; +use risingwave_common::row::Row; +use risingwave_common::types::*; +use risingwave_common_estimate_size::EstimateSize; +use risingwave_expr::aggregate::{AggCall, AggStateDyn, AggregateFunction, AggregateState}; +use risingwave_expr::{build_aggregate, Result}; + +/// Computes the approx percentile, a value corresponding to the specified fraction within the +/// ordered set of aggregated argument values. This will interpolate between adjacent input items if +/// needed. +/// +/// ```slt +/// statement ok +/// create table t(x int, y bigint, z real, w double, v varchar); +/// +/// statement ok +/// insert into t values(1,10,100,1000,'10000'),(2,20,200,2000,'20000'),(3,30,300,3000,'30000'); +/// +/// query R +/// select percentile_cont(0.45) within group (order by x desc) from t; +/// ---- +/// 2.1 +/// +/// query R +/// select percentile_cont(0.45) within group (order by y desc) from t; +/// ---- +/// 21 +/// +/// query R +/// select percentile_cont(0.45) within group (order by z desc) from t; +/// ---- +/// 210 +/// +/// query R +/// select percentile_cont(0.45) within group (order by w desc) from t; +/// ---- +/// 2100 +/// +/// query R +/// select percentile_cont(NULL) within group (order by w desc) from t; +/// ---- +/// NULL +/// +/// statement ok +/// drop table t; +/// ``` +#[build_aggregate("approx_percentile(float8) -> float8")] +fn build(agg: &AggCall) -> Result> { + let fraction = agg.direct_args[0] + .literal() + .map(|x| (*x.as_float64()).into()); + Ok(Box::new(ApproxPercentile { fraction })) +} + +pub struct ApproxPercentile { + fraction: Option, +} + +#[derive(Debug, Default, EstimateSize)] +struct State(Vec); + +impl AggStateDyn for State {} + +impl ApproxPercentile { + fn add_datum(&self, state: &mut State, datum_ref: DatumRef<'_>) { + if let Some(datum) = datum_ref.to_owned_datum() { + state.0.push((*datum.as_float64()).into()); + } + } +} + +#[async_trait::async_trait] +impl AggregateFunction for ApproxPercentile { + fn return_type(&self) -> DataType { + DataType::Float64 + } + + fn create_state(&self) -> Result { + Ok(AggregateState::Any(Box::::default())) + } + + async fn update(&self, state: &mut AggregateState, input: &StreamChunk) -> Result<()> { + let state = state.downcast_mut(); + for (_, row) in input.rows() { + self.add_datum(state, row.datum_at(0)); + } + Ok(()) + } + + async fn update_range( + &self, + state: &mut AggregateState, + input: &StreamChunk, + range: Range, + ) -> Result<()> { + let state = state.downcast_mut(); + for (_, row) in input.rows_in(range) { + self.add_datum(state, row.datum_at(0)); + } + Ok(()) + } + + async fn get_result(&self, state: &AggregateState) -> Result { + let state = &state.downcast_ref::().0; + Ok( + if let Some(fraction) = self.fraction + && !state.is_empty() + { + let rn = fraction * (state.len() - 1) as f64; + let crn = f64::ceil(rn); + let frn = f64::floor(rn); + let result = if crn == frn { + state[crn as usize] + } else { + (crn - rn) * state[frn as usize] + (rn - frn) * state[crn as usize] + }; + Some(result.into()) + } else { + None + }, + ) + } +} diff --git a/src/expr/impl/src/aggregate/mod.rs b/src/expr/impl/src/aggregate/mod.rs index c0b6a5ae64c3..349574018fed 100644 --- a/src/expr/impl/src/aggregate/mod.rs +++ b/src/expr/impl/src/aggregate/mod.rs @@ -13,6 +13,7 @@ // limitations under the License. mod approx_count_distinct; +mod approx_percentile; mod array_agg; mod bit_and; mod bit_or; diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 65f81406b5f1..79505481ecac 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -511,6 +511,7 @@ impl Binder { } } (AggKind::Builtin(PbAggKind::Mode), [], [_arg]) => {} + (AggKind::Builtin(PbAggKind::ApproxPercentile), [_arg, _arg2], [_arg3]) => {} _ => { return Err(ErrorCode::InvalidInputSyntax(format!( "invalid direct args or within group argument for `{}` aggregation", diff --git a/src/frontend/src/binder/mod.rs b/src/frontend/src/binder/mod.rs index 1ca640f54b56..a39589288a63 100644 --- a/src/frontend/src/binder/mod.rs +++ b/src/frontend/src/binder/mod.rs @@ -763,4 +763,221 @@ mod tests { expected.assert_eq(&format!("{:#?}", bound)); } + + #[tokio::test] + async fn test_bind_approx_percentile() { + let stmt = risingwave_sqlparser::parser::Parser::parse_sql( + "SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (ORDER BY generate_series) FROM generate_series(1, 100)", + ).unwrap().into_iter().next().unwrap(); + let parse_expected = expect![[r#" + Query( + Query { + with: None, + body: Select( + Select { + distinct: All, + projection: [ + UnnamedExpr( + Function( + Function { + name: ObjectName( + [ + Ident { + value: "approx_percentile", + quote_style: None, + }, + ], + ), + args: [ + Unnamed( + Expr( + Value( + Number( + "0.5", + ), + ), + ), + ), + Unnamed( + Expr( + Value( + Number( + "0.01", + ), + ), + ), + ), + ], + variadic: false, + over: None, + distinct: false, + order_by: [], + filter: None, + within_group: Some( + OrderByExpr { + expr: Identifier( + Ident { + value: "generate_series", + quote_style: None, + }, + ), + asc: None, + nulls_first: None, + }, + ), + }, + ), + ), + ], + from: [ + TableWithJoins { + relation: TableFunction { + name: ObjectName( + [ + Ident { + value: "generate_series", + quote_style: None, + }, + ], + ), + alias: None, + args: [ + Unnamed( + Expr( + Value( + Number( + "1", + ), + ), + ), + ), + Unnamed( + Expr( + Value( + Number( + "100", + ), + ), + ), + ), + ], + with_ordinality: false, + }, + joins: [], + }, + ], + lateral_views: [], + selection: None, + group_by: [], + having: None, + }, + ), + order_by: [], + limit: None, + offset: None, + fetch: None, + }, + )"#]]; + parse_expected.assert_eq(&format!("{:#?}", stmt)); + + let mut binder = mock_binder(); + let bound = binder.bind(stmt).unwrap(); + + let expected = expect![[r#" + Query( + BoundQuery { + body: Select( + BoundSelect { + distinct: All, + select_items: [ + AggCall( + AggCall { + agg_kind: ApproxPercentile, + return_type: Float64, + args: [ + FunctionCall( + FunctionCall { + func_type: Cast, + return_type: Float64, + inputs: [ + InputRef( + InputRef { + index: 0, + data_type: Int32, + }, + ), + ], + }, + ), + ], + filter: Condition { + conjunctions: [], + }, + }, + ), + ], + aliases: [ + Some( + "approx_percentile", + ), + ], + from: Some( + TableFunction { + expr: TableFunction( + FunctionCall { + function_type: GenerateSeries, + return_type: Int32, + args: [ + Literal( + Literal { + data: Some( + Int32( + 1, + ), + ), + data_type: Some( + Int32, + ), + }, + ), + Literal( + Literal { + data: Some( + Int32( + 100, + ), + ), + data_type: Some( + Int32, + ), + }, + ), + ], + }, + ), + with_ordinality: false, + }, + ), + where_clause: None, + group_by: GroupKey( + [], + ), + having: None, + schema: Schema { + fields: [ + approx_percentile:Float64, + ], + }, + }, + ), + order: [], + limit: None, + offset: None, + with_ties: false, + extra_order_exprs: [], + }, + )"#]]; + + expected.assert_eq(&format!("{:#?}", bound)); + } } From 6cb3678e9a3e4bf009b6401871206107aa65bd60 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Thu, 18 Jul 2024 17:14:05 +0800 Subject: [PATCH 04/33] expose function direct_args --- src/frontend/src/binder/mod.rs | 44 +++++++++++++++++++++++++++++++ src/frontend/src/expr/agg_call.rs | 4 +++ 2 files changed, 48 insertions(+) diff --git a/src/frontend/src/binder/mod.rs b/src/frontend/src/binder/mod.rs index a39589288a63..29ed638e4f0f 100644 --- a/src/frontend/src/binder/mod.rs +++ b/src/frontend/src/binder/mod.rs @@ -913,6 +913,50 @@ mod tests { filter: Condition { conjunctions: [], }, + distinct: false, + order_by: OrderBy { + sort_exprs: [ + OrderByExpr { + expr: InputRef( + InputRef { + index: 0, + data_type: Int32, + }, + ), + order_type: OrderType { + direction: Ascending, + nulls_are: Largest, + }, + }, + ], + }, + direct_args: [ + Literal { + data: Some( + Decimal( + Normalized( + 0.5, + ), + ), + ), + data_type: Some( + Decimal, + ), + }, + Literal { + data: Some( + Decimal( + Normalized( + 0.01, + ), + ), + ), + data_type: Some( + Decimal, + ), + }, + ], + user_defined: false, }, ), ], diff --git a/src/frontend/src/expr/agg_call.rs b/src/frontend/src/expr/agg_call.rs index 9be381f4512a..c058112daee5 100644 --- a/src/frontend/src/expr/agg_call.rs +++ b/src/frontend/src/expr/agg_call.rs @@ -38,6 +38,10 @@ impl std::fmt::Debug for AggCall { .field("return_type", &self.return_type) .field("args", &self.args) .field("filter", &self.filter) + .field("distinct", &self.distinct) + .field("order_by", &self.order_by) + .field("direct_args", &self.direct_args) + .field("user_defined", &self.user_defined.is_some()) .finish() } else { let mut builder = f.debug_tuple(&format!("{}", self.agg_kind)); From 732238c564443c2583f2069fa1bf8cb904014e32 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Fri, 19 Jul 2024 13:38:01 +0800 Subject: [PATCH 05/33] add planner test case --- src/frontend/planner_test/tests/testdata/input/agg.yaml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index 70f83549ff18..e64bd9337a93 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -1000,3 +1000,8 @@ expected_outputs: - batch_plan - stream_plan +- sql: | + CREATE TABLE t (v1 int); + SELECT approx_percentile(v1, 0.5, 0.01) from t; + expected_outputs: + - stream_plan \ No newline at end of file From dcc59180d995469be3909591cb6bfe78ccf0debc Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sat, 20 Jul 2024 11:35:08 +0800 Subject: [PATCH 06/33] add logical project --- src/frontend/planner_test/tests/testdata/input/agg.yaml | 4 ++-- src/frontend/planner_test/tests/testdata/output/agg.yaml | 8 ++++++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index e64bd9337a93..7318e21b50ea 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -1002,6 +1002,6 @@ - stream_plan - sql: | CREATE TABLE t (v1 int); - SELECT approx_percentile(v1, 0.5, 0.01) from t; + SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) from t; expected_outputs: - - stream_plan \ No newline at end of file + - logical_plan \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 4c75b8318774..cc4cee22cb94 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1863,3 +1863,11 @@ └─StreamHashAgg { group_key: [t.a, t.b], aggs: [sum(t.c), sum(t.d), count(t.d), max(t.e), count] } └─StreamExchange { dist: HashShard(t.a, t.b) } └─StreamTableScan { table: t, columns: [t.a, t.b, t.c, t.d, t.e, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } +- sql: | + CREATE TABLE t (v1 int); + SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) from t; + logical_plan: |- + LogicalProject { exprs: [approx_percentile($expr1 order_by(t.v1 ASC))] } + └─LogicalAgg { aggs: [approx_percentile($expr1 order_by(t.v1 ASC))] } + └─LogicalProject { exprs: [t.v1::Float64 as $expr1, t.v1] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id] } From 1369c9b3576a860ef540d4453bb804398e203519 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 21 Jul 2024 20:31:29 +0800 Subject: [PATCH 07/33] extract agg calls --- .../src/optimizer/plan_node/logical_agg.rs | 60 +++++++++++++++---- 1 file changed, 49 insertions(+), 11 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index b272aca20ee0..9a175bd87016 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -276,6 +276,7 @@ impl LogicalAgg { // NOTE(kwannoel): We can't split LocalApproxPercentile into Project + Agg, because // for the agg step, we need to count by bucket id, and not just dispatch a single record // downstream. So we need a custom executor for this logic. + // TODO: Ban distinct agg? fn gen_approx_percentile_plan( &self, stream_input: PlanRef, @@ -290,21 +291,57 @@ impl LogicalAgg { } let shared_input = LogicalShare::new(stream_input).to_stream(ctx)?; - let (rewritten_simple_agg, lhs_mapping, rhs_mapping) = - Self::rewrite_simple_agg_for_approx_percentile(shared_input.clone())?; - let rewritten_approx_percentile = StreamLocalApproxPercentile::new(shared_input); - let global_approx_percentile = - StreamGlobalApproxPercentile::new(rewritten_approx_percentile.into()); + let (approx_percentile_agg_call, non_approx_percentile_agg_calls, lhs_mapping, rhs_mapping) = + self.extract_approx_percentile(); + let approx_percentile_agg = self.build_approx_percentile_agg(shared_input.clone()); + let simple_agg_without_approx_percentile = Agg::new( + non_approx_percentile_agg_calls, + self.group_key().clone(), + shared_input, + ) + .with_grouping_sets(self.grouping_sets().clone()) + .with_enable_two_phase(self.core().enable_two_phase) + .into(); let agg_merge = - StreamKeyedMerge::new(rewritten_simple_agg, global_approx_percentile.into()); + StreamKeyedMerge::new(simple_agg_without_approx_percentile, approx_percentile_agg); Ok(agg_merge.into()) } - /// Returns rewritten simple agg + Mapping of the lhs and rhs indices for keyed merge. - fn rewrite_simple_agg_for_approx_percentile( - input: PlanRef, - ) -> Result<(PlanRef, ColIndexMapping, ColIndexMapping)> { - todo!() + fn extract_approx_percentile( + &self, + ) -> ( + PlanAggCall, + Vec, + ColIndexMapping, + ColIndexMapping, + ) { + let mut approx_percentile_agg_call = None; + let estimated_len = self.agg_calls().len() - 1; + let mut remaining_agg_calls = Vec::with_capacity(estimated_len); + let mut approx_percentile_col_mapping = Vec::with_capacity(estimated_len); + let mut remaining_simple_agg_col_mapping = Vec::with_capacity(estimated_len); + for (output_idx, agg_call) in self.agg_calls().iter().enumerate() { + if agg_call.agg_kind == AggKind::ApproxPercentile { + approx_percentile_agg_call = Some(agg_call.clone()); + approx_percentile_col_mapping.push(Some(output_idx)); + } else { + remaining_agg_calls.push(agg_call.clone()); + remaining_simple_agg_col_mapping.push(Some(output_idx)); + } + } + ( + approx_percentile_agg_call.expect("ApproxPercentile agg call not found"), + remaining_agg_calls, + ColIndexMapping::new(approx_percentile_col_mapping, self.agg_calls().len()), + ColIndexMapping::new(remaining_simple_agg_col_mapping, self.agg_calls().len()), + ) + } + + fn build_approx_percentile_agg(&self, input: PlanRef) -> PlanRef { + let local_approx_percentile = StreamLocalApproxPercentile::new(input); + let global_approx_percentile = + StreamGlobalApproxPercentile::new(local_approx_percentile.into()); + global_approx_percentile.into() } pub fn core(&self) -> &Agg { @@ -653,6 +690,7 @@ impl LogicalAggBuilder { /// 2. Add the agg call to current `Agg`, and return an `InputRef` to it. /// /// Note that the rewriter does not traverse into inputs of agg calls. + /// FIXME(kwannoel): Rewrite the desc approx percentile for descending order. fn try_rewrite_agg_call(&mut self, mut agg_call: AggCall) -> Result { if matches!(agg_call.agg_kind, agg_kinds::must_have_order_by!()) && agg_call.order_by.sort_exprs.is_empty() From a55c666a072107599add66ee987987be3940ae20 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 21 Jul 2024 22:07:41 +0800 Subject: [PATCH 08/33] implement local approx percentile core parts --- .../src/optimizer/plan_node/generic/mod.rs | 3 + .../src/optimizer/plan_node/logical_agg.rs | 18 +++-- .../stream_global_approx_percentile.rs | 6 +- .../stream_local_approx_percentile.rs | 76 +++++++++++++++---- 4 files changed, 82 insertions(+), 21 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/generic/mod.rs b/src/frontend/src/optimizer/plan_node/generic/mod.rs index d83ab50d8923..3e01dee8aa0b 100644 --- a/src/frontend/src/optimizer/plan_node/generic/mod.rs +++ b/src/frontend/src/optimizer/plan_node/generic/mod.rs @@ -12,6 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +//! This module contains the generic plan nodes that are shared by all the plan nodes. +//! They are meant to reuse the common fields between logical, batch and stream nodes. + use std::borrow::Cow; use std::hash::Hash; diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 9a175bd87016..0af405d56b33 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -293,7 +293,8 @@ impl LogicalAgg { let shared_input = LogicalShare::new(stream_input).to_stream(ctx)?; let (approx_percentile_agg_call, non_approx_percentile_agg_calls, lhs_mapping, rhs_mapping) = self.extract_approx_percentile(); - let approx_percentile_agg = self.build_approx_percentile_agg(shared_input.clone()); + let approx_percentile_agg = + self.build_approx_percentile_agg(shared_input.clone(), &approx_percentile_agg_call); let simple_agg_without_approx_percentile = Agg::new( non_approx_percentile_agg_calls, self.group_key().clone(), @@ -337,10 +338,17 @@ impl LogicalAgg { ) } - fn build_approx_percentile_agg(&self, input: PlanRef) -> PlanRef { - let local_approx_percentile = StreamLocalApproxPercentile::new(input); - let global_approx_percentile = - StreamGlobalApproxPercentile::new(local_approx_percentile.into()); + fn build_approx_percentile_agg( + &self, + input: PlanRef, + approx_percentile_agg_call: &PlanAggCall, + ) -> PlanRef { + let local_approx_percentile = + StreamLocalApproxPercentile::new(input, &approx_percentile_agg_call); + let global_approx_percentile = StreamGlobalApproxPercentile::new( + local_approx_percentile.into(), + &approx_percentile_agg_call, + ); global_approx_percentile.into() } diff --git a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs index 9a0f4661e05e..b1b4087eeab5 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs @@ -20,8 +20,8 @@ use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::utils::{childless_record, watermark_pretty, Distill}; use crate::optimizer::plan_node::{ - ExprRewritable, PlanBase, PlanTreeNodeUnary, Stream, StreamHopWindow, StreamKeyedMerge, - StreamNode, + ExprRewritable, PlanAggCall, PlanBase, PlanTreeNodeUnary, Stream, StreamHopWindow, + StreamKeyedMerge, StreamNode, }; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::PlanRef; @@ -32,7 +32,7 @@ pub struct StreamGlobalApproxPercentile { } impl StreamGlobalApproxPercentile { - pub fn new(input: PlanRef) -> Self { + pub fn new(input: PlanRef, approx_percentile_agg_call: &PlanAggCall) -> Self { Self { base: todo!() } } } diff --git a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs index 24959eddbed4..620461d29321 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs @@ -12,15 +12,20 @@ // See the License for the specific language governing permissions and // limitations under the License. -use pretty_xmlish::XmlNode; +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::types::DataType; +use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; -use crate::expr::{ExprRewriter, ExprVisitor}; +use crate::expr::{ExprRewriter, ExprVisitor, InputRef, InputRefDisplay, Literal}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; -use crate::optimizer::plan_node::utils::Distill; +use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef}; +use crate::optimizer::plan_node::stream::StreamPlanRef; +use crate::optimizer::plan_node::utils::{childless_record, Distill, IndicesDisplay, watermark_pretty}; use crate::optimizer::plan_node::{ - ExprRewritable, PlanBase, PlanNode, PlanTreeNodeUnary, Stream, StreamGlobalApproxPercentile, - StreamHopWindow, StreamNode, + ExprRewritable, PlanAggCall, PlanBase, PlanNode, PlanTreeNodeUnary, Stream, + StreamGlobalApproxPercentile, StreamHopWindow, StreamNode, }; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::PlanRef; @@ -28,27 +33,72 @@ use crate::PlanRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamLocalApproxPercentile { pub base: PlanBase, + input: PlanRef, + quantile: Literal, + relative_error: Literal, + percentile_col: InputRef, + order_type: OrderType, } impl StreamLocalApproxPercentile { - pub fn new(input: PlanRef) -> Self { - Self { base: todo!() } + pub fn new(input: PlanRef, approx_percentile_agg_call: &PlanAggCall) -> Self { + let base = PlanBase::new_stream( + input.ctx(), + input.schema().clone(), + input.stream_key().map(|k| k.to_vec()), + input.functional_dependency().clone(), + input.distribution().clone(), + input.append_only(), + input.emit_on_window_close(), + input.watermark_columns().clone(), + input.columns_monotonicity().clone(), + ); + Self { + base, + input, + quantile: approx_percentile_agg_call.direct_args[0].clone(), + relative_error: approx_percentile_agg_call.direct_args[1].clone(), + percentile_col: approx_percentile_agg_call.inputs[0].clone(), + order_type: approx_percentile_agg_call.order_by[0].order_type, + } } } impl Distill for StreamLocalApproxPercentile { fn distill<'a>(&self) -> XmlNode<'a> { - todo!() + let mut out = Vec::with_capacity(4); + let output_type = DataType::Float64; + out.push(( + "percentile_col", + Pretty::display(&InputRefDisplay { + input_ref: &self.percentile_col, + input_schema: self.input.schema(), + }), + )); + out.push(("quantile", Pretty::debug(&self.quantile))); + out.push(("relative_error", Pretty::debug(&self.relative_error))); + out.push(("order_type", Pretty::display(&self.order_type))); + if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) { + out.push(("output_watermarks", ow)); + } + childless_record("StreamLocalApproxPercentile", out) } } impl PlanTreeNodeUnary for StreamLocalApproxPercentile { fn input(&self) -> PlanRef { - todo!() + self.input.clone() } fn clone_with_input(&self, input: PlanRef) -> Self { - todo!() + Self { + base: self.base.clone(), + input, + quantile: self.quantile.clone(), + relative_error: self.relative_error.clone(), + percentile_col: self.percentile_col.clone(), + order_type: self.order_type, + } } } @@ -62,16 +112,16 @@ impl StreamNode for StreamLocalApproxPercentile { impl ExprRewritable for StreamLocalApproxPercentile { fn has_rewritable_expr(&self) -> bool { - todo!() + false } fn rewrite_exprs(&self, _rewriter: &mut dyn ExprRewriter) -> PlanRef { - todo!() + unimplemented!() } } impl ExprVisitable for StreamLocalApproxPercentile { fn visit_exprs(&self, v: &mut dyn ExprVisitor) { - todo!() + unimplemented!() } } From 84b04b226dc51a7d62d6cdae1f36121c650f0508 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 21 Jul 2024 23:11:48 +0800 Subject: [PATCH 09/33] fix local approx percentile schema --- src/common/src/catalog/schema.rs | 9 +++++++++ .../plan_node/stream_global_approx_percentile.rs | 6 +++++- .../plan_node/stream_local_approx_percentile.rs | 12 ++++++++++-- 3 files changed, 24 insertions(+), 3 deletions(-) diff --git a/src/common/src/catalog/schema.rs b/src/common/src/catalog/schema.rs index 113d9f804b3d..e4da4b48ddf5 100644 --- a/src/common/src/catalog/schema.rs +++ b/src/common/src/catalog/schema.rs @@ -47,6 +47,15 @@ impl Field { name: self.name.to_string(), } } + + pub fn new(name: impl Into, data_type: DataType) -> Self { + Self { + data_type, + name: name.into(), + sub_fields: vec![], + type_name: String::new(), + } + } } impl From<&ColumnDesc> for Field { diff --git a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs index b1b4087eeab5..bfb07b2d75d5 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs @@ -29,11 +29,15 @@ use crate::PlanRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamGlobalApproxPercentile { pub base: PlanBase, + input: PlanRef, } impl StreamGlobalApproxPercentile { pub fn new(input: PlanRef, approx_percentile_agg_call: &PlanAggCall) -> Self { - Self { base: todo!() } + Self { + base: todo!(), + input, + } } } diff --git a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs index 620461d29321..fefa47b29b7d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs @@ -22,7 +22,9 @@ use crate::expr::{ExprRewriter, ExprVisitor, InputRef, InputRefDisplay, Literal} use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef}; use crate::optimizer::plan_node::stream::StreamPlanRef; -use crate::optimizer::plan_node::utils::{childless_record, Distill, IndicesDisplay, watermark_pretty}; +use crate::optimizer::plan_node::utils::{ + childless_record, watermark_pretty, Distill, IndicesDisplay, +}; use crate::optimizer::plan_node::{ ExprRewritable, PlanAggCall, PlanBase, PlanNode, PlanTreeNodeUnary, Stream, StreamGlobalApproxPercentile, StreamHopWindow, StreamNode, @@ -30,6 +32,8 @@ use crate::optimizer::plan_node::{ use crate::stream_fragmenter::BuildFragmentGraphState; use crate::PlanRef; +// Does not contain `core` because no other plan nodes share +// common fields and schema, even GlobalApproxPercentile. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamLocalApproxPercentile { pub base: PlanBase, @@ -42,9 +46,13 @@ pub struct StreamLocalApproxPercentile { impl StreamLocalApproxPercentile { pub fn new(input: PlanRef, approx_percentile_agg_call: &PlanAggCall) -> Self { + let schema = Schema::new(vec![ + Field::new("bucket_id", DataType::Int64), + Field::new("count", DataType::Int64), + ]); let base = PlanBase::new_stream( input.ctx(), - input.schema().clone(), + schema, input.stream_key().map(|k| k.to_vec()), input.functional_dependency().clone(), input.distribution().clone(), From 477c5fc9017276dc45d07ec9bc4bea8388a6fdfd Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 21 Jul 2024 23:19:39 +0800 Subject: [PATCH 10/33] handle global approx percentile --- .../stream_global_approx_percentile.rs | 37 +++++++++++++------ .../stream_local_approx_percentile.rs | 3 +- 2 files changed, 27 insertions(+), 13 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs index bfb07b2d75d5..4729bddfd1e0 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs @@ -13,11 +13,14 @@ // limitations under the License. use pretty_xmlish::XmlNode; +use risingwave_common::catalog::{Field, Schema}; +use risingwave_common::types::DataType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; -use crate::optimizer::plan_node::generic::GenericPlanRef; +use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef}; +use crate::optimizer::plan_node::stream::StreamPlanRef; use crate::optimizer::plan_node::utils::{childless_record, watermark_pretty, Distill}; use crate::optimizer::plan_node::{ ExprRewritable, PlanAggCall, PlanBase, PlanTreeNodeUnary, Stream, StreamHopWindow, @@ -34,26 +37,38 @@ pub struct StreamGlobalApproxPercentile { impl StreamGlobalApproxPercentile { pub fn new(input: PlanRef, approx_percentile_agg_call: &PlanAggCall) -> Self { - Self { - base: todo!(), - input, - } + let schema = Schema::new(vec![Field::new("approx_percentile", DataType::Float64)]); + let base = PlanBase::new_stream( + input.ctx(), + schema, + input.stream_key().map(|k| k.to_vec()), + input.functional_dependency().clone(), + input.distribution().clone(), + input.append_only(), + input.emit_on_window_close(), + input.watermark_columns().clone(), + input.columns_monotonicity().clone(), + ); + Self { base, input } } } impl Distill for StreamGlobalApproxPercentile { fn distill<'a>(&self) -> XmlNode<'a> { - todo!() + childless_record("StreamGlobalApproxPercentile", vec![]) } } impl PlanTreeNodeUnary for StreamGlobalApproxPercentile { fn input(&self) -> PlanRef { - todo!() + self.input.clone() } fn clone_with_input(&self, input: PlanRef) -> Self { - todo!() + Self { + base: self.base.clone(), + input, + } } } @@ -67,16 +82,16 @@ impl StreamNode for StreamGlobalApproxPercentile { impl ExprRewritable for StreamGlobalApproxPercentile { fn has_rewritable_expr(&self) -> bool { - todo!() + false } fn rewrite_exprs(&self, _rewriter: &mut dyn ExprRewriter) -> PlanRef { - todo!() + unimplemented!() } } impl ExprVisitable for StreamGlobalApproxPercentile { fn visit_exprs(&self, v: &mut dyn ExprVisitor) { - todo!() + unimplemented!() } } diff --git a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs index fefa47b29b7d..30ee60fcd322 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs @@ -74,8 +74,7 @@ impl StreamLocalApproxPercentile { impl Distill for StreamLocalApproxPercentile { fn distill<'a>(&self) -> XmlNode<'a> { - let mut out = Vec::with_capacity(4); - let output_type = DataType::Float64; + let mut out = Vec::with_capacity(5); out.push(( "percentile_col", Pretty::display(&InputRefDisplay { From f4acd015836e2736a4e7dda6e51a52f7087990f0 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 21 Jul 2024 23:29:45 +0800 Subject: [PATCH 11/33] implement stream keyed merge --- .../src/optimizer/plan_node/logical_agg.rs | 8 ++- .../optimizer/plan_node/stream_keyed_merge.rs | 58 +++++++++++++++---- 2 files changed, 52 insertions(+), 14 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 0af405d56b33..b19c78b90165 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -303,8 +303,12 @@ impl LogicalAgg { .with_grouping_sets(self.grouping_sets().clone()) .with_enable_two_phase(self.core().enable_two_phase) .into(); - let agg_merge = - StreamKeyedMerge::new(simple_agg_without_approx_percentile, approx_percentile_agg); + let agg_merge = StreamKeyedMerge::new( + simple_agg_without_approx_percentile, + approx_percentile_agg, + lhs_mapping, + rhs_mapping, + ); Ok(agg_merge.into()) } diff --git a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs index ed726dda3bef..c917f0c5522b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs +++ b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs @@ -12,13 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -use pretty_xmlish::XmlNode; +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::HopWindowNode; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; -use crate::optimizer::plan_node::utils::Distill; +use crate::optimizer::plan_node::generic::PhysicalPlanRef; +use crate::optimizer::plan_node::stream::StreamPlanRef; +use crate::optimizer::plan_node::utils::{childless_record, Distill}; use crate::optimizer::plan_node::{ ExprRewritable, PlanBase, PlanTreeNodeBinary, Stream, StreamHopWindow, StreamLocalApproxPercentile, StreamNode, @@ -28,38 +31,69 @@ use crate::PlanRef; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamKeyedMerge { + pub base: PlanBase, pub lhs_input: PlanRef, pub rhs_input: PlanRef, - pub base: PlanBase, + /// Maps input from the lhs to the output. + pub lhs_mapping: ColIndexMapping, + /// Maps input from the rhs to the output. + pub rhs_mapping: ColIndexMapping, } impl StreamKeyedMerge { - pub fn new(lhs_input: PlanRef, rhs_input: PlanRef) -> Self { + pub fn new( + lhs_input: PlanRef, + rhs_input: PlanRef, + lhs_mapping: ColIndexMapping, + rhs_mapping: ColIndexMapping, + ) -> Self { + let base = PlanBase::new_stream( + lhs_input.ctx(), + lhs_input.schema().clone(), + lhs_input.stream_key().map(|k| k.to_vec()), + lhs_input.functional_dependency().clone(), + lhs_input.distribution().clone(), + lhs_input.append_only(), + lhs_input.emit_on_window_close(), + lhs_input.watermark_columns().clone(), + lhs_input.columns_monotonicity().clone(), + ); Self { + base, lhs_input, rhs_input, - base: todo!(), + lhs_mapping, + rhs_mapping, } } } impl Distill for StreamKeyedMerge { fn distill<'a>(&self) -> XmlNode<'a> { - todo!() + let mut out = Vec::with_capacity(2); + out.push(("lhs_col_mapping", Pretty::debug(&self.lhs_mapping))); + out.push(("rhs_col_mapping", Pretty::debug(&self.rhs_mapping))); + childless_record("StreamKeyedMerge", out) } } impl PlanTreeNodeBinary for StreamKeyedMerge { fn left(&self) -> PlanRef { - todo!() + self.lhs_input.clone() } fn right(&self) -> PlanRef { - todo!() + self.rhs_input.clone() } fn clone_with_left_right(&self, left: PlanRef, right: PlanRef) -> Self { - todo!() + Self { + base: self.base.clone(), + lhs_input: left, + rhs_input: right, + lhs_mapping: self.lhs_mapping.clone(), + rhs_mapping: self.rhs_mapping.clone(), + } } } @@ -73,16 +107,16 @@ impl StreamNode for StreamKeyedMerge { impl ExprRewritable for StreamKeyedMerge { fn has_rewritable_expr(&self) -> bool { - todo!() + false } fn rewrite_exprs(&self, _rewriter: &mut dyn ExprRewriter) -> PlanRef { - todo!() + unimplemented!() } } impl ExprVisitable for StreamKeyedMerge { fn visit_exprs(&self, v: &mut dyn ExprVisitor) { - todo!() + unimplemented!() } } From 1a5d098fb1d1c103af82b3be0fea2ec836b96f29 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Sun, 21 Jul 2024 23:42:06 +0800 Subject: [PATCH 12/33] fix stream share --- src/expr/core/src/aggregate/def.rs | 1 + src/frontend/planner_test/tests/testdata/input/agg.yaml | 3 ++- src/frontend/planner_test/tests/testdata/output/agg.yaml | 7 +++++++ src/frontend/src/optimizer/plan_node/logical_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_share.rs | 9 +++++++++ 5 files changed, 20 insertions(+), 2 deletions(-) diff --git a/src/expr/core/src/aggregate/def.rs b/src/expr/core/src/aggregate/def.rs index 1fb985585cdf..2e17c0fe3a81 100644 --- a/src/expr/core/src/aggregate/def.rs +++ b/src/expr/core/src/aggregate/def.rs @@ -339,6 +339,7 @@ pub mod agg_kinds { | PbAggKind::Count | PbAggKind::Avg | PbAggKind::ApproxCountDistinct + | PbAggKind::ApproxPercentile | PbAggKind::VarPop | PbAggKind::VarSamp | PbAggKind::StddevPop diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index 7318e21b50ea..6ec3b6d3706d 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -1004,4 +1004,5 @@ CREATE TABLE t (v1 int); SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) from t; expected_outputs: - - logical_plan \ No newline at end of file + - logical_plan + - stream_plan \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index cc4cee22cb94..f4f170d0ba1b 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1871,3 +1871,10 @@ └─LogicalAgg { aggs: [approx_percentile($expr1 order_by(t.v1 ASC))] } └─LogicalProject { exprs: [t.v1::Float64 as $expr1, t.v1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + stream_plan: |- + StreamMaterialize { columns: [approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamProject { exprs: [approx_percentile($expr1 order_by(t.v1 ASC))] } + └─StreamSimpleAgg { aggs: [approx_percentile($expr1 order_by(t.v1 ASC)), count] } + └─StreamExchange { dist: Single } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index b19c78b90165..bb58d172fbf4 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -290,7 +290,7 @@ impl LogicalAgg { .into()); } - let shared_input = LogicalShare::new(stream_input).to_stream(ctx)?; + let shared_input: PlanRef = StreamShare::new_from_input(stream_input).into(); let (approx_percentile_agg_call, non_approx_percentile_agg_calls, lhs_mapping, rhs_mapping) = self.extract_approx_percentile(); let approx_percentile_agg = diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index b082d82b022d..7e6f87fa5c27 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::cell::RefCell; + use pretty_xmlish::XmlNode; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::PbStreamNode; @@ -50,6 +52,13 @@ impl StreamShare { StreamShare { base, core } } + + pub fn new_from_input(input: PlanRef) -> Self { + let core = generic::Share { + input: RefCell::new(input), + }; + Self::new(core) + } } impl Distill for StreamShare { From 3f6b9f7ef3b267313aa56f7b56ccec22f25837a6 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 22 Jul 2024 02:13:27 +0800 Subject: [PATCH 13/33] test case of approx_percentile alone --- .../tests/testdata/output/agg.yaml | 16 ++-- .../src/optimizer/plan_node/generic/agg.rs | 1 + .../src/optimizer/plan_node/logical_agg.rs | 76 +++++++++++++++---- .../stream_global_approx_percentile.rs | 7 +- .../optimizer/plan_node/stream_keyed_merge.rs | 1 + .../stream_local_approx_percentile.rs | 10 +-- 6 files changed, 81 insertions(+), 30 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index f4f170d0ba1b..05bf361e8849 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1867,14 +1867,14 @@ CREATE TABLE t (v1 int); SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) from t; logical_plan: |- - LogicalProject { exprs: [approx_percentile($expr1 order_by(t.v1 ASC))] } - └─LogicalAgg { aggs: [approx_percentile($expr1 order_by(t.v1 ASC))] } - └─LogicalProject { exprs: [t.v1::Float64 as $expr1, t.v1] } + LogicalProject { exprs: [approx_percentile($expr1)] } + └─LogicalAgg { aggs: [approx_percentile($expr1)] } + └─LogicalProject { exprs: [t.v1::Float64 as $expr1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: |- - StreamMaterialize { columns: [approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [approx_percentile($expr1 order_by(t.v1 ASC))] } - └─StreamSimpleAgg { aggs: [approx_percentile($expr1 order_by(t.v1 ASC)), count] } - └─StreamExchange { dist: Single } - └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } + StreamMaterialize { columns: [approx_percentile], stream_key: [approx_percentile], pk_columns: [approx_percentile], pk_conflict: NoCheck } + └─StreamExchange { dist: HashShard(approx_percentile) } + └─StreamGlobalApproxPercentile + └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index a085e46e12ef..14193b378376 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -133,6 +133,7 @@ impl Agg { self.agg_calls.iter().all(|c| { matches!(c.agg_kind, agg_kinds::single_value_state!()) || (matches!(c.agg_kind, agg_kinds::single_value_state_iff_in_append_only!() if stream_input_append_only)) + || (matches!(c.agg_kind, AggKind::ApproxPercentile)) }) } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index bb58d172fbf4..b43feb017a71 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -61,9 +61,39 @@ impl LogicalAgg { /// Generate plan for stateless 2-phase streaming agg. /// Should only be used iff input is distributed. Input must be converted to stream form. fn gen_stateless_two_phase_streaming_agg_plan(&self, stream_input: PlanRef) -> Result { + println!("generating stateless simple agg plan"); debug_assert!(self.group_key().is_empty()); let mut core = self.core.clone(); - core.input = stream_input; + + // First, handle approx percentile. + let has_approx_percentile = self + .agg_calls() + .iter() + .any(|agg_call| agg_call.agg_kind == AggKind::ApproxPercentile); + let approx_percentile_info = if has_approx_percentile { + let ( + approx_percentile_agg_call, + non_approx_percentile_agg_calls, + lhs_mapping, + rhs_mapping, + ) = self.extract_approx_percentile(); + if non_approx_percentile_agg_calls.is_empty() { + let approx_percentile_agg: PlanRef = + self.build_approx_percentile_agg(stream_input.clone(), &approx_percentile_agg_call); + return Ok(approx_percentile_agg); + } else { + let stream_input: PlanRef = StreamShare::new_from_input(stream_input).into(); + let approx_percentile_agg: PlanRef = + self.build_approx_percentile_agg(stream_input.clone(), &approx_percentile_agg_call); + + core.input = stream_input; + core.agg_calls = non_approx_percentile_agg_calls; + Some((approx_percentile_agg, lhs_mapping, rhs_mapping)) + } + } else { + core.input = stream_input; + None + }; let local_agg = StreamStatelessSimpleAgg::new(core); let exchange = RequiredDist::single().enforce_if_not_satisfies(local_agg.into(), &Order::any())?; @@ -78,7 +108,17 @@ impl LogicalAgg { IndexSet::empty(), exchange, )); - Ok(global_agg.into()) + if let Some((approx_percentile_agg, lhs_mapping, rhs_mapping)) = approx_percentile_info { + let keyed_merge = StreamKeyedMerge::new( + global_agg.into(), + approx_percentile_agg, + lhs_mapping, + rhs_mapping, + ); + Ok(keyed_merge.into()) + } else { + Ok(global_agg.into()) + } } /// Generate plan for stateless/stateful 2-phase streaming agg. @@ -198,15 +238,14 @@ impl LogicalAgg { } else { self.core.can_two_phase_agg() }); - - // Handle 2-phase approx_percentile aggregation. - if self - .agg_calls() - .iter() - .any(|agg_call| agg_call.agg_kind == AggKind::ApproxPercentile) - { - return self.gen_approx_percentile_plan(stream_input, ctx); - } + // // Handle 2-phase approx_percentile aggregation. + // if self + // .agg_calls() + // .iter() + // .any(|agg_call| agg_call.agg_kind == AggKind::ApproxPercentile) + // { + // return self.gen_two_phase_approx_percentile_plan(stream_input, ctx); + // } // Stateless 2-phase simple agg // can be applied on stateless simple agg calls, @@ -277,7 +316,7 @@ impl LogicalAgg { // for the agg step, we need to count by bucket id, and not just dispatch a single record // downstream. So we need a custom executor for this logic. // TODO: Ban distinct agg? - fn gen_approx_percentile_plan( + fn gen_two_phase_approx_percentile_plan( &self, stream_input: PlanRef, ctx: &mut ToStreamContext, @@ -295,7 +334,7 @@ impl LogicalAgg { self.extract_approx_percentile(); let approx_percentile_agg = self.build_approx_percentile_agg(shared_input.clone(), &approx_percentile_agg_call); - let simple_agg_without_approx_percentile = Agg::new( + let simple_agg_without_approx_percentile: PlanRef = Agg::new( non_approx_percentile_agg_calls, self.group_key().clone(), shared_input, @@ -303,6 +342,8 @@ impl LogicalAgg { .with_grouping_sets(self.grouping_sets().clone()) .with_enable_two_phase(self.core().enable_two_phase) .into(); + let simple_agg_without_approx_percentile = + simple_agg_without_approx_percentile.to_stream(ctx)?; let agg_merge = StreamKeyedMerge::new( simple_agg_without_approx_percentile, approx_percentile_agg, @@ -1245,6 +1286,15 @@ impl ToStream for LogicalAgg { }, final_agg.agg_calls().len(), ) + } else if let Some(approx_percentile_agg) = plan.as_stream_global_approx_percentile() { + if eowc { + return Err(ErrorCode::InvalidInputSyntax( + "`EMIT ON WINDOW CLOSE` cannot be used for aggregation without `GROUP BY`" + .to_string(), + ) + .into()); + } + (plan.clone(), 1) } else { panic!("the root PlanNode must be either StreamHashAgg or StreamSimpleAgg"); }; diff --git a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs index 4729bddfd1e0..62d93138856b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use fixedbitset::FixedBitSet; use pretty_xmlish::XmlNode; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::DataType; @@ -38,15 +39,16 @@ pub struct StreamGlobalApproxPercentile { impl StreamGlobalApproxPercentile { pub fn new(input: PlanRef, approx_percentile_agg_call: &PlanAggCall) -> Self { let schema = Schema::new(vec![Field::new("approx_percentile", DataType::Float64)]); + let watermark_columns = FixedBitSet::with_capacity(1); let base = PlanBase::new_stream( input.ctx(), schema, - input.stream_key().map(|k| k.to_vec()), + Some(vec![0]), input.functional_dependency().clone(), input.distribution().clone(), input.append_only(), input.emit_on_window_close(), - input.watermark_columns().clone(), + watermark_columns, input.columns_monotonicity().clone(), ); Self { base, input } @@ -92,6 +94,5 @@ impl ExprRewritable for StreamGlobalApproxPercentile { impl ExprVisitable for StreamGlobalApproxPercentile { fn visit_exprs(&self, v: &mut dyn ExprVisitor) { - unimplemented!() } } diff --git a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs index c917f0c5522b..493b10400e03 100644 --- a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs +++ b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs @@ -47,6 +47,7 @@ impl StreamKeyedMerge { lhs_mapping: ColIndexMapping, rhs_mapping: ColIndexMapping, ) -> Self { + // FIXME: schema is wrong. let base = PlanBase::new_stream( lhs_input.ctx(), lhs_input.schema().clone(), diff --git a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs index 30ee60fcd322..64a355c471a7 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use fixedbitset::FixedBitSet; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::DataType; @@ -41,7 +42,6 @@ pub struct StreamLocalApproxPercentile { quantile: Literal, relative_error: Literal, percentile_col: InputRef, - order_type: OrderType, } impl StreamLocalApproxPercentile { @@ -50,6 +50,8 @@ impl StreamLocalApproxPercentile { Field::new("bucket_id", DataType::Int64), Field::new("count", DataType::Int64), ]); + // FIXME(kwannoel): How does watermark work with FixedBitSet + let watermark_columns = FixedBitSet::with_capacity(2); let base = PlanBase::new_stream( input.ctx(), schema, @@ -58,7 +60,7 @@ impl StreamLocalApproxPercentile { input.distribution().clone(), input.append_only(), input.emit_on_window_close(), - input.watermark_columns().clone(), + watermark_columns, input.columns_monotonicity().clone(), ); Self { @@ -67,7 +69,6 @@ impl StreamLocalApproxPercentile { quantile: approx_percentile_agg_call.direct_args[0].clone(), relative_error: approx_percentile_agg_call.direct_args[1].clone(), percentile_col: approx_percentile_agg_call.inputs[0].clone(), - order_type: approx_percentile_agg_call.order_by[0].order_type, } } } @@ -84,7 +85,6 @@ impl Distill for StreamLocalApproxPercentile { )); out.push(("quantile", Pretty::debug(&self.quantile))); out.push(("relative_error", Pretty::debug(&self.relative_error))); - out.push(("order_type", Pretty::display(&self.order_type))); if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) { out.push(("output_watermarks", ow)); } @@ -104,7 +104,6 @@ impl PlanTreeNodeUnary for StreamLocalApproxPercentile { quantile: self.quantile.clone(), relative_error: self.relative_error.clone(), percentile_col: self.percentile_col.clone(), - order_type: self.order_type, } } } @@ -129,6 +128,5 @@ impl ExprRewritable for StreamLocalApproxPercentile { impl ExprVisitable for StreamLocalApproxPercentile { fn visit_exprs(&self, v: &mut dyn ExprVisitor) { - unimplemented!() } } From 85adca28bc10761ae3e4d9176e8a3c2616ba74c5 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 22 Jul 2024 13:43:18 +0800 Subject: [PATCH 14/33] add tests for stateless normal agg + approx percentile agg, fix distribution of global agg --- .../tests/testdata/input/agg.yaml | 17 ++- .../tests/testdata/output/agg.yaml | 54 ++++++++- .../src/optimizer/plan_node/logical_agg.rs | 103 ++++++------------ .../stream_global_approx_percentile.rs | 6 +- .../optimizer/plan_node/stream_keyed_merge.rs | 23 ++-- .../stream_local_approx_percentile.rs | 3 +- 6 files changed, 117 insertions(+), 89 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index 6ec3b6d3706d..9109b31e81a7 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -1000,9 +1000,24 @@ expected_outputs: - batch_plan - stream_plan -- sql: | +- name: test simple approx_percentile alone + sql: | CREATE TABLE t (v1 int); SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) from t; + expected_outputs: + - logical_plan + - stream_plan +- name: test simple approx_percentile with other simple aggs + sql: | + CREATE TABLE t (v1 int); + SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1), sum(v1) from t; + expected_outputs: + - logical_plan + - stream_plan +- name: test simple approx_percentile with other simple aggs (sum, count) + sql: | + CREATE TABLE t (v1 int); + SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1), sum(v1), count(v1) from t; expected_outputs: - logical_plan - stream_plan \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 05bf361e8849..1cce4e53f67e 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1863,7 +1863,8 @@ └─StreamHashAgg { group_key: [t.a, t.b], aggs: [sum(t.c), sum(t.d), count(t.d), max(t.e), count] } └─StreamExchange { dist: HashShard(t.a, t.b) } └─StreamTableScan { table: t, columns: [t.a, t.b, t.c, t.d, t.e, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } -- sql: | +- name: test simple approx_percentile alone + sql: | CREATE TABLE t (v1 int); SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) from t; logical_plan: |- @@ -1873,8 +1874,53 @@ └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: |- StreamMaterialize { columns: [approx_percentile], stream_key: [approx_percentile], pk_columns: [approx_percentile], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(approx_percentile) } + └─StreamGlobalApproxPercentile + └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } +- name: test simple approx_percentile with other simple aggs + sql: | + CREATE TABLE t (v1 int); + SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1), sum(v1) from t; + logical_plan: |- + LogicalProject { exprs: [approx_percentile($expr1), sum(t.v1)] } + └─LogicalAgg { aggs: [approx_percentile($expr1), sum(t.v1)] } + └─LogicalProject { exprs: [t.v1::Float64 as $expr1, t.v1] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + stream_plan: |- + StreamMaterialize { columns: [approx_percentile, sum], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamKeyedMerge { output: [approx_percentile($expr10011):Float64, sum(t.v1):Int64] } + ├─StreamSimpleAgg { aggs: [sum(sum(t.v1)), count] } + │ └─StreamExchange { dist: Single } + │ └─StreamStatelessSimpleAgg { aggs: [sum(t.v1)] } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamGlobalApproxPercentile └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } - └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamShare { id: 2 } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } +- name: test simple approx_percentile with other simple aggs (sum, count) + sql: | + CREATE TABLE t (v1 int); + SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1), sum(v1), count(v1) from t; + logical_plan: |- + LogicalProject { exprs: [approx_percentile($expr1), sum(t.v1), count(t.v1)] } + └─LogicalAgg { aggs: [approx_percentile($expr1), sum(t.v1), count(t.v1)] } + └─LogicalProject { exprs: [t.v1::Float64 as $expr1, t.v1] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + stream_plan: |- + StreamMaterialize { columns: [approx_percentile, sum, count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamKeyedMerge { output: [approx_percentile($expr10011):Float64, sum(t.v1):Int64, count(t.v1):Int64] } + ├─StreamSimpleAgg { aggs: [sum(sum(t.v1)), sum0(count(t.v1)), count] } + │ └─StreamExchange { dist: Single } + │ └─StreamStatelessSimpleAgg { aggs: [sum(t.v1), count(t.v1)] } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamGlobalApproxPercentile + └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + └─StreamShare { id: 2 } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index b43feb017a71..29de1b6b9621 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -61,9 +61,10 @@ impl LogicalAgg { /// Generate plan for stateless 2-phase streaming agg. /// Should only be used iff input is distributed. Input must be converted to stream form. fn gen_stateless_two_phase_streaming_agg_plan(&self, stream_input: PlanRef) -> Result { - println!("generating stateless simple agg plan"); debug_assert!(self.group_key().is_empty()); let mut core = self.core.clone(); + let schema = self.base.schema().clone(); + println!("agg schema: {:?}", schema); // First, handle approx percentile. let has_approx_percentile = self @@ -77,34 +78,37 @@ impl LogicalAgg { lhs_mapping, rhs_mapping, ) = self.extract_approx_percentile(); - if non_approx_percentile_agg_calls.is_empty() { - let approx_percentile_agg: PlanRef = - self.build_approx_percentile_agg(stream_input.clone(), &approx_percentile_agg_call); - return Ok(approx_percentile_agg); - } else { + // FIXME(kwannoel): Handle case where there's multiple approx percentile also. + let requires_keyed_merge = !non_approx_percentile_agg_calls.is_empty(); + if requires_keyed_merge { let stream_input: PlanRef = StreamShare::new_from_input(stream_input).into(); - let approx_percentile_agg: PlanRef = - self.build_approx_percentile_agg(stream_input.clone(), &approx_percentile_agg_call); + let approx_percentile_agg: PlanRef = self + .build_approx_percentile_agg(stream_input.clone(), &approx_percentile_agg_call); core.input = stream_input; core.agg_calls = non_approx_percentile_agg_calls; Some((approx_percentile_agg, lhs_mapping, rhs_mapping)) + } else { + let approx_percentile_agg: PlanRef = self + .build_approx_percentile_agg(stream_input.clone(), &approx_percentile_agg_call); + return Ok(approx_percentile_agg); } } else { core.input = stream_input; None }; + let total_agg_calls = core.agg_calls + .iter() + .enumerate() + .map(|(partial_output_idx, agg_call)| { + agg_call.partial_to_total_agg_call(partial_output_idx) + }) + .collect_vec(); let local_agg = StreamStatelessSimpleAgg::new(core); let exchange = RequiredDist::single().enforce_if_not_satisfies(local_agg.into(), &Order::any())?; let global_agg = new_stream_simple_agg(Agg::new( - self.agg_calls() - .iter() - .enumerate() - .map(|(partial_output_idx, agg_call)| { - agg_call.partial_to_total_agg_call(partial_output_idx) - }) - .collect(), + total_agg_calls, IndexSet::empty(), exchange, )); @@ -114,6 +118,7 @@ impl LogicalAgg { approx_percentile_agg, lhs_mapping, rhs_mapping, + schema, ); Ok(keyed_merge.into()) } else { @@ -297,62 +302,7 @@ impl LogicalAgg { } } - /// ------- REWRITES ------- - /// - /// SimpleAgg { agg_calls: [count(*), approx_percentile(quantile, relative_error, v1), sum(v1)] } - /// -> Input - /// - /// -------- INTO ---------- - /// - /// KeyedMerge { agg_calls: [count(*), approx_percentile(quantile, relative_error, v1), sum(v1)] } - /// -> SimpleAgg { agg_calls: [count(*)] } - /// -> StreamShare { id: 0 } - /// -> Input - /// -> GlobalApproxPercentile { args: [bucket_id, count] } - /// -> LocalApproxPercentile { args: [relative_error, ] } - /// -> StreamShare { id: 0 } - /// -> Input - // NOTE(kwannoel): We can't split LocalApproxPercentile into Project + Agg, because - // for the agg step, we need to count by bucket id, and not just dispatch a single record - // downstream. So we need a custom executor for this logic. - // TODO: Ban distinct agg? - fn gen_two_phase_approx_percentile_plan( - &self, - stream_input: PlanRef, - ctx: &mut ToStreamContext, - ) -> Result { - if !self.group_key().is_empty() { - return Err(ErrorCode::NotSupported( - "ApproxPercentile aggregation with GROUP BY is not supported".into(), - "try to remove GROUP BY".into(), - ) - .into()); - } - - let shared_input: PlanRef = StreamShare::new_from_input(stream_input).into(); - let (approx_percentile_agg_call, non_approx_percentile_agg_calls, lhs_mapping, rhs_mapping) = - self.extract_approx_percentile(); - let approx_percentile_agg = - self.build_approx_percentile_agg(shared_input.clone(), &approx_percentile_agg_call); - let simple_agg_without_approx_percentile: PlanRef = Agg::new( - non_approx_percentile_agg_calls, - self.group_key().clone(), - shared_input, - ) - .with_grouping_sets(self.grouping_sets().clone()) - .with_enable_two_phase(self.core().enable_two_phase) - .into(); - let simple_agg_without_approx_percentile = - simple_agg_without_approx_percentile.to_stream(ctx)?; - let agg_merge = StreamKeyedMerge::new( - simple_agg_without_approx_percentile, - approx_percentile_agg, - lhs_mapping, - rhs_mapping, - ); - Ok(agg_merge.into()) - } - + // TODO(kwannoel): Handle multiple approx_percentile. fn extract_approx_percentile( &self, ) -> ( @@ -1295,8 +1245,17 @@ impl ToStream for LogicalAgg { .into()); } (plan.clone(), 1) + } else if let Some(stream_keyed_merge) = plan.as_stream_keyed_merge() { + if eowc { + return Err(ErrorCode::InvalidInputSyntax( + "`EMIT ON WINDOW CLOSE` cannot be used for aggregation without `GROUP BY`" + .to_string(), + ) + .into()); + } + (plan.clone(), stream_keyed_merge.base.schema().len()) } else { - panic!("the root PlanNode must be either StreamHashAgg or StreamSimpleAgg"); + panic!("the root PlanNode must be StreamHashAgg, StreamSimpleAgg, StreamGlobalApproxPercentile, or StreamKeyedMerge"); }; if self.agg_calls().len() == n_final_agg_calls { diff --git a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs index 62d93138856b..a165d6991033 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs @@ -27,6 +27,7 @@ use crate::optimizer::plan_node::{ ExprRewritable, PlanAggCall, PlanBase, PlanTreeNodeUnary, Stream, StreamHopWindow, StreamKeyedMerge, StreamNode, }; +use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::PlanRef; @@ -45,7 +46,7 @@ impl StreamGlobalApproxPercentile { schema, Some(vec![0]), input.functional_dependency().clone(), - input.distribution().clone(), + Distribution::Single, input.append_only(), input.emit_on_window_close(), watermark_columns, @@ -93,6 +94,5 @@ impl ExprRewritable for StreamGlobalApproxPercentile { } impl ExprVisitable for StreamGlobalApproxPercentile { - fn visit_exprs(&self, v: &mut dyn ExprVisitor) { - } + fn visit_exprs(&self, v: &mut dyn ExprVisitor) {} } diff --git a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs index 493b10400e03..86034a3b3889 100644 --- a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs +++ b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs @@ -13,15 +13,16 @@ // limitations under the License. use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::catalog::Schema; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::HopWindowNode; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; -use crate::optimizer::plan_node::generic::PhysicalPlanRef; +use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef}; use crate::optimizer::plan_node::stream::StreamPlanRef; -use crate::optimizer::plan_node::utils::{childless_record, Distill}; +use crate::optimizer::plan_node::utils::{childless_record, Distill, IndicesDisplay}; use crate::optimizer::plan_node::{ ExprRewritable, PlanBase, PlanTreeNodeBinary, Stream, StreamHopWindow, StreamLocalApproxPercentile, StreamNode, @@ -46,11 +47,13 @@ impl StreamKeyedMerge { rhs_input: PlanRef, lhs_mapping: ColIndexMapping, rhs_mapping: ColIndexMapping, + schema: Schema, ) -> Self { + println!("keyed merge schema: {:?}", schema); // FIXME: schema is wrong. let base = PlanBase::new_stream( lhs_input.ctx(), - lhs_input.schema().clone(), + schema, lhs_input.stream_key().map(|k| k.to_vec()), lhs_input.functional_dependency().clone(), lhs_input.distribution().clone(), @@ -71,10 +74,17 @@ impl StreamKeyedMerge { impl Distill for StreamKeyedMerge { fn distill<'a>(&self) -> XmlNode<'a> { - let mut out = Vec::with_capacity(2); - out.push(("lhs_col_mapping", Pretty::debug(&self.lhs_mapping))); - out.push(("rhs_col_mapping", Pretty::debug(&self.rhs_mapping))); + let mut out = Vec::with_capacity(1); + // out.push(("lhs_col_mapping", Pretty::debug(&self.lhs_mapping))); + // out.push(("rhs_col_mapping", Pretty::debug(&self.rhs_mapping))); + + if self.base.ctx().is_explain_verbose() { + let f = |t| Pretty::debug(&t); + let e = Pretty::Array(self.base.schema().fields().iter().map(f).collect()); + out = vec![("output", e)]; + } childless_record("StreamKeyedMerge", out) + } } @@ -118,6 +128,5 @@ impl ExprRewritable for StreamKeyedMerge { impl ExprVisitable for StreamKeyedMerge { fn visit_exprs(&self, v: &mut dyn ExprVisitor) { - unimplemented!() } } diff --git a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs index 64a355c471a7..21a44677d936 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs @@ -127,6 +127,5 @@ impl ExprRewritable for StreamLocalApproxPercentile { } impl ExprVisitable for StreamLocalApproxPercentile { - fn visit_exprs(&self, v: &mut dyn ExprVisitor) { - } + fn visit_exprs(&self, v: &mut dyn ExprVisitor) {} } From a12e333fc31208693f065a034b4fb3a95ec4caa9 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 22 Jul 2024 16:05:17 +0800 Subject: [PATCH 15/33] fix agg key merge schema --- .../tests/testdata/input/agg.yaml | 9 ++- .../tests/testdata/output/agg.yaml | 77 +++++++++++-------- .../src/optimizer/plan_node/logical_agg.rs | 7 +- .../optimizer/plan_node/stream_keyed_merge.rs | 31 ++++++-- 4 files changed, 82 insertions(+), 42 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index 9109b31e81a7..fe5e39b50730 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -1000,6 +1000,13 @@ expected_outputs: - batch_plan - stream_plan +- name: test duplicate agg + sql: | + CREATE TABLE t (v1 int); + SELECT sum(v1) as x, count(v1) as y, sum(v1) as z, count(v1) as w from t; + expected_outputs: + - logical_plan + - stream_plan - name: test simple approx_percentile alone sql: | CREATE TABLE t (v1 int); @@ -1017,7 +1024,7 @@ - name: test simple approx_percentile with other simple aggs (sum, count) sql: | CREATE TABLE t (v1 int); - SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1), sum(v1), count(v1) from t; + SELECT sum(v1) as s1, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1), sum(v1) as s2, count(v1) from t; expected_outputs: - logical_plan - stream_plan \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 1cce4e53f67e..7aceaa57c45b 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1863,6 +1863,22 @@ └─StreamHashAgg { group_key: [t.a, t.b], aggs: [sum(t.c), sum(t.d), count(t.d), max(t.e), count] } └─StreamExchange { dist: HashShard(t.a, t.b) } └─StreamTableScan { table: t, columns: [t.a, t.b, t.c, t.d, t.e, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } +- name: test duplicate agg + sql: | + CREATE TABLE t (v1 int); + SELECT sum(v1) as x, count(v1) as y, sum(v1) as z, count(v1) as w from t; + logical_plan: |- + LogicalProject { exprs: [sum(t.v1), count(t.v1), sum(t.v1), count(t.v1)] } + └─LogicalAgg { aggs: [sum(t.v1), count(t.v1)] } + └─LogicalProject { exprs: [t.v1] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + stream_plan: |- + StreamMaterialize { columns: [x, y, z, w], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamProject { exprs: [sum(sum(t.v1)), sum0(count(t.v1)), sum(sum(t.v1)), sum0(count(t.v1))] } + └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), sum0(count(t.v1)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(t.v1), count(t.v1)] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: test simple approx_percentile alone sql: | CREATE TABLE t (v1 int); @@ -1888,39 +1904,40 @@ └─LogicalProject { exprs: [t.v1::Float64 as $expr1, t.v1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: |- - StreamMaterialize { columns: [approx_percentile, sum], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamKeyedMerge { output: [approx_percentile($expr10011):Float64, sum(t.v1):Int64] } - ├─StreamSimpleAgg { aggs: [sum(sum(t.v1)), count] } - │ └─StreamExchange { dist: Single } - │ └─StreamStatelessSimpleAgg { aggs: [sum(t.v1)] } - │ └─StreamShare { id: 2 } - │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } - │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamGlobalApproxPercentile - └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } - └─StreamShare { id: 2 } - └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + StreamMaterialize { columns: [approx_percentile, sum], stream_key: [approx_percentile], pk_columns: [approx_percentile], pk_conflict: NoCheck } + └─StreamKeyedMerge { output: [approx_percentile:Float64, sum(sum(t.v1)):Int64] } + ├─StreamGlobalApproxPercentile + │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(t.v1)] } + └─StreamShare { id: 2 } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: test simple approx_percentile with other simple aggs (sum, count) sql: | CREATE TABLE t (v1 int); - SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1), sum(v1), count(v1) from t; + SELECT sum(v1) as s1, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1), sum(v1) as s2, count(v1) from t; logical_plan: |- - LogicalProject { exprs: [approx_percentile($expr1), sum(t.v1), count(t.v1)] } - └─LogicalAgg { aggs: [approx_percentile($expr1), sum(t.v1), count(t.v1)] } - └─LogicalProject { exprs: [t.v1::Float64 as $expr1, t.v1] } + LogicalProject { exprs: [sum(t.v1), approx_percentile($expr1), sum(t.v1), count(t.v1)] } + └─LogicalAgg { aggs: [sum(t.v1), approx_percentile($expr1), count(t.v1)] } + └─LogicalProject { exprs: [t.v1, t.v1::Float64 as $expr1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: |- - StreamMaterialize { columns: [approx_percentile, sum, count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamKeyedMerge { output: [approx_percentile($expr10011):Float64, sum(t.v1):Int64, count(t.v1):Int64] } - ├─StreamSimpleAgg { aggs: [sum(sum(t.v1)), sum0(count(t.v1)), count] } - │ └─StreamExchange { dist: Single } - │ └─StreamStatelessSimpleAgg { aggs: [sum(t.v1), count(t.v1)] } - │ └─StreamShare { id: 2 } - │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } - │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamGlobalApproxPercentile - └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } - └─StreamShare { id: 2 } - └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } - └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + StreamMaterialize { columns: [s1, approx_percentile, s2, count], stream_key: [s2], pk_columns: [s2], pk_conflict: NoCheck } + └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum(sum(t.v1)), sum0(count(t.v1))] } + └─StreamKeyedMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count(t.v1)):Int64] } + ├─StreamGlobalApproxPercentile + │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), sum0(count(t.v1)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(t.v1), count(t.v1)] } + └─StreamShare { id: 2 } + └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 29de1b6b9621..46eb59fbd00d 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -63,8 +63,6 @@ impl LogicalAgg { fn gen_stateless_two_phase_streaming_agg_plan(&self, stream_input: PlanRef) -> Result { debug_assert!(self.group_key().is_empty()); let mut core = self.core.clone(); - let schema = self.base.schema().clone(); - println!("agg schema: {:?}", schema); // First, handle approx percentile. let has_approx_percentile = self @@ -114,12 +112,11 @@ impl LogicalAgg { )); if let Some((approx_percentile_agg, lhs_mapping, rhs_mapping)) = approx_percentile_info { let keyed_merge = StreamKeyedMerge::new( - global_agg.into(), approx_percentile_agg, + global_agg.into(), lhs_mapping, rhs_mapping, - schema, - ); + )?; Ok(keyed_merge.into()) } else { Ok(global_agg.into()) diff --git a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs index 86034a3b3889..cac5ca63ed76 100644 --- a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs +++ b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs @@ -12,7 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use anyhow::anyhow; +use fixedbitset::FixedBitSet; use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_common::bail; use risingwave_common::catalog::Schema; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::stream_plan::stream_node::PbNodeBody; @@ -29,6 +32,7 @@ use crate::optimizer::plan_node::{ }; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::PlanRef; +use crate::error::Result; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamKeyedMerge { @@ -47,9 +51,24 @@ impl StreamKeyedMerge { rhs_input: PlanRef, lhs_mapping: ColIndexMapping, rhs_mapping: ColIndexMapping, - schema: Schema, - ) -> Self { - println!("keyed merge schema: {:?}", schema); + ) -> Result { + assert_eq!(lhs_mapping.target_size(), rhs_mapping.target_size()); + let mut schema_fields = Vec::with_capacity(lhs_mapping.target_size()); + let mut o2i_lhs = lhs_mapping.inverse().ok_or_else(|| anyhow!("lhs_mapping should be invertible"))?; + let mut o2i_rhs = rhs_mapping.inverse().ok_or_else(|| anyhow!("rhs_mapping should be invertible"))?; + for output_idx in 0..lhs_mapping.target_size() { + if let Some(lhs_idx) = o2i_lhs.try_map(output_idx) { + schema_fields.push(lhs_input.schema().fields()[lhs_idx].clone()); + } else if let Some(rhs_idx) = o2i_rhs.try_map(output_idx) { + println!("rhs schema: {:?}", rhs_input.schema().fields()); + schema_fields.push(rhs_input.schema().fields()[rhs_idx].clone()); + } else { + bail!("output index {} not found in either lhs or rhs mapping", output_idx); + } + } + let schema = Schema::new(schema_fields); + let watermark_columns = FixedBitSet::with_capacity(schema.fields.len()); + // FIXME: schema is wrong. let base = PlanBase::new_stream( lhs_input.ctx(), @@ -59,16 +78,16 @@ impl StreamKeyedMerge { lhs_input.distribution().clone(), lhs_input.append_only(), lhs_input.emit_on_window_close(), - lhs_input.watermark_columns().clone(), + watermark_columns, lhs_input.columns_monotonicity().clone(), ); - Self { + Ok(Self { base, lhs_input, rhs_input, lhs_mapping, rhs_mapping, - } + }) } } From 852dada5001218d50a6760b3bf61b4f13c0f3f94 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 22 Jul 2024 16:09:33 +0800 Subject: [PATCH 16/33] fix global approx percentile stream key --- src/frontend/planner_test/tests/testdata/output/agg.yaml | 6 +++--- .../optimizer/plan_node/stream_global_approx_percentile.rs | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 7aceaa57c45b..b2d2a78c839d 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1889,7 +1889,7 @@ └─LogicalProject { exprs: [t.v1::Float64 as $expr1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: |- - StreamMaterialize { columns: [approx_percentile], stream_key: [approx_percentile], pk_columns: [approx_percentile], pk_conflict: NoCheck } + StreamMaterialize { columns: [approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamGlobalApproxPercentile └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } @@ -1904,7 +1904,7 @@ └─LogicalProject { exprs: [t.v1::Float64 as $expr1, t.v1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: |- - StreamMaterialize { columns: [approx_percentile, sum], stream_key: [approx_percentile], pk_columns: [approx_percentile], pk_conflict: NoCheck } + StreamMaterialize { columns: [approx_percentile, sum], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamKeyedMerge { output: [approx_percentile:Float64, sum(sum(t.v1)):Int64] } ├─StreamGlobalApproxPercentile │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } @@ -1927,7 +1927,7 @@ └─LogicalProject { exprs: [t.v1, t.v1::Float64 as $expr1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: |- - StreamMaterialize { columns: [s1, approx_percentile, s2, count], stream_key: [s2], pk_columns: [s2], pk_conflict: NoCheck } + StreamMaterialize { columns: [s1, approx_percentile, s2, count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum(sum(t.v1)), sum0(count(t.v1))] } └─StreamKeyedMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count(t.v1)):Int64] } ├─StreamGlobalApproxPercentile diff --git a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs index a165d6991033..11d2ccdcb40e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs @@ -44,7 +44,7 @@ impl StreamGlobalApproxPercentile { let base = PlanBase::new_stream( input.ctx(), schema, - Some(vec![0]), + Some(vec![]), input.functional_dependency().clone(), Distribution::Single, input.append_only(), From c9a9b054044ef8c24c6852a7dd15106ebc69ed01 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 22 Jul 2024 17:05:41 +0800 Subject: [PATCH 17/33] handle multiple approx percentile --- .../tests/testdata/input/agg.yaml | 21 ++++ .../tests/testdata/output/agg.yaml | 65 ++++++++++++ .../src/optimizer/plan_node/logical_agg.rs | 99 ++++++++++++------- 3 files changed, 149 insertions(+), 36 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index fe5e39b50730..2fcfc8db3831 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -1025,6 +1025,27 @@ sql: | CREATE TABLE t (v1 int); SELECT sum(v1) as s1, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1), sum(v1) as s2, count(v1) from t; + expected_outputs: + - logical_plan + - stream_plan +- name: test simple approx_percentile with duplicate approx_percentile + sql: | + CREATE TABLE t (v1 int); + SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) as x, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) as y from t; + expected_outputs: + - logical_plan + - stream_plan +- name: test simple approx_percentile with different approx_percentile + sql: | + CREATE TABLE t (v1 int, v2 int); + SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) as x, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v2) as y from t; + expected_outputs: + - logical_plan + - stream_plan +- name: test simple approx_percentile with different approx_percentile interleaved with stateless simple aggs + sql: | + CREATE TABLE t (v1 int, v2 int); + SELECT sum(v1) as s1, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) as x, count(*), sum(v2) as s2, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v2) as y from t; expected_outputs: - logical_plan - stream_plan \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index b2d2a78c839d..04d93920342b 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1941,3 +1941,68 @@ └─StreamShare { id: 2 } └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } +- name: test simple approx_percentile with duplicate approx_percentile + sql: | + CREATE TABLE t (v1 int); + SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) as x, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) as y from t; + logical_plan: |- + LogicalProject { exprs: [approx_percentile($expr1), approx_percentile($expr1)] } + └─LogicalAgg { aggs: [approx_percentile($expr1)] } + └─LogicalProject { exprs: [t.v1::Float64 as $expr1] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + stream_plan: |- + StreamMaterialize { columns: [x, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamProject { exprs: [approx_percentile, approx_percentile] } + └─StreamGlobalApproxPercentile + └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } +- name: test simple approx_percentile with different approx_percentile + sql: | + CREATE TABLE t (v1 int, v2 int); + SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) as x, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v2) as y from t; + logical_plan: |- + LogicalProject { exprs: [approx_percentile($expr1), approx_percentile($expr2)] } + └─LogicalAgg { aggs: [approx_percentile($expr1), approx_percentile($expr2)] } + └─LogicalProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + stream_plan: |- + StreamMaterialize { columns: [x, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamKeyedMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } + ├─StreamGlobalApproxPercentile + │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamGlobalApproxPercentile + └─StreamLocalApproxPercentile { percentile_col: $expr4, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + └─StreamProject { exprs: [t.v1::Float64 as $expr3, t.v2::Float64 as $expr4, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } +- name: test simple approx_percentile with different approx_percentile interleaved with stateless simple aggs + sql: | + CREATE TABLE t (v1 int, v2 int); + SELECT sum(v1) as s1, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) as x, count(*), sum(v2) as s2, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v2) as y from t; + logical_plan: |- + LogicalProject { exprs: [sum(t.v1), approx_percentile($expr1), count, sum(t.v2), approx_percentile($expr2)] } + └─LogicalAgg { aggs: [sum(t.v1), approx_percentile($expr1), count, sum(t.v2), approx_percentile($expr2)] } + └─LogicalProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + stream_plan: |- + StreamMaterialize { columns: [s1, x, count, s2, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamKeyedMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count):Int64, sum(sum(t.v2)):Int64, approx_percentile:Float64] } + ├─StreamKeyedMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } + │ ├─StreamGlobalApproxPercentile + │ │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + │ │ └─StreamShare { id: 2 } + │ │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } + │ │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + │ └─StreamGlobalApproxPercentile + │ └─StreamLocalApproxPercentile { percentile_col: $expr2, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), sum0(count), sum(sum(t.v2)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(t.v1), count, sum(t.v2)] } + └─StreamShare { id: 2 } + └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 46eb59fbd00d..ff2d5d08c3a9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -45,6 +45,13 @@ use crate::utils::{ ColIndexMapping, ColIndexMappingRewriteExt, Condition, GroupBy, IndexSet, Substitute, }; +pub struct SeparatedAggInfo { + pub approx_percentile_agg_calls: Vec, + pub non_approx_percentile_agg_calls: Vec, + pub approx_percentile_col_mapping: ColIndexMapping, + pub non_approx_percentile_col_mapping: ColIndexMapping, +} + /// `LogicalAgg` groups input data by their group key and computes aggregation functions. /// /// It corresponds to the `GROUP BY` operator in a SQL query statement together with the aggregate @@ -70,25 +77,24 @@ impl LogicalAgg { .iter() .any(|agg_call| agg_call.agg_kind == AggKind::ApproxPercentile); let approx_percentile_info = if has_approx_percentile { - let ( - approx_percentile_agg_call, + let SeparatedAggInfo { + approx_percentile_agg_calls, non_approx_percentile_agg_calls, - lhs_mapping, - rhs_mapping, - ) = self.extract_approx_percentile(); - // FIXME(kwannoel): Handle case where there's multiple approx percentile also. + approx_percentile_col_mapping, + non_approx_percentile_col_mapping, + } = self.separate_normal_and_special_agg(); let requires_keyed_merge = !non_approx_percentile_agg_calls.is_empty(); if requires_keyed_merge { let stream_input: PlanRef = StreamShare::new_from_input(stream_input).into(); let approx_percentile_agg: PlanRef = self - .build_approx_percentile_agg(stream_input.clone(), &approx_percentile_agg_call); + .build_approx_percentile_aggs(stream_input.clone(), &approx_percentile_agg_calls); core.input = stream_input; core.agg_calls = non_approx_percentile_agg_calls; - Some((approx_percentile_agg, lhs_mapping, rhs_mapping)) + Some((approx_percentile_agg, approx_percentile_col_mapping, non_approx_percentile_col_mapping)) } else { let approx_percentile_agg: PlanRef = self - .build_approx_percentile_agg(stream_input.clone(), &approx_percentile_agg_call); + .build_approx_percentile_aggs(stream_input.clone(), &approx_percentile_agg_calls); return Ok(approx_percentile_agg); } } else { @@ -240,14 +246,6 @@ impl LogicalAgg { } else { self.core.can_two_phase_agg() }); - // // Handle 2-phase approx_percentile aggregation. - // if self - // .agg_calls() - // .iter() - // .any(|agg_call| agg_call.agg_kind == AggKind::ApproxPercentile) - // { - // return self.gen_two_phase_approx_percentile_plan(stream_input, ctx); - // } // Stateless 2-phase simple agg // can be applied on stateless simple agg calls, @@ -300,34 +298,29 @@ impl LogicalAgg { } // TODO(kwannoel): Handle multiple approx_percentile. - fn extract_approx_percentile( + fn separate_normal_and_special_agg( &self, - ) -> ( - PlanAggCall, - Vec, - ColIndexMapping, - ColIndexMapping, - ) { - let mut approx_percentile_agg_call = None; + ) -> SeparatedAggInfo { let estimated_len = self.agg_calls().len() - 1; - let mut remaining_agg_calls = Vec::with_capacity(estimated_len); + let mut approx_percentile_agg_calls = Vec::with_capacity(estimated_len); + let mut non_approx_percentile_agg_calls = Vec::with_capacity(estimated_len); let mut approx_percentile_col_mapping = Vec::with_capacity(estimated_len); - let mut remaining_simple_agg_col_mapping = Vec::with_capacity(estimated_len); + let mut non_approx_percentile_col_mapping = Vec::with_capacity(estimated_len); for (output_idx, agg_call) in self.agg_calls().iter().enumerate() { if agg_call.agg_kind == AggKind::ApproxPercentile { - approx_percentile_agg_call = Some(agg_call.clone()); + approx_percentile_agg_calls.push(agg_call.clone()); approx_percentile_col_mapping.push(Some(output_idx)); } else { - remaining_agg_calls.push(agg_call.clone()); - remaining_simple_agg_col_mapping.push(Some(output_idx)); + non_approx_percentile_agg_calls.push(agg_call.clone()); + non_approx_percentile_col_mapping.push(Some(output_idx)); } } - ( - approx_percentile_agg_call.expect("ApproxPercentile agg call not found"), - remaining_agg_calls, - ColIndexMapping::new(approx_percentile_col_mapping, self.agg_calls().len()), - ColIndexMapping::new(remaining_simple_agg_col_mapping, self.agg_calls().len()), - ) + SeparatedAggInfo { + approx_percentile_agg_calls, + non_approx_percentile_agg_calls, + approx_percentile_col_mapping: ColIndexMapping::new(approx_percentile_col_mapping, self.agg_calls().len()), + non_approx_percentile_col_mapping: ColIndexMapping::new(non_approx_percentile_col_mapping, self.agg_calls().len()), + } } fn build_approx_percentile_agg( @@ -344,6 +337,40 @@ impl LogicalAgg { global_approx_percentile.into() } + /// If only 1 approx percentile, just return it. + /// Otherwise build a tree of approx percentile with KeyedMerge. + /// e.g. + /// ApproxPercentile(col1, 0.5) as x, + /// ApproxPercentile(col2, 0.5) as y, + /// ApproxPercentile(col3, 0.5) as z + /// will be built as + /// KeyedMerge + /// / \ + /// KeyedMerge z + /// / \ + /// x y + + fn build_approx_percentile_aggs( + &self, + input: PlanRef, + approx_percentile_agg_call: &[PlanAggCall], + ) -> PlanRef { + let approx_percentile_plans = approx_percentile_agg_call + .iter() + .map(|agg_call| self.build_approx_percentile_agg(input.clone(), agg_call)) + .collect_vec(); + assert!(approx_percentile_plans.len() >= 1); + let mut iter = approx_percentile_plans.into_iter(); + let mut acc = iter.next().unwrap(); + for (current_size, plan) in iter.enumerate().map(|(i, p)| (i + 1, p)) { + let new_size = current_size + 1; + let keyed_merge = StreamKeyedMerge::new(acc, plan, ColIndexMapping::identity_or_none(current_size, new_size), ColIndexMapping::new(vec![Some(current_size)], new_size)) + .expect("failed to build keyed merge"); + acc = keyed_merge.into(); + } + acc + } + pub fn core(&self) -> &Agg { &self.core } From f4b6586ec0786757576955ea3dc2659b4de7ffb7 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 22 Jul 2024 18:22:18 +0800 Subject: [PATCH 18/33] handle descending case --- src/expr/core/src/aggregate/def.rs | 1 - .../tests/testdata/input/agg.yaml | 7 ++ .../tests/testdata/output/agg.yaml | 39 +++++++-- src/frontend/src/binder/expr/function.rs | 57 ++++++++++++- .../src/optimizer/plan_node/generic/agg.rs | 2 +- .../src/optimizer/plan_node/logical_agg.rs | 83 ++++++++++++++----- .../optimizer/plan_node/stream_keyed_merge.rs | 19 +++-- .../src/optimizer/rule/distinct_agg_rule.rs | 2 +- 8 files changed, 171 insertions(+), 39 deletions(-) diff --git a/src/expr/core/src/aggregate/def.rs b/src/expr/core/src/aggregate/def.rs index 2e17c0fe3a81..1fb985585cdf 100644 --- a/src/expr/core/src/aggregate/def.rs +++ b/src/expr/core/src/aggregate/def.rs @@ -339,7 +339,6 @@ pub mod agg_kinds { | PbAggKind::Count | PbAggKind::Avg | PbAggKind::ApproxCountDistinct - | PbAggKind::ApproxPercentile | PbAggKind::VarPop | PbAggKind::VarSamp | PbAggKind::StddevPop diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index 2fcfc8db3831..25f62054f1e6 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -1046,6 +1046,13 @@ sql: | CREATE TABLE t (v1 int, v2 int); SELECT sum(v1) as s1, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) as x, count(*), sum(v2) as s2, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v2) as y from t; + expected_outputs: + - logical_plan + - stream_plan +- name: test simple approx_percentile with descending order + sql: | + CREATE TABLE t (v1 int, v2 int); + SELECT sum(v1) as s1, approx_percentile(0.2, 0.01) WITHIN GROUP (order by v1 desc) from t; expected_outputs: - logical_plan - stream_plan \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 04d93920342b..e1e5a6d57053 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1891,7 +1891,7 @@ stream_plan: |- StreamMaterialize { columns: [approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamGlobalApproxPercentile - └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: test simple approx_percentile with other simple aggs @@ -1907,7 +1907,7 @@ StreamMaterialize { columns: [approx_percentile, sum], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamKeyedMerge { output: [approx_percentile:Float64, sum(sum(t.v1)):Int64] } ├─StreamGlobalApproxPercentile - │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ └─StreamShare { id: 2 } │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -1931,7 +1931,7 @@ └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum(sum(t.v1)), sum0(count(t.v1))] } └─StreamKeyedMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count(t.v1)):Int64] } ├─StreamGlobalApproxPercentile - │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ └─StreamShare { id: 2 } │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -1954,7 +1954,7 @@ StreamMaterialize { columns: [x, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [approx_percentile, approx_percentile] } └─StreamGlobalApproxPercentile - └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: test simple approx_percentile with different approx_percentile @@ -1970,11 +1970,11 @@ StreamMaterialize { columns: [x, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamKeyedMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } ├─StreamGlobalApproxPercentile - │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamGlobalApproxPercentile - └─StreamLocalApproxPercentile { percentile_col: $expr4, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + └─StreamLocalApproxPercentile { percentile_col: $expr4, quantile: 0.5:Float64, relative_error: 0.01:Float64 } └─StreamProject { exprs: [t.v1::Float64 as $expr3, t.v2::Float64 as $expr4, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: test simple approx_percentile with different approx_percentile interleaved with stateless simple aggs @@ -1991,12 +1991,12 @@ └─StreamKeyedMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count):Int64, sum(sum(t.v2)):Int64, approx_percentile:Float64] } ├─StreamKeyedMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } │ ├─StreamGlobalApproxPercentile - │ │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + │ │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ │ └─StreamShare { id: 2 } │ │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } │ │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } │ └─StreamGlobalApproxPercentile - │ └─StreamLocalApproxPercentile { percentile_col: $expr2, quantile: 0.5:Decimal, relative_error: 0.01:Decimal } + │ └─StreamLocalApproxPercentile { percentile_col: $expr2, quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ └─StreamShare { id: 2 } │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -2006,3 +2006,26 @@ └─StreamShare { id: 2 } └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } +- name: test simple approx_percentile with descending order + sql: | + CREATE TABLE t (v1 int, v2 int); + SELECT sum(v1) as s1, approx_percentile(0.2, 0.01) WITHIN GROUP (order by v1 desc) from t; + logical_plan: |- + LogicalProject { exprs: [sum(t.v1), approx_percentile($expr1)] } + └─LogicalAgg { aggs: [sum(t.v1), approx_percentile($expr1)] } + └─LogicalProject { exprs: [t.v1, t.v1::Float64 as $expr1] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + stream_plan: |- + StreamMaterialize { columns: [s1, approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamKeyedMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64] } + ├─StreamGlobalApproxPercentile + │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.8:Float64, relative_error: 0.01:Float64 } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamSimpleAgg { aggs: [sum(sum(t.v1)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(t.v1)] } + └─StreamShare { id: 2 } + └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 79505481ecac..aa9574712f9a 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -511,7 +511,62 @@ impl Binder { } } (AggKind::Builtin(PbAggKind::Mode), [], [_arg]) => {} - (AggKind::Builtin(PbAggKind::ApproxPercentile), [_arg, _arg2], [_arg3]) => {} + (AggKind::Builtin(PbAggKind::ApproxPercentile), [percentile, relative_error], [_percentile_col]) => { + percentile + .cast_implicit_mut(DataType::Float64) + .map_err(|_e| { + ErrorCode::InvalidInputSyntax(format!( + "direct arg in `{}` must be castable to float64", + kind + )) + })?; + + let Some(Ok(percentile_datum)) = percentile.try_fold_const() else { + bail_not_implemented!( + issue = 14079, + "variable as direct argument of ordered-set aggregate", + ) + }; + + if let Some(ref percentile_scalar) = percentile_datum + && !(0.0..=1.0).contains(&percentile_scalar.as_float64().0) + { + return Err(ErrorCode::InvalidInputSyntax(format!( + "direct arg in `{}` must between 0.0 and 1.0", + kind + )) + .into()); + } + // note that the fraction can be NULL + *percentile = Literal::new(percentile_datum, DataType::Float64).into(); + + relative_error + .cast_implicit_mut(DataType::Float64) + .map_err(|_e| { + ErrorCode::InvalidInputSyntax(format!( + "direct arg in `{}` must be castable to float64", + kind + )) + })?; + let Some(Ok(relative_error_datum)) = relative_error.try_fold_const() else { + bail_not_implemented!( + issue = 14079, + "variable as direct argument of ordered-set aggregate", + ) + }; + + if let Some(ref relative_error_scalar) = relative_error_datum + && !(0.0..=1.0).contains(&relative_error_scalar.as_float64().0) + { + return Err(ErrorCode::InvalidInputSyntax(format!( + "direct arg in `{}` must between 0.0 and 1.0", + kind + )) + .into()); + } + // note that the fraction can be NULL + *relative_error = Literal::new(relative_error_datum, DataType::Float64).into(); + } _ => { return Err(ErrorCode::InvalidInputSyntax(format!( "invalid direct args or within group argument for `{}` aggregation", diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 14193b378376..8f5c4a32cc14 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -105,7 +105,7 @@ impl Agg { && !self.agg_calls.is_empty() && self.agg_calls.iter().all(|call| { let agg_kind_ok = !matches!(call.agg_kind, agg_kinds::simply_cannot_two_phase!()); - let order_ok = matches!(call.agg_kind, agg_kinds::result_unaffected_by_order_by!()) + let order_ok = matches!(call.agg_kind, agg_kinds::result_unaffected_by_order_by!() | AggKind::ApproxPercentile) || call.order_by.is_empty(); let distinct_ok = matches!(call.agg_kind, agg_kinds::result_unaffected_by_distinct!()) diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index ff2d5d08c3a9..a741e4ddf746 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -12,10 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use crate::expr::OrderByExpr; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::types::{DataType, Datum, ScalarImpl}; -use risingwave_common::util::sort_util::ColumnOrder; +use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_common::{bail_not_implemented, not_implemented}; use risingwave_expr::aggregate::{agg_kinds, AggKind, PbAggKind}; @@ -42,7 +43,7 @@ use crate::optimizer::plan_node::{ }; use crate::optimizer::property::{Distribution, Order, RequiredDist}; use crate::utils::{ - ColIndexMapping, ColIndexMappingRewriteExt, Condition, GroupBy, IndexSet, Substitute, + ColIndexMapping, ColIndexMappingRewriteExt, Condition, DynEq, GroupBy, IndexSet, Substitute, }; pub struct SeparatedAggInfo { @@ -86,22 +87,31 @@ impl LogicalAgg { let requires_keyed_merge = !non_approx_percentile_agg_calls.is_empty(); if requires_keyed_merge { let stream_input: PlanRef = StreamShare::new_from_input(stream_input).into(); - let approx_percentile_agg: PlanRef = self - .build_approx_percentile_aggs(stream_input.clone(), &approx_percentile_agg_calls); + let approx_percentile_agg: PlanRef = self.build_approx_percentile_aggs( + stream_input.clone(), + &approx_percentile_agg_calls, + ); core.input = stream_input; core.agg_calls = non_approx_percentile_agg_calls; - Some((approx_percentile_agg, approx_percentile_col_mapping, non_approx_percentile_col_mapping)) + Some(( + approx_percentile_agg, + approx_percentile_col_mapping, + non_approx_percentile_col_mapping, + )) } else { - let approx_percentile_agg: PlanRef = self - .build_approx_percentile_aggs(stream_input.clone(), &approx_percentile_agg_calls); + let approx_percentile_agg: PlanRef = self.build_approx_percentile_aggs( + stream_input.clone(), + &approx_percentile_agg_calls, + ); return Ok(approx_percentile_agg); } } else { core.input = stream_input; None }; - let total_agg_calls = core.agg_calls + let total_agg_calls = core + .agg_calls .iter() .enumerate() .map(|(partial_output_idx, agg_call)| { @@ -111,11 +121,8 @@ impl LogicalAgg { let local_agg = StreamStatelessSimpleAgg::new(core); let exchange = RequiredDist::single().enforce_if_not_satisfies(local_agg.into(), &Order::any())?; - let global_agg = new_stream_simple_agg(Agg::new( - total_agg_calls, - IndexSet::empty(), - exchange, - )); + let global_agg = + new_stream_simple_agg(Agg::new(total_agg_calls, IndexSet::empty(), exchange)); if let Some((approx_percentile_agg, lhs_mapping, rhs_mapping)) = approx_percentile_info { let keyed_merge = StreamKeyedMerge::new( approx_percentile_agg, @@ -298,9 +305,7 @@ impl LogicalAgg { } // TODO(kwannoel): Handle multiple approx_percentile. - fn separate_normal_and_special_agg( - &self, - ) -> SeparatedAggInfo { + fn separate_normal_and_special_agg(&self) -> SeparatedAggInfo { let estimated_len = self.agg_calls().len() - 1; let mut approx_percentile_agg_calls = Vec::with_capacity(estimated_len); let mut non_approx_percentile_agg_calls = Vec::with_capacity(estimated_len); @@ -318,8 +323,14 @@ impl LogicalAgg { SeparatedAggInfo { approx_percentile_agg_calls, non_approx_percentile_agg_calls, - approx_percentile_col_mapping: ColIndexMapping::new(approx_percentile_col_mapping, self.agg_calls().len()), - non_approx_percentile_col_mapping: ColIndexMapping::new(non_approx_percentile_col_mapping, self.agg_calls().len()), + approx_percentile_col_mapping: ColIndexMapping::new( + approx_percentile_col_mapping, + self.agg_calls().len(), + ), + non_approx_percentile_col_mapping: ColIndexMapping::new( + non_approx_percentile_col_mapping, + self.agg_calls().len(), + ), } } @@ -364,8 +375,13 @@ impl LogicalAgg { let mut acc = iter.next().unwrap(); for (current_size, plan) in iter.enumerate().map(|(i, p)| (i + 1, p)) { let new_size = current_size + 1; - let keyed_merge = StreamKeyedMerge::new(acc, plan, ColIndexMapping::identity_or_none(current_size, new_size), ColIndexMapping::new(vec![Some(current_size)], new_size)) - .expect("failed to build keyed merge"); + let keyed_merge = StreamKeyedMerge::new( + acc, + plan, + ColIndexMapping::identity_or_none(current_size, new_size), + ColIndexMapping::new(vec![Some(current_size)], new_size), + ) + .expect("failed to build keyed merge"); acc = keyed_merge.into(); } acc @@ -641,6 +657,33 @@ impl LogicalAggBuilder { _ => unreachable!(), } } + AggKind::ApproxPercentile + => + { + if agg_call.order_by.sort_exprs[0].order_type == OrderType::descending() { + let prev_percentile = agg_call.direct_args[0].clone(); + let new_percentile = 1.0 - prev_percentile.get_data().as_ref().unwrap().as_float64().into_inner(); + let new_percentile = Some(ScalarImpl::Float64(new_percentile.into())); + let new_percentile = Literal::new( + new_percentile, + DataType::Float64, + ); + let new_direct_args = vec![new_percentile, agg_call.direct_args[1].clone()]; + + let new_agg_call = AggCall { + order_by: OrderBy::any(), + direct_args: new_direct_args, + ..agg_call + }; + Ok(push_agg_call(new_agg_call)?.into()) + } else { + let new_agg_call = AggCall { + order_by: OrderBy::any(), + ..agg_call + }; + Ok(push_agg_call(new_agg_call)?.into()) + } + } _ => Ok(push_agg_call(agg_call)?.into()), } } diff --git a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs index cac5ca63ed76..b2116ad2a21d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs +++ b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs @@ -21,6 +21,7 @@ use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::HopWindowNode; +use crate::error::Result; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef}; @@ -32,7 +33,6 @@ use crate::optimizer::plan_node::{ }; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::PlanRef; -use crate::error::Result; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamKeyedMerge { @@ -54,8 +54,12 @@ impl StreamKeyedMerge { ) -> Result { assert_eq!(lhs_mapping.target_size(), rhs_mapping.target_size()); let mut schema_fields = Vec::with_capacity(lhs_mapping.target_size()); - let mut o2i_lhs = lhs_mapping.inverse().ok_or_else(|| anyhow!("lhs_mapping should be invertible"))?; - let mut o2i_rhs = rhs_mapping.inverse().ok_or_else(|| anyhow!("rhs_mapping should be invertible"))?; + let mut o2i_lhs = lhs_mapping + .inverse() + .ok_or_else(|| anyhow!("lhs_mapping should be invertible"))?; + let mut o2i_rhs = rhs_mapping + .inverse() + .ok_or_else(|| anyhow!("rhs_mapping should be invertible"))?; for output_idx in 0..lhs_mapping.target_size() { if let Some(lhs_idx) = o2i_lhs.try_map(output_idx) { schema_fields.push(lhs_input.schema().fields()[lhs_idx].clone()); @@ -63,7 +67,10 @@ impl StreamKeyedMerge { println!("rhs schema: {:?}", rhs_input.schema().fields()); schema_fields.push(rhs_input.schema().fields()[rhs_idx].clone()); } else { - bail!("output index {} not found in either lhs or rhs mapping", output_idx); + bail!( + "output index {} not found in either lhs or rhs mapping", + output_idx + ); } } let schema = Schema::new(schema_fields); @@ -103,7 +110,6 @@ impl Distill for StreamKeyedMerge { out = vec![("output", e)]; } childless_record("StreamKeyedMerge", out) - } } @@ -146,6 +152,5 @@ impl ExprRewritable for StreamKeyedMerge { } impl ExprVisitable for StreamKeyedMerge { - fn visit_exprs(&self, v: &mut dyn ExprVisitor) { - } + fn visit_exprs(&self, v: &mut dyn ExprVisitor) {} } diff --git a/src/frontend/src/optimizer/rule/distinct_agg_rule.rs b/src/frontend/src/optimizer/rule/distinct_agg_rule.rs index 353b4ff7a7f6..919fcde2bff5 100644 --- a/src/frontend/src/optimizer/rule/distinct_agg_rule.rs +++ b/src/frontend/src/optimizer/rule/distinct_agg_rule.rs @@ -57,7 +57,7 @@ impl Rule for DistinctAggRule { c.agg_kind ); let agg_kind_ok = !matches!(c.agg_kind, agg_kinds::simply_cannot_two_phase!()); - let order_ok = matches!(c.agg_kind, agg_kinds::result_unaffected_by_order_by!()) + let order_ok = matches!(c.agg_kind, agg_kinds::result_unaffected_by_order_by!() | AggKind::ApproxPercentile) || c.order_by.is_empty(); agg_kind_ok && order_ok }) { From 965ae196d2e575f001579e25fc606408562da9a7 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 22 Jul 2024 21:18:01 +0800 Subject: [PATCH 19/33] fmt --- src/frontend/src/binder/expr/function.rs | 4 +- .../src/optimizer/plan_node/generic/agg.rs | 6 ++- .../src/optimizer/plan_node/logical_agg.rs | 50 ++++++++----------- .../stream_global_approx_percentile.rs | 11 ++-- .../optimizer/plan_node/stream_keyed_merge.rs | 12 ++--- .../stream_local_approx_percentile.rs | 10 ++-- .../src/optimizer/rule/distinct_agg_rule.rs | 6 ++- 7 files changed, 45 insertions(+), 54 deletions(-) diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index aa9574712f9a..453de89f3dde 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -535,7 +535,7 @@ impl Binder { "direct arg in `{}` must between 0.0 and 1.0", kind )) - .into()); + .into()); } // note that the fraction can be NULL *percentile = Literal::new(percentile_datum, DataType::Float64).into(); @@ -562,7 +562,7 @@ impl Binder { "direct arg in `{}` must between 0.0 and 1.0", kind )) - .into()); + .into()); } // note that the fraction can be NULL *relative_error = Literal::new(relative_error_datum, DataType::Float64).into(); diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 8f5c4a32cc14..419b6058f7c6 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -105,8 +105,10 @@ impl Agg { && !self.agg_calls.is_empty() && self.agg_calls.iter().all(|call| { let agg_kind_ok = !matches!(call.agg_kind, agg_kinds::simply_cannot_two_phase!()); - let order_ok = matches!(call.agg_kind, agg_kinds::result_unaffected_by_order_by!() | AggKind::ApproxPercentile) - || call.order_by.is_empty(); + let order_ok = matches!( + call.agg_kind, + agg_kinds::result_unaffected_by_order_by!() | AggKind::ApproxPercentile + ) || call.order_by.is_empty(); let distinct_ok = matches!(call.agg_kind, agg_kinds::result_unaffected_by_distinct!()) || !call.distinct; diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index a741e4ddf746..1fcaa47de342 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use crate::expr::OrderByExpr; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::types::{DataType, Datum, ScalarImpl}; @@ -23,8 +22,8 @@ use risingwave_expr::aggregate::{agg_kinds, AggKind, PbAggKind}; use super::generic::{self, Agg, GenericPlanRef, PlanAggCall, ProjectBuilder}; use super::utils::impl_distill_by_unit; use super::{ - BatchHashAgg, BatchSimpleAgg, ColPrunable, ExprRewritable, Logical, LogicalShare, PlanBase, - PlanRef, PlanTreeNodeUnary, PredicatePushdown, StreamHashAgg, StreamProject, StreamShare, + BatchHashAgg, BatchSimpleAgg, ColPrunable, ExprRewritable, Logical, PlanBase, PlanRef, + PlanTreeNodeUnary, PredicatePushdown, StreamHashAgg, StreamProject, StreamShare, StreamSimpleAgg, StreamStatelessSimpleAgg, ToBatch, ToStream, }; use crate::error::{ErrorCode, Result, RwError}; @@ -43,7 +42,7 @@ use crate::optimizer::plan_node::{ }; use crate::optimizer::property::{Distribution, Order, RequiredDist}; use crate::utils::{ - ColIndexMapping, ColIndexMappingRewriteExt, Condition, DynEq, GroupBy, IndexSet, Substitute, + ColIndexMapping, ColIndexMappingRewriteExt, Condition, GroupBy, IndexSet, Substitute, }; pub struct SeparatedAggInfo { @@ -225,11 +224,7 @@ impl LogicalAgg { } /// Generates distributed stream plan. - fn gen_dist_stream_agg_plan( - &self, - stream_input: PlanRef, - ctx: &mut ToStreamContext, - ) -> Result { + fn gen_dist_stream_agg_plan(&self, stream_input: PlanRef) -> Result { use super::stream::prelude::*; let input_dist = stream_input.distribution(); @@ -340,24 +335,22 @@ impl LogicalAgg { approx_percentile_agg_call: &PlanAggCall, ) -> PlanRef { let local_approx_percentile = - StreamLocalApproxPercentile::new(input, &approx_percentile_agg_call); - let global_approx_percentile = StreamGlobalApproxPercentile::new( - local_approx_percentile.into(), - &approx_percentile_agg_call, - ); + StreamLocalApproxPercentile::new(input, approx_percentile_agg_call); + let global_approx_percentile = + StreamGlobalApproxPercentile::new(local_approx_percentile.into()); global_approx_percentile.into() } /// If only 1 approx percentile, just return it. - /// Otherwise build a tree of approx percentile with KeyedMerge. + /// Otherwise build a tree of approx percentile with `KeyedMerge`. /// e.g. /// ApproxPercentile(col1, 0.5) as x, /// ApproxPercentile(col2, 0.5) as y, /// ApproxPercentile(col3, 0.5) as z /// will be built as - /// KeyedMerge + /// `KeyedMerge` /// / \ - /// KeyedMerge z + /// `KeyedMerge` z /// / \ /// x y @@ -370,7 +363,7 @@ impl LogicalAgg { .iter() .map(|agg_call| self.build_approx_percentile_agg(input.clone(), agg_call)) .collect_vec(); - assert!(approx_percentile_plans.len() >= 1); + assert!(!approx_percentile_plans.is_empty()); let mut iter = approx_percentile_plans.into_iter(); let mut acc = iter.next().unwrap(); for (current_size, plan) in iter.enumerate().map(|(i, p)| (i + 1, p)) { @@ -657,17 +650,18 @@ impl LogicalAggBuilder { _ => unreachable!(), } } - AggKind::ApproxPercentile - => - { + AggKind::ApproxPercentile => { if agg_call.order_by.sort_exprs[0].order_type == OrderType::descending() { let prev_percentile = agg_call.direct_args[0].clone(); - let new_percentile = 1.0 - prev_percentile.get_data().as_ref().unwrap().as_float64().into_inner(); + let new_percentile = 1.0 + - prev_percentile + .get_data() + .as_ref() + .unwrap() + .as_float64() + .into_inner(); let new_percentile = Some(ScalarImpl::Float64(new_percentile.into())); - let new_percentile = Literal::new( - new_percentile, - DataType::Float64, - ); + let new_percentile = Literal::new(new_percentile, DataType::Float64); let new_direct_args = vec![new_percentile, agg_call.direct_args[1].clone()]; let new_agg_call = AggCall { @@ -1283,7 +1277,7 @@ impl ToStream for LogicalAgg { return logical_dedup.to_stream(ctx); } - let plan = self.gen_dist_stream_agg_plan(stream_input, ctx)?; + let plan = self.gen_dist_stream_agg_plan(stream_input)?; let (plan, n_final_agg_calls) = if let Some(final_agg) = plan.as_stream_simple_agg() { if eowc { @@ -1303,7 +1297,7 @@ impl ToStream for LogicalAgg { }, final_agg.agg_calls().len(), ) - } else if let Some(approx_percentile_agg) = plan.as_stream_global_approx_percentile() { + } else if let Some(_approx_percentile_agg) = plan.as_stream_global_approx_percentile() { if eowc { return Err(ErrorCode::InvalidInputSyntax( "`EMIT ON WINDOW CLOSE` cannot be used for aggregation without `GROUP BY`" diff --git a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs index 11d2ccdcb40e..caab6da90890 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs @@ -20,12 +20,11 @@ use risingwave_pb::stream_plan::stream_node::PbNodeBody; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; -use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef}; +use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::stream::StreamPlanRef; -use crate::optimizer::plan_node::utils::{childless_record, watermark_pretty, Distill}; +use crate::optimizer::plan_node::utils::{childless_record, Distill}; use crate::optimizer::plan_node::{ - ExprRewritable, PlanAggCall, PlanBase, PlanTreeNodeUnary, Stream, StreamHopWindow, - StreamKeyedMerge, StreamNode, + ExprRewritable, PlanBase, PlanTreeNodeUnary, Stream, StreamNode, }; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -38,7 +37,7 @@ pub struct StreamGlobalApproxPercentile { } impl StreamGlobalApproxPercentile { - pub fn new(input: PlanRef, approx_percentile_agg_call: &PlanAggCall) -> Self { + pub fn new(input: PlanRef) -> Self { let schema = Schema::new(vec![Field::new("approx_percentile", DataType::Float64)]); let watermark_columns = FixedBitSet::with_capacity(1); let base = PlanBase::new_stream( @@ -94,5 +93,5 @@ impl ExprRewritable for StreamGlobalApproxPercentile { } impl ExprVisitable for StreamGlobalApproxPercentile { - fn visit_exprs(&self, v: &mut dyn ExprVisitor) {} + fn visit_exprs(&self, _v: &mut dyn ExprVisitor) {} } diff --git a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs index b2116ad2a21d..6e6826ef6a44 100644 --- a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs +++ b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs @@ -19,17 +19,15 @@ use risingwave_common::bail; use risingwave_common::catalog::Schema; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_pb::stream_plan::stream_node::PbNodeBody; -use risingwave_pb::stream_plan::HopWindowNode; use crate::error::Result; use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef}; use crate::optimizer::plan_node::stream::StreamPlanRef; -use crate::optimizer::plan_node::utils::{childless_record, Distill, IndicesDisplay}; +use crate::optimizer::plan_node::utils::{childless_record, Distill}; use crate::optimizer::plan_node::{ - ExprRewritable, PlanBase, PlanTreeNodeBinary, Stream, StreamHopWindow, - StreamLocalApproxPercentile, StreamNode, + ExprRewritable, PlanBase, PlanTreeNodeBinary, Stream, StreamNode, }; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::PlanRef; @@ -54,10 +52,10 @@ impl StreamKeyedMerge { ) -> Result { assert_eq!(lhs_mapping.target_size(), rhs_mapping.target_size()); let mut schema_fields = Vec::with_capacity(lhs_mapping.target_size()); - let mut o2i_lhs = lhs_mapping + let o2i_lhs = lhs_mapping .inverse() .ok_or_else(|| anyhow!("lhs_mapping should be invertible"))?; - let mut o2i_rhs = rhs_mapping + let o2i_rhs = rhs_mapping .inverse() .ok_or_else(|| anyhow!("rhs_mapping should be invertible"))?; for output_idx in 0..lhs_mapping.target_size() { @@ -152,5 +150,5 @@ impl ExprRewritable for StreamKeyedMerge { } impl ExprVisitable for StreamKeyedMerge { - fn visit_exprs(&self, v: &mut dyn ExprVisitor) {} + fn visit_exprs(&self, _v: &mut dyn ExprVisitor) {} } diff --git a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs index 21a44677d936..4f312524ee40 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs @@ -16,19 +16,15 @@ use fixedbitset::FixedBitSet; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::DataType; -use risingwave_common::util::sort_util::OrderType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use crate::expr::{ExprRewriter, ExprVisitor, InputRef, InputRefDisplay, Literal}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef}; use crate::optimizer::plan_node::stream::StreamPlanRef; -use crate::optimizer::plan_node::utils::{ - childless_record, watermark_pretty, Distill, IndicesDisplay, -}; +use crate::optimizer::plan_node::utils::{childless_record, watermark_pretty, Distill}; use crate::optimizer::plan_node::{ - ExprRewritable, PlanAggCall, PlanBase, PlanNode, PlanTreeNodeUnary, Stream, - StreamGlobalApproxPercentile, StreamHopWindow, StreamNode, + ExprRewritable, PlanAggCall, PlanBase, PlanTreeNodeUnary, Stream, StreamNode, }; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::PlanRef; @@ -127,5 +123,5 @@ impl ExprRewritable for StreamLocalApproxPercentile { } impl ExprVisitable for StreamLocalApproxPercentile { - fn visit_exprs(&self, v: &mut dyn ExprVisitor) {} + fn visit_exprs(&self, _v: &mut dyn ExprVisitor) {} } diff --git a/src/frontend/src/optimizer/rule/distinct_agg_rule.rs b/src/frontend/src/optimizer/rule/distinct_agg_rule.rs index 919fcde2bff5..9d4634335617 100644 --- a/src/frontend/src/optimizer/rule/distinct_agg_rule.rs +++ b/src/frontend/src/optimizer/rule/distinct_agg_rule.rs @@ -57,8 +57,10 @@ impl Rule for DistinctAggRule { c.agg_kind ); let agg_kind_ok = !matches!(c.agg_kind, agg_kinds::simply_cannot_two_phase!()); - let order_ok = matches!(c.agg_kind, agg_kinds::result_unaffected_by_order_by!() | AggKind::ApproxPercentile) - || c.order_by.is_empty(); + let order_ok = matches!( + c.agg_kind, + agg_kinds::result_unaffected_by_order_by!() | AggKind::ApproxPercentile + ) || c.order_by.is_empty(); agg_kind_ok && order_ok }) { tracing::warn!("DistinctAggRule: unsupported agg kind, fallback to backend impl"); From 556ea556fae10d6b87c04477bcc6b33c06f53cf6 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 22 Jul 2024 21:21:11 +0800 Subject: [PATCH 20/33] remove wrong example --- .../impl/src/aggregate/approx_percentile.rs | 39 ------------------- 1 file changed, 39 deletions(-) diff --git a/src/expr/impl/src/aggregate/approx_percentile.rs b/src/expr/impl/src/aggregate/approx_percentile.rs index 9b5b0c9e4ac0..c66193c10756 100644 --- a/src/expr/impl/src/aggregate/approx_percentile.rs +++ b/src/expr/impl/src/aggregate/approx_percentile.rs @@ -21,45 +21,6 @@ use risingwave_common_estimate_size::EstimateSize; use risingwave_expr::aggregate::{AggCall, AggStateDyn, AggregateFunction, AggregateState}; use risingwave_expr::{build_aggregate, Result}; -/// Computes the approx percentile, a value corresponding to the specified fraction within the -/// ordered set of aggregated argument values. This will interpolate between adjacent input items if -/// needed. -/// -/// ```slt -/// statement ok -/// create table t(x int, y bigint, z real, w double, v varchar); -/// -/// statement ok -/// insert into t values(1,10,100,1000,'10000'),(2,20,200,2000,'20000'),(3,30,300,3000,'30000'); -/// -/// query R -/// select percentile_cont(0.45) within group (order by x desc) from t; -/// ---- -/// 2.1 -/// -/// query R -/// select percentile_cont(0.45) within group (order by y desc) from t; -/// ---- -/// 21 -/// -/// query R -/// select percentile_cont(0.45) within group (order by z desc) from t; -/// ---- -/// 210 -/// -/// query R -/// select percentile_cont(0.45) within group (order by w desc) from t; -/// ---- -/// 2100 -/// -/// query R -/// select percentile_cont(NULL) within group (order by w desc) from t; -/// ---- -/// NULL -/// -/// statement ok -/// drop table t; -/// ``` #[build_aggregate("approx_percentile(float8) -> float8")] fn build(agg: &AggCall) -> Result> { let fraction = agg.direct_args[0] From 0abeac10a5f80c7bd0f6ca260ecc99d44f8a024d Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 22 Jul 2024 21:30:27 +0800 Subject: [PATCH 21/33] reuse functionality --- src/frontend/src/binder/expr/function.rs | 114 +++++++---------------- 1 file changed, 33 insertions(+), 81 deletions(-) diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 453de89f3dde..10905bc85c01 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -424,6 +424,36 @@ impl Binder { )?))) } + fn decimal_to_float64(decimal_expr: &mut ExprImpl, kind: &AggKind) -> Result<()> { + if decimal_expr.cast_implicit_mut(DataType::Float64).is_err() { + return Err(ErrorCode::InvalidInputSyntax(format!( + "direct arg in `{}` must be castable to float64", + kind + )) + .into()); + } + + let Some(Ok(fraction_datum)) = decimal_expr.try_fold_const() else { + bail_not_implemented!( + issue = 14079, + "variable as direct argument of ordered-set aggregate", + ); + }; + + if let Some(ref fraction_value) = fraction_datum + && !(0.0..=1.0).contains(&fraction_value.as_float64().0) + { + return Err(ErrorCode::InvalidInputSyntax(format!( + "direct arg in `{}` must between 0.0 and 1.0", + kind + )) + .into()); + } + // note that the fraction can be NULL + *decimal_expr = Literal::new(fraction_datum, DataType::Float64).into(); + Ok(()) + } + fn bind_ordered_set_agg( &mut self, f: Function, @@ -474,33 +504,7 @@ impl Binder { [fraction], [arg], ) => { - if fraction.cast_implicit_mut(DataType::Float64).is_err() { - return Err(ErrorCode::InvalidInputSyntax(format!( - "direct arg in `{}` must be castable to float64", - kind - )) - .into()); - } - - let Some(Ok(fraction_datum)) = fraction.try_fold_const() else { - bail_not_implemented!( - issue = 14079, - "variable as direct argument of ordered-set aggregate", - ); - }; - - if let Some(ref fraction_value) = fraction_datum - && !(0.0..=1.0).contains(&fraction_value.as_float64().0) - { - return Err(ErrorCode::InvalidInputSyntax(format!( - "direct arg in `{}` must between 0.0 and 1.0", - kind - )) - .into()); - } - // note that the fraction can be NULL - *fraction = Literal::new(fraction_datum, DataType::Float64).into(); - + Self::decimal_to_float64(fraction, &kind)?; if matches!(&kind, AggKind::Builtin(PbAggKind::PercentileCont)) { arg.cast_implicit_mut(DataType::Float64).map_err(|_| { ErrorCode::InvalidInputSyntax(format!( @@ -512,60 +516,8 @@ impl Binder { } (AggKind::Builtin(PbAggKind::Mode), [], [_arg]) => {} (AggKind::Builtin(PbAggKind::ApproxPercentile), [percentile, relative_error], [_percentile_col]) => { - percentile - .cast_implicit_mut(DataType::Float64) - .map_err(|_e| { - ErrorCode::InvalidInputSyntax(format!( - "direct arg in `{}` must be castable to float64", - kind - )) - })?; - - let Some(Ok(percentile_datum)) = percentile.try_fold_const() else { - bail_not_implemented!( - issue = 14079, - "variable as direct argument of ordered-set aggregate", - ) - }; - - if let Some(ref percentile_scalar) = percentile_datum - && !(0.0..=1.0).contains(&percentile_scalar.as_float64().0) - { - return Err(ErrorCode::InvalidInputSyntax(format!( - "direct arg in `{}` must between 0.0 and 1.0", - kind - )) - .into()); - } - // note that the fraction can be NULL - *percentile = Literal::new(percentile_datum, DataType::Float64).into(); - - relative_error - .cast_implicit_mut(DataType::Float64) - .map_err(|_e| { - ErrorCode::InvalidInputSyntax(format!( - "direct arg in `{}` must be castable to float64", - kind - )) - })?; - let Some(Ok(relative_error_datum)) = relative_error.try_fold_const() else { - bail_not_implemented!( - issue = 14079, - "variable as direct argument of ordered-set aggregate", - ) - }; - - if let Some(ref relative_error_scalar) = relative_error_datum - && !(0.0..=1.0).contains(&relative_error_scalar.as_float64().0) - { - return Err(ErrorCode::InvalidInputSyntax(format!( - "direct arg in `{}` must between 0.0 and 1.0", - kind - )) - .into()); - } - // note that the fraction can be NULL - *relative_error = Literal::new(relative_error_datum, DataType::Float64).into(); + Self::decimal_to_float64(percentile, &kind)?; + Self::decimal_to_float64(relative_error, &kind)?; } _ => { return Err(ErrorCode::InvalidInputSyntax(format!( From 436c253e9a24abfbfef95e08866f83a8c2d6d90f Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 22 Jul 2024 22:14:51 +0800 Subject: [PATCH 22/33] cleanup two-phase approx percentile --- .../tests/testdata/output/agg.yaml | 25 ++++-- .../src/optimizer/plan_node/logical_agg.rs | 86 +++++++++---------- 2 files changed, 57 insertions(+), 54 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index e1e5a6d57053..c27bca7d125a 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1969,14 +1969,23 @@ stream_plan: |- StreamMaterialize { columns: [x, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamKeyedMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } - ├─StreamGlobalApproxPercentile - │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } - │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamGlobalApproxPercentile - └─StreamLocalApproxPercentile { percentile_col: $expr4, quantile: 0.5:Float64, relative_error: 0.01:Float64 } - └─StreamProject { exprs: [t.v1::Float64 as $expr3, t.v2::Float64 as $expr4, t._row_id] } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + ├─StreamKeyedMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } + │ ├─StreamGlobalApproxPercentile + │ │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ │ └─StreamShare { id: 2 } + │ │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } + │ │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + │ └─StreamGlobalApproxPercentile + │ └─StreamLocalApproxPercentile { percentile_col: $expr2, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + │ └─StreamShare { id: 2 } + │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } + │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamSimpleAgg { aggs: [count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [] } + └─StreamShare { id: 2 } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: test simple approx_percentile with different approx_percentile interleaved with stateless simple aggs sql: | CREATE TABLE t (v1 int, v2 int); diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 1fcaa47de342..6beb7b94b94f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -71,44 +71,29 @@ impl LogicalAgg { debug_assert!(self.group_key().is_empty()); let mut core = self.core.clone(); - // First, handle approx percentile. - let has_approx_percentile = self - .agg_calls() - .iter() - .any(|agg_call| agg_call.agg_kind == AggKind::ApproxPercentile); - let approx_percentile_info = if has_approx_percentile { - let SeparatedAggInfo { - approx_percentile_agg_calls, - non_approx_percentile_agg_calls, - approx_percentile_col_mapping, - non_approx_percentile_col_mapping, - } = self.separate_normal_and_special_agg(); - let requires_keyed_merge = !non_approx_percentile_agg_calls.is_empty(); - if requires_keyed_merge { - let stream_input: PlanRef = StreamShare::new_from_input(stream_input).into(); - let approx_percentile_agg: PlanRef = self.build_approx_percentile_aggs( - stream_input.clone(), - &approx_percentile_agg_calls, - ); - - core.input = stream_input; - core.agg_calls = non_approx_percentile_agg_calls; - Some(( - approx_percentile_agg, - approx_percentile_col_mapping, - non_approx_percentile_col_mapping, - )) - } else { - let approx_percentile_agg: PlanRef = self.build_approx_percentile_aggs( - stream_input.clone(), - &approx_percentile_agg_calls, - ); - return Ok(approx_percentile_agg); - } + // ====== Handle approx percentile aggs + let SeparatedAggInfo { + approx_percentile_agg_calls, + non_approx_percentile_agg_calls, + approx_percentile_col_mapping, + non_approx_percentile_col_mapping, + } = self.separate_normal_and_special_agg(); + + let needs_keyed_merge = (non_approx_percentile_agg_calls.len() >= 1 && approx_percentile_agg_calls.len() >= 1) || approx_percentile_agg_calls.len() >= 2; + core.input = if needs_keyed_merge { + // If there's keyed merge, we need to share the input. + StreamShare::new_from_input(stream_input.clone()).into() } else { - core.input = stream_input; - None + stream_input }; + core.agg_calls = non_approx_percentile_agg_calls; + + let approx_percentile = self.build_approx_percentile_aggs( + core.input.clone(), + &approx_percentile_agg_calls, + ); + + // ====== Handle normal aggs let total_agg_calls = core .agg_calls .iter() @@ -122,14 +107,20 @@ impl LogicalAgg { RequiredDist::single().enforce_if_not_satisfies(local_agg.into(), &Order::any())?; let global_agg = new_stream_simple_agg(Agg::new(total_agg_calls, IndexSet::empty(), exchange)); - if let Some((approx_percentile_agg, lhs_mapping, rhs_mapping)) = approx_percentile_info { - let keyed_merge = StreamKeyedMerge::new( - approx_percentile_agg, - global_agg.into(), - lhs_mapping, - rhs_mapping, - )?; - Ok(keyed_merge.into()) + + // ====== Merge approx percentile and normal aggs + if let Some(approx_percentile) = approx_percentile { + if needs_keyed_merge { + let keyed_merge = StreamKeyedMerge::new( + approx_percentile, + global_agg.into(), + approx_percentile_col_mapping, + non_approx_percentile_col_mapping, + )?; + Ok(keyed_merge.into()) + } else { + Ok(approx_percentile.into()) + } } else { Ok(global_agg.into()) } @@ -358,7 +349,10 @@ impl LogicalAgg { &self, input: PlanRef, approx_percentile_agg_call: &[PlanAggCall], - ) -> PlanRef { + ) -> Option { + if approx_percentile_agg_call.is_empty() { + return None; + } let approx_percentile_plans = approx_percentile_agg_call .iter() .map(|agg_call| self.build_approx_percentile_agg(input.clone(), agg_call)) @@ -377,7 +371,7 @@ impl LogicalAgg { .expect("failed to build keyed merge"); acc = keyed_merge.into(); } - acc + Some(acc) } pub fn core(&self) -> &Agg { From a2dfaae8e494fb71d9a4dd63af19c420880b73e9 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 22 Jul 2024 22:16:58 +0800 Subject: [PATCH 23/33] remove resolved comments --- src/frontend/src/optimizer/plan_node/logical_agg.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 6beb7b94b94f..74834ed3824b 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -290,7 +290,6 @@ impl LogicalAgg { } } - // TODO(kwannoel): Handle multiple approx_percentile. fn separate_normal_and_special_agg(&self) -> SeparatedAggInfo { let estimated_len = self.agg_calls().len() - 1; let mut approx_percentile_agg_calls = Vec::with_capacity(estimated_len); @@ -748,7 +747,6 @@ impl LogicalAggBuilder { /// 2. Add the agg call to current `Agg`, and return an `InputRef` to it. /// /// Note that the rewriter does not traverse into inputs of agg calls. - /// FIXME(kwannoel): Rewrite the desc approx percentile for descending order. fn try_rewrite_agg_call(&mut self, mut agg_call: AggCall) -> Result { if matches!(agg_call.agg_kind, agg_kinds::must_have_order_by!()) && agg_call.order_by.sort_exprs.is_empty() From c3eb3a1bab652feed107569faec0eef8e7a28ce1 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 22 Jul 2024 22:23:09 +0800 Subject: [PATCH 24/33] resolve comments --- src/frontend/src/optimizer/plan_node/logical_agg.rs | 1 + src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs | 3 --- 2 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 74834ed3824b..8e645ab4b4aa 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -645,6 +645,7 @@ impl LogicalAggBuilder { } AggKind::ApproxPercentile => { if agg_call.order_by.sort_exprs[0].order_type == OrderType::descending() { + // Rewrite DESC into 1.0-percentile for approx_percentile. let prev_percentile = agg_call.direct_args[0].clone(); let new_percentile = 1.0 - prev_percentile diff --git a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs index 6e6826ef6a44..2df7d3eafa57 100644 --- a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs +++ b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs @@ -74,7 +74,6 @@ impl StreamKeyedMerge { let schema = Schema::new(schema_fields); let watermark_columns = FixedBitSet::with_capacity(schema.fields.len()); - // FIXME: schema is wrong. let base = PlanBase::new_stream( lhs_input.ctx(), schema, @@ -99,8 +98,6 @@ impl StreamKeyedMerge { impl Distill for StreamKeyedMerge { fn distill<'a>(&self) -> XmlNode<'a> { let mut out = Vec::with_capacity(1); - // out.push(("lhs_col_mapping", Pretty::debug(&self.lhs_mapping))); - // out.push(("rhs_col_mapping", Pretty::debug(&self.rhs_mapping))); if self.base.ctx().is_explain_verbose() { let f = |t| Pretty::debug(&t); From 8b836fc5a10998622c6391df78cce932dd9f422a Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Mon, 22 Jul 2024 22:30:05 +0800 Subject: [PATCH 25/33] fmt --- src/frontend/src/optimizer/plan_node/logical_agg.rs | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 8e645ab4b4aa..07292e57e6b5 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -79,7 +79,9 @@ impl LogicalAgg { non_approx_percentile_col_mapping, } = self.separate_normal_and_special_agg(); - let needs_keyed_merge = (non_approx_percentile_agg_calls.len() >= 1 && approx_percentile_agg_calls.len() >= 1) || approx_percentile_agg_calls.len() >= 2; + let needs_keyed_merge = (!non_approx_percentile_agg_calls.is_empty() + && !approx_percentile_agg_calls.is_empty()) + || approx_percentile_agg_calls.len() >= 2; core.input = if needs_keyed_merge { // If there's keyed merge, we need to share the input. StreamShare::new_from_input(stream_input.clone()).into() @@ -88,10 +90,8 @@ impl LogicalAgg { }; core.agg_calls = non_approx_percentile_agg_calls; - let approx_percentile = self.build_approx_percentile_aggs( - core.input.clone(), - &approx_percentile_agg_calls, - ); + let approx_percentile = + self.build_approx_percentile_aggs(core.input.clone(), &approx_percentile_agg_calls); // ====== Handle normal aggs let total_agg_calls = core @@ -119,7 +119,7 @@ impl LogicalAgg { )?; Ok(keyed_merge.into()) } else { - Ok(approx_percentile.into()) + Ok(approx_percentile) } } else { Ok(global_agg.into()) From 32f63013582abed8b4a38323ce25b91828633b4e Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 24 Jul 2024 10:21:57 +0800 Subject: [PATCH 26/33] Update src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs Co-authored-by: Eric Fu --- src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs index 2df7d3eafa57..2f09d2267f18 100644 --- a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs +++ b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs @@ -62,7 +62,6 @@ impl StreamKeyedMerge { if let Some(lhs_idx) = o2i_lhs.try_map(output_idx) { schema_fields.push(lhs_input.schema().fields()[lhs_idx].clone()); } else if let Some(rhs_idx) = o2i_rhs.try_map(output_idx) { - println!("rhs schema: {:?}", rhs_input.schema().fields()); schema_fields.push(rhs_input.schema().fields()[rhs_idx].clone()); } else { bail!( From c20bf2c56acb0a5380c585b6a63319823739702e Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 24 Jul 2024 11:03:07 +0800 Subject: [PATCH 27/33] fix conflicts --- src/expr/core/src/aggregate/def.rs | 9 +++++---- src/frontend/src/binder/expr/function.rs | 8 ++++++-- src/frontend/src/expr/agg_call.rs | 1 - src/frontend/src/optimizer/plan_node/generic/agg.rs | 5 +++-- src/frontend/src/optimizer/plan_node/logical_agg.rs | 4 ++-- src/frontend/src/optimizer/rule/distinct_agg_rule.rs | 3 ++- 6 files changed, 18 insertions(+), 12 deletions(-) diff --git a/src/expr/core/src/aggregate/def.rs b/src/expr/core/src/aggregate/def.rs index 1fb985585cdf..d35077698da6 100644 --- a/src/expr/core/src/aggregate/def.rs +++ b/src/expr/core/src/aggregate/def.rs @@ -313,7 +313,7 @@ pub mod agg_kinds { | PbAggKind::StddevSamp | PbAggKind::VarPop | PbAggKind::VarSamp - | PbAggKind::Grouping, + | PbAggKind::Grouping // ApproxPercentile always uses custom agg executors, // rather than an aggregation operator | PbAggKind::ApproxPercentile @@ -446,13 +446,14 @@ pub mod agg_kinds { macro_rules! ordered_set { () => { AggKind::Builtin( - PbAggKind::PercentileCont | PbAggKind::PercentileDisc | PbAggKind::Mode | PbAggKind::ApproxPercentile + PbAggKind::PercentileCont + | PbAggKind::PercentileDisc + | PbAggKind::Mode + | PbAggKind::ApproxPercentile, ) }; } pub use ordered_set; - - use crate::aggregate::{AggKind, PbAggKind}; } impl AggKind { diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 10905bc85c01..b9610819756b 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -515,7 +515,11 @@ impl Binder { } } (AggKind::Builtin(PbAggKind::Mode), [], [_arg]) => {} - (AggKind::Builtin(PbAggKind::ApproxPercentile), [percentile, relative_error], [_percentile_col]) => { + ( + AggKind::Builtin(PbAggKind::ApproxPercentile), + [percentile, relative_error], + [_percentile_col], + ) => { Self::decimal_to_float64(percentile, &kind)?; Self::decimal_to_float64(relative_error, &kind)?; } @@ -579,7 +583,7 @@ impl Binder { if matches!( kind, AggKind::Builtin(PbAggKind::ApproxCountDistinct) - | AggKind::Builtin(PbAggKind::ApproxPercentile) + | AggKind::Builtin(PbAggKind::ApproxPercentile) ) { return Err(ErrorCode::InvalidInputSyntax(format!( "DISTINCT is not allowed for approximate aggregation `{}`", diff --git a/src/frontend/src/expr/agg_call.rs b/src/frontend/src/expr/agg_call.rs index c058112daee5..452d37652d34 100644 --- a/src/frontend/src/expr/agg_call.rs +++ b/src/frontend/src/expr/agg_call.rs @@ -41,7 +41,6 @@ impl std::fmt::Debug for AggCall { .field("distinct", &self.distinct) .field("order_by", &self.order_by) .field("direct_args", &self.direct_args) - .field("user_defined", &self.user_defined.is_some()) .finish() } else { let mut builder = f.debug_tuple(&format!("{}", self.agg_kind)); diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 419b6058f7c6..c0f41b8d82c7 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -107,7 +107,8 @@ impl Agg { let agg_kind_ok = !matches!(call.agg_kind, agg_kinds::simply_cannot_two_phase!()); let order_ok = matches!( call.agg_kind, - agg_kinds::result_unaffected_by_order_by!() | AggKind::ApproxPercentile + agg_kinds::result_unaffected_by_order_by!() + | AggKind::Builtin(PbAggKind::ApproxPercentile) ) || call.order_by.is_empty(); let distinct_ok = matches!(call.agg_kind, agg_kinds::result_unaffected_by_distinct!()) @@ -135,7 +136,7 @@ impl Agg { self.agg_calls.iter().all(|c| { matches!(c.agg_kind, agg_kinds::single_value_state!()) || (matches!(c.agg_kind, agg_kinds::single_value_state_iff_in_append_only!() if stream_input_append_only)) - || (matches!(c.agg_kind, AggKind::ApproxPercentile)) + || (matches!(c.agg_kind, AggKind::Builtin(PbAggKind::ApproxPercentile))) }) } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 07292e57e6b5..67ada8c704b3 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -297,7 +297,7 @@ impl LogicalAgg { let mut approx_percentile_col_mapping = Vec::with_capacity(estimated_len); let mut non_approx_percentile_col_mapping = Vec::with_capacity(estimated_len); for (output_idx, agg_call) in self.agg_calls().iter().enumerate() { - if agg_call.agg_kind == AggKind::ApproxPercentile { + if agg_call.agg_kind == AggKind::Builtin(PbAggKind::ApproxPercentile) { approx_percentile_agg_calls.push(agg_call.clone()); approx_percentile_col_mapping.push(Some(output_idx)); } else { @@ -643,7 +643,7 @@ impl LogicalAggBuilder { _ => unreachable!(), } } - AggKind::ApproxPercentile => { + AggKind::Builtin(PbAggKind::ApproxPercentile) => { if agg_call.order_by.sort_exprs[0].order_type == OrderType::descending() { // Rewrite DESC into 1.0-percentile for approx_percentile. let prev_percentile = agg_call.direct_args[0].clone(); diff --git a/src/frontend/src/optimizer/rule/distinct_agg_rule.rs b/src/frontend/src/optimizer/rule/distinct_agg_rule.rs index 9d4634335617..52aad1336a7b 100644 --- a/src/frontend/src/optimizer/rule/distinct_agg_rule.rs +++ b/src/frontend/src/optimizer/rule/distinct_agg_rule.rs @@ -59,7 +59,8 @@ impl Rule for DistinctAggRule { let agg_kind_ok = !matches!(c.agg_kind, agg_kinds::simply_cannot_two_phase!()); let order_ok = matches!( c.agg_kind, - agg_kinds::result_unaffected_by_order_by!() | AggKind::ApproxPercentile + agg_kinds::result_unaffected_by_order_by!() + | AggKind::Builtin(PbAggKind::ApproxPercentile) ) || c.order_by.is_empty(); agg_kind_ok && order_ok }) { From 6821023427366f4ec7c3b1be32d69c393d393fa0 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 24 Jul 2024 11:18:20 +0800 Subject: [PATCH 28/33] add quantile and relative error to global approx percentile --- .../tests/testdata/output/agg.yaml | 18 ++++++------- .../src/optimizer/plan_node/logical_agg.rs | 6 +++-- .../stream_global_approx_percentile.rs | 27 ++++++++++++++----- 3 files changed, 34 insertions(+), 17 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index c27bca7d125a..81e4185b128c 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1890,7 +1890,7 @@ └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: |- StreamMaterialize { columns: [approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamGlobalApproxPercentile + └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -1906,7 +1906,7 @@ stream_plan: |- StreamMaterialize { columns: [approx_percentile, sum], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamKeyedMerge { output: [approx_percentile:Float64, sum(sum(t.v1)):Int64] } - ├─StreamGlobalApproxPercentile + ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ └─StreamShare { id: 2 } │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v1, t._row_id] } @@ -1930,7 +1930,7 @@ StreamMaterialize { columns: [s1, approx_percentile, s2, count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum(sum(t.v1)), sum0(count(t.v1))] } └─StreamKeyedMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count(t.v1)):Int64] } - ├─StreamGlobalApproxPercentile + ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ └─StreamShare { id: 2 } │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } @@ -1953,7 +1953,7 @@ stream_plan: |- StreamMaterialize { columns: [x, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [approx_percentile, approx_percentile] } - └─StreamGlobalApproxPercentile + └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } @@ -1970,12 +1970,12 @@ StreamMaterialize { columns: [x, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamKeyedMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } ├─StreamKeyedMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } - │ ├─StreamGlobalApproxPercentile + │ ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ │ └─StreamShare { id: 2 } │ │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } │ │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - │ └─StreamGlobalApproxPercentile + │ └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ └─StreamLocalApproxPercentile { percentile_col: $expr2, quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ └─StreamShare { id: 2 } │ └─StreamProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2, t._row_id] } @@ -1999,12 +1999,12 @@ StreamMaterialize { columns: [s1, x, count, s2, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamKeyedMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count):Int64, sum(sum(t.v2)):Int64, approx_percentile:Float64] } ├─StreamKeyedMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } - │ ├─StreamGlobalApproxPercentile + │ ├─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ │ └─StreamShare { id: 2 } │ │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } │ │ └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - │ └─StreamGlobalApproxPercentile + │ └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ └─StreamLocalApproxPercentile { percentile_col: $expr2, quantile: 0.5:Float64, relative_error: 0.01:Float64 } │ └─StreamShare { id: 2 } │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2, t._row_id] } @@ -2027,7 +2027,7 @@ stream_plan: |- StreamMaterialize { columns: [s1, approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamKeyedMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64] } - ├─StreamGlobalApproxPercentile + ├─StreamGlobalApproxPercentile { quantile: 0.8:Float64, relative_error: 0.01:Float64 } │ └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.8:Float64, relative_error: 0.01:Float64 } │ └─StreamShare { id: 2 } │ └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr1, t._row_id] } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index 67ada8c704b3..f70098bbb091 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -326,8 +326,10 @@ impl LogicalAgg { ) -> PlanRef { let local_approx_percentile = StreamLocalApproxPercentile::new(input, approx_percentile_agg_call); - let global_approx_percentile = - StreamGlobalApproxPercentile::new(local_approx_percentile.into()); + let global_approx_percentile = StreamGlobalApproxPercentile::new( + local_approx_percentile.into(), + approx_percentile_agg_call, + ); global_approx_percentile.into() } diff --git a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs index caab6da90890..4741c28661bb 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs @@ -13,18 +13,18 @@ // limitations under the License. use fixedbitset::FixedBitSet; -use pretty_xmlish::XmlNode; +use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::catalog::{Field, Schema}; use risingwave_common::types::DataType; use risingwave_pb::stream_plan::stream_node::PbNodeBody; -use crate::expr::{ExprRewriter, ExprVisitor}; +use crate::expr::{ExprRewriter, ExprVisitor, Literal}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::stream::StreamPlanRef; use crate::optimizer::plan_node::utils::{childless_record, Distill}; use crate::optimizer::plan_node::{ - ExprRewritable, PlanBase, PlanTreeNodeUnary, Stream, StreamNode, + ExprRewritable, PlanAggCall, PlanBase, PlanTreeNodeUnary, Stream, StreamNode, }; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -34,10 +34,14 @@ use crate::PlanRef; pub struct StreamGlobalApproxPercentile { pub base: PlanBase, input: PlanRef, + /// Quantile + quantile: Literal, + /// Used to compute the exponent bucket base. + relative_error: Literal, } impl StreamGlobalApproxPercentile { - pub fn new(input: PlanRef) -> Self { + pub fn new(input: PlanRef, approx_percentile_agg_call: &PlanAggCall) -> Self { let schema = Schema::new(vec![Field::new("approx_percentile", DataType::Float64)]); let watermark_columns = FixedBitSet::with_capacity(1); let base = PlanBase::new_stream( @@ -51,13 +55,22 @@ impl StreamGlobalApproxPercentile { watermark_columns, input.columns_monotonicity().clone(), ); - Self { base, input } + Self { + base, + input, + quantile: approx_percentile_agg_call.direct_args[0].clone(), + relative_error: approx_percentile_agg_call.direct_args[1].clone(), + } } } impl Distill for StreamGlobalApproxPercentile { fn distill<'a>(&self) -> XmlNode<'a> { - childless_record("StreamGlobalApproxPercentile", vec![]) + let out = vec![ + ("quantile", Pretty::debug(&self.quantile)), + ("relative_error", Pretty::debug(&self.relative_error)), + ]; + childless_record("StreamGlobalApproxPercentile", out) } } @@ -70,6 +83,8 @@ impl PlanTreeNodeUnary for StreamGlobalApproxPercentile { Self { base: self.base.clone(), input, + quantile: self.quantile.clone(), + relative_error: self.relative_error.clone(), } } } From 4af937f6997341e8a9e16cbee5812a3446426d9d Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 24 Jul 2024 11:22:46 +0800 Subject: [PATCH 29/33] docs for stream keyed merge --- src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs index 2f09d2267f18..e84a2c9dd9b0 100644 --- a/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs +++ b/src/frontend/src/optimizer/plan_node/stream_keyed_merge.rs @@ -32,6 +32,10 @@ use crate::optimizer::plan_node::{ use crate::stream_fragmenter::BuildFragmentGraphState; use crate::PlanRef; +/// `StreamKeyedMerge` is used for merging two streams with the same stream key and distribution. +/// It will buffer the outputs from its input streams until we receive a barrier. +/// On receiving a barrier, it will `Project` their outputs according +/// to the provided `lhs_mapping` and `rhs_mapping`. #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamKeyedMerge { pub base: PlanBase, @@ -51,6 +55,8 @@ impl StreamKeyedMerge { rhs_mapping: ColIndexMapping, ) -> Result { assert_eq!(lhs_mapping.target_size(), rhs_mapping.target_size()); + assert_eq!(lhs_input.distribution(), rhs_input.distribution()); + assert_eq!(lhs_input.stream_key(), rhs_input.stream_key()); let mut schema_fields = Vec::with_capacity(lhs_mapping.target_size()); let o2i_lhs = lhs_mapping .inverse() From 6cf0dd1650e0eab658c4445697e2a209eb9ee60b Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 24 Jul 2024 11:27:49 +0800 Subject: [PATCH 30/33] fill with todos for agg framework --- .../impl/src/aggregate/approx_percentile.rs | 53 ++++--------------- 1 file changed, 10 insertions(+), 43 deletions(-) diff --git a/src/expr/impl/src/aggregate/approx_percentile.rs b/src/expr/impl/src/aggregate/approx_percentile.rs index c66193c10756..7f9ae3fb4a53 100644 --- a/src/expr/impl/src/aggregate/approx_percentile.rs +++ b/src/expr/impl/src/aggregate/approx_percentile.rs @@ -15,7 +15,6 @@ use std::ops::Range; use risingwave_common::array::*; -use risingwave_common::row::Row; use risingwave_common::types::*; use risingwave_common_estimate_size::EstimateSize; use risingwave_expr::aggregate::{AggCall, AggStateDyn, AggregateFunction, AggregateState}; @@ -29,6 +28,7 @@ fn build(agg: &AggCall) -> Result> { Ok(Box::new(ApproxPercentile { fraction })) } +#[allow(dead_code)] pub struct ApproxPercentile { fraction: Option, } @@ -38,14 +38,6 @@ struct State(Vec); impl AggStateDyn for State {} -impl ApproxPercentile { - fn add_datum(&self, state: &mut State, datum_ref: DatumRef<'_>) { - if let Some(datum) = datum_ref.to_owned_datum() { - state.0.push((*datum.as_float64()).into()); - } - } -} - #[async_trait::async_trait] impl AggregateFunction for ApproxPercentile { fn return_type(&self) -> DataType { @@ -53,48 +45,23 @@ impl AggregateFunction for ApproxPercentile { } fn create_state(&self) -> Result { - Ok(AggregateState::Any(Box::::default())) + todo!() } - async fn update(&self, state: &mut AggregateState, input: &StreamChunk) -> Result<()> { - let state = state.downcast_mut(); - for (_, row) in input.rows() { - self.add_datum(state, row.datum_at(0)); - } - Ok(()) + async fn update(&self, _state: &mut AggregateState, _input: &StreamChunk) -> Result<()> { + todo!() } async fn update_range( &self, - state: &mut AggregateState, - input: &StreamChunk, - range: Range, + _state: &mut AggregateState, + _input: &StreamChunk, + _range: Range, ) -> Result<()> { - let state = state.downcast_mut(); - for (_, row) in input.rows_in(range) { - self.add_datum(state, row.datum_at(0)); - } - Ok(()) + todo!() } - async fn get_result(&self, state: &AggregateState) -> Result { - let state = &state.downcast_ref::().0; - Ok( - if let Some(fraction) = self.fraction - && !state.is_empty() - { - let rn = fraction * (state.len() - 1) as f64; - let crn = f64::ceil(rn); - let frn = f64::floor(rn); - let result = if crn == frn { - state[crn as usize] - } else { - (crn - rn) * state[frn as usize] + (rn - frn) * state[crn as usize] - }; - Some(result.into()) - } else { - None - }, - ) + async fn get_result(&self, _state: &AggregateState) -> Result { + todo!() } } From 1dd7c2ba1dd3ef00cd62fcbfac23d52ff4832ec3 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 24 Jul 2024 11:54:28 +0800 Subject: [PATCH 31/33] fix tests --- src/frontend/src/binder/mod.rs | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/src/frontend/src/binder/mod.rs b/src/frontend/src/binder/mod.rs index 29ed638e4f0f..af1be41a711e 100644 --- a/src/frontend/src/binder/mod.rs +++ b/src/frontend/src/binder/mod.rs @@ -780,6 +780,7 @@ mod tests { UnnamedExpr( Function( Function { + aggregate: false, name: ObjectName( [ Ident { @@ -892,7 +893,9 @@ mod tests { select_items: [ AggCall( AggCall { - agg_kind: ApproxPercentile, + agg_kind: Builtin( + ApproxPercentile, + ), return_type: Float64, args: [ FunctionCall( @@ -933,30 +936,29 @@ mod tests { direct_args: [ Literal { data: Some( - Decimal( - Normalized( + Float64( + OrderedFloat( 0.5, ), ), ), data_type: Some( - Decimal, + Float64, ), }, Literal { data: Some( - Decimal( - Normalized( + Float64( + OrderedFloat( 0.01, ), ), ), data_type: Some( - Decimal, + Float64, ), }, ], - user_defined: false, }, ), ], From 1e195fc4af8f863182ede93fbe8bb7e4e42c091a Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 24 Jul 2024 16:18:14 +0800 Subject: [PATCH 32/33] refactor SeparatedAggInfo + reuse Field::with_name --- src/common/src/catalog/schema.rs | 9 ---- .../src/optimizer/plan_node/logical_agg.rs | 43 ++++++++++++------- .../stream_global_approx_percentile.rs | 5 ++- .../stream_local_approx_percentile.rs | 4 +- 4 files changed, 33 insertions(+), 28 deletions(-) diff --git a/src/common/src/catalog/schema.rs b/src/common/src/catalog/schema.rs index e4da4b48ddf5..113d9f804b3d 100644 --- a/src/common/src/catalog/schema.rs +++ b/src/common/src/catalog/schema.rs @@ -47,15 +47,6 @@ impl Field { name: self.name.to_string(), } } - - pub fn new(name: impl Into, data_type: DataType) -> Self { - Self { - data_type, - name: name.into(), - sub_fields: vec![], - type_name: String::new(), - } - } } impl From<&ColumnDesc> for Field { diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index f70098bbb091..b73f014f1318 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -45,11 +45,15 @@ use crate::utils::{ ColIndexMapping, ColIndexMappingRewriteExt, Condition, GroupBy, IndexSet, Substitute, }; +pub struct AggInfo { + pub calls: Vec, + pub col_mapping: ColIndexMapping, +} + +/// `SeparatedAggInfo` is used to separate normal and approx percentile aggs. pub struct SeparatedAggInfo { - pub approx_percentile_agg_calls: Vec, - pub non_approx_percentile_agg_calls: Vec, - pub approx_percentile_col_mapping: ColIndexMapping, - pub non_approx_percentile_col_mapping: ColIndexMapping, + normal: AggInfo, + approx: AggInfo, } /// `LogicalAgg` groups input data by their group key and computes aggregation functions. @@ -72,12 +76,16 @@ impl LogicalAgg { let mut core = self.core.clone(); // ====== Handle approx percentile aggs - let SeparatedAggInfo { - approx_percentile_agg_calls, - non_approx_percentile_agg_calls, - approx_percentile_col_mapping, - non_approx_percentile_col_mapping, - } = self.separate_normal_and_special_agg(); + let SeparatedAggInfo { normal, approx } = self.separate_normal_and_special_agg(); + + let AggInfo { + calls: non_approx_percentile_agg_calls, + col_mapping: non_approx_percentile_col_mapping, + } = normal; + let AggInfo { + calls: approx_percentile_agg_calls, + col_mapping: approx_percentile_col_mapping, + } = approx; let needs_keyed_merge = (!non_approx_percentile_agg_calls.is_empty() && !approx_percentile_agg_calls.is_empty()) @@ -305,18 +313,21 @@ impl LogicalAgg { non_approx_percentile_col_mapping.push(Some(output_idx)); } } - SeparatedAggInfo { - approx_percentile_agg_calls, - non_approx_percentile_agg_calls, - approx_percentile_col_mapping: ColIndexMapping::new( + let normal = AggInfo { + calls: approx_percentile_agg_calls, + col_mapping: ColIndexMapping::new( approx_percentile_col_mapping, self.agg_calls().len(), ), - non_approx_percentile_col_mapping: ColIndexMapping::new( + }; + let approx = AggInfo { + calls: non_approx_percentile_agg_calls, + col_mapping: ColIndexMapping::new( non_approx_percentile_col_mapping, self.agg_calls().len(), ), - } + }; + SeparatedAggInfo { normal, approx } } fn build_approx_percentile_agg( diff --git a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs index 4741c28661bb..22fe3b33ab69 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_approx_percentile.rs @@ -42,7 +42,10 @@ pub struct StreamGlobalApproxPercentile { impl StreamGlobalApproxPercentile { pub fn new(input: PlanRef, approx_percentile_agg_call: &PlanAggCall) -> Self { - let schema = Schema::new(vec![Field::new("approx_percentile", DataType::Float64)]); + let schema = Schema::new(vec![Field::with_name( + DataType::Float64, + "approx_percentile", + )]); let watermark_columns = FixedBitSet::with_capacity(1); let base = PlanBase::new_stream( input.ctx(), diff --git a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs index 4f312524ee40..a4fb2a602917 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_approx_percentile.rs @@ -43,8 +43,8 @@ pub struct StreamLocalApproxPercentile { impl StreamLocalApproxPercentile { pub fn new(input: PlanRef, approx_percentile_agg_call: &PlanAggCall) -> Self { let schema = Schema::new(vec![ - Field::new("bucket_id", DataType::Int64), - Field::new("count", DataType::Int64), + Field::with_name(DataType::Int64, "bucket_id"), + Field::with_name(DataType::Int64, "count"), ]); // FIXME(kwannoel): How does watermark work with FixedBitSet let watermark_columns = FixedBitSet::with_capacity(2); From 5e816a1011bc74bec6a59d8b94ae6e5d80b79412 Mon Sep 17 00:00:00 2001 From: Noel Kwan Date: Wed, 24 Jul 2024 19:35:19 +0800 Subject: [PATCH 33/33] fix --- src/frontend/src/optimizer/plan_node/logical_agg.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index b73f014f1318..54c59883b10f 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -314,16 +314,16 @@ impl LogicalAgg { } } let normal = AggInfo { - calls: approx_percentile_agg_calls, + calls: non_approx_percentile_agg_calls, col_mapping: ColIndexMapping::new( - approx_percentile_col_mapping, + non_approx_percentile_col_mapping, self.agg_calls().len(), ), }; let approx = AggInfo { - calls: non_approx_percentile_agg_calls, + calls: approx_percentile_agg_calls, col_mapping: ColIndexMapping::new( - non_approx_percentile_col_mapping, + approx_percentile_col_mapping, self.agg_calls().len(), ), };