diff --git a/src/common/function/src/aggrs/aggr_wrapper.rs b/src/common/function/src/aggrs/aggr_wrapper.rs index 0bfd52ed3b..7c97414554 100644 --- a/src/common/function/src/aggrs/aggr_wrapper.rs +++ b/src/common/function/src/aggrs/aggr_wrapper.rs @@ -26,6 +26,8 @@ use std::sync::Arc; use arrow::array::StructArray; use arrow_schema::Fields; +use common_telemetry::debug; +use datafusion::functions_aggregate::all_default_aggregate_functions; use datafusion::optimizer::analyzer::type_coercion::TypeCoercion; use datafusion::optimizer::AnalyzerRule; use datafusion::physical_planner::create_aggregate_expr_and_maybe_filter; @@ -39,6 +41,8 @@ use datafusion_expr::{ use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datatypes::arrow::datatypes::{DataType, Field}; +use crate::function_registry::FunctionRegistry; + /// Returns the name of the state function for the given aggregate function name. /// The state function is used to compute the state of the aggregate function. /// The state function's name is in the format `___state @@ -65,9 +69,9 @@ pub struct StateMergeHelper; #[derive(Debug, Clone)] pub struct StepAggrPlan { /// Upper merge plan, which is the aggregate plan that merges the states of the state function. - pub upper_merge: Arc, + pub upper_merge: LogicalPlan, /// Lower state plan, which is the aggregate plan that computes the state of the aggregate function. - pub lower_state: Arc, + pub lower_state: LogicalPlan, } pub fn get_aggr_func(expr: &Expr) -> Option<&datafusion_expr::expr::AggregateFunction> { @@ -83,6 +87,36 @@ pub fn get_aggr_func(expr: &Expr) -> Option<&datafusion_expr::expr::AggregateFun } impl StateMergeHelper { + /// Register all the `state` function of supported aggregate functions. + /// Note that can't register `merge` function here, as it needs to be created from the original aggregate function with given input types. + pub fn register(registry: &FunctionRegistry) { + let all_default = all_default_aggregate_functions(); + let greptime_custom_aggr_functions = registry.aggregate_functions(); + + // if our custom aggregate function have the same name as the default aggregate function, we will override it. + let supported = all_default + .into_iter() + .chain(greptime_custom_aggr_functions.into_iter().map(Arc::new)) + .collect::>(); + debug!( + "Registering state functions for supported: {:?}", + supported.iter().map(|f| f.name()).collect::>() + ); + + let state_func = supported.into_iter().filter_map(|f| { + StateWrapper::new((*f).clone()) + .inspect_err( + |e| common_telemetry::error!(e; "Failed to register state function for {:?}", f), + ) + .ok() + .map(AggregateUDF::new_from_impl) + }); + + for func in state_func { + registry.register_aggr(func); + } + } + /// Split an aggregate plan into two aggregate plans, one for the state function and one for the merge function. pub fn split_aggr_node(aggr_plan: Aggregate) -> datafusion_common::Result { let aggr = { @@ -166,18 +200,18 @@ impl StateMergeHelper { let lower_plan = LogicalPlan::Aggregate(lower); // update aggregate's output schema - let lower_plan = Arc::new(lower_plan.recompute_schema()?); + let lower_plan = lower_plan.recompute_schema()?; let mut upper = aggr.clone(); let aggr_plan = LogicalPlan::Aggregate(aggr); upper.aggr_expr = upper_aggr_exprs; - upper.input = lower_plan.clone(); + upper.input = Arc::new(lower_plan.clone()); // upper schema's output schema should be the same as the original aggregate plan's output schema - let upper_check = upper.clone(); - let upper_plan = Arc::new(LogicalPlan::Aggregate(upper_check).recompute_schema()?); + let upper_check = upper; + let upper_plan = LogicalPlan::Aggregate(upper_check).recompute_schema()?; if *upper_plan.schema() != *aggr_plan.schema() { return Err(datafusion_common::DataFusionError::Internal(format!( - "Upper aggregate plan's schema is not the same as the original aggregate plan's schema: \n[transformed]:{}\n[ original]{}", + "Upper aggregate plan's schema is not the same as the original aggregate plan's schema: \n[transformed]:{}\n[original]:{}", upper_plan.schema(), aggr_plan.schema() ))); } @@ -407,15 +441,18 @@ impl AggregateUDFImpl for MergeWrapper { &'a self, acc_args: datafusion_expr::function::AccumulatorArgs<'b>, ) -> datafusion_common::Result> { - if acc_args.schema.fields().len() != 1 - || !matches!(acc_args.schema.field(0).data_type(), DataType::Struct(_)) + if acc_args.exprs.len() != 1 + || !matches!( + acc_args.exprs[0].data_type(acc_args.schema)?, + DataType::Struct(_) + ) { return Err(datafusion_common::DataFusionError::Internal(format!( "Expected one struct type as input, got: {:?}", acc_args.schema ))); } - let input_type = acc_args.schema.field(0).data_type(); + let input_type = acc_args.exprs[0].data_type(acc_args.schema)?; let DataType::Struct(fields) = input_type else { return Err(datafusion_common::DataFusionError::Internal(format!( "Expected a struct type for input, got: {:?}", @@ -424,7 +461,7 @@ impl AggregateUDFImpl for MergeWrapper { }; let inner_accum = self.original_phy_expr.create_accumulator()?; - Ok(Box::new(MergeAccum::new(inner_accum, fields))) + Ok(Box::new(MergeAccum::new(inner_accum, &fields))) } fn as_any(&self) -> &dyn std::any::Any { diff --git a/src/common/function/src/aggrs/aggr_wrapper/tests.rs b/src/common/function/src/aggrs/aggr_wrapper/tests.rs index 23a516a619..652c6b678d 100644 --- a/src/common/function/src/aggrs/aggr_wrapper/tests.rs +++ b/src/common/function/src/aggrs/aggr_wrapper/tests.rs @@ -258,7 +258,7 @@ async fn test_sum_udaf() { ) .recompute_schema() .unwrap(); - assert_eq!(res.lower_state.as_ref(), &expected_lower_plan); + assert_eq!(&res.lower_state, &expected_lower_plan); let expected_merge_plan = LogicalPlan::Aggregate( Aggregate::try_new( @@ -297,7 +297,7 @@ async fn test_sum_udaf() { ) .unwrap(), ); - assert_eq!(res.upper_merge.as_ref(), &expected_merge_plan); + assert_eq!(&res.upper_merge, &expected_merge_plan); let phy_aggr_state_plan = DefaultPhysicalPlanner::default() .create_physical_plan(&res.lower_state, &ctx.state()) @@ -405,7 +405,7 @@ async fn test_avg_udaf() { let coerced_aggr_state_plan = TypeCoercion::new() .analyze(expected_aggr_state_plan.clone(), &Default::default()) .unwrap(); - assert_eq!(res.lower_state.as_ref(), &coerced_aggr_state_plan); + assert_eq!(&res.lower_state, &coerced_aggr_state_plan); assert_eq!( res.lower_state.schema().as_arrow(), &arrow_schema::Schema::new(vec![Field::new( @@ -456,7 +456,7 @@ async fn test_avg_udaf() { ) .unwrap(), ); - assert_eq!(res.upper_merge.as_ref(), &expected_merge_plan); + assert_eq!(&res.upper_merge, &expected_merge_plan); let phy_aggr_state_plan = DefaultPhysicalPlanner::default() .create_physical_plan(&coerced_aggr_state_plan, &ctx.state()) diff --git a/src/common/function/src/function_registry.rs b/src/common/function/src/function_registry.rs index 7e196f4a15..e84fb903b6 100644 --- a/src/common/function/src/function_registry.rs +++ b/src/common/function/src/function_registry.rs @@ -20,6 +20,7 @@ use datafusion_expr::AggregateUDF; use once_cell::sync::Lazy; use crate::admin::AdminFunction; +use crate::aggrs::aggr_wrapper::StateMergeHelper; use crate::aggrs::approximate::ApproximateFunction; use crate::aggrs::count_hash::CountHash; use crate::aggrs::vector::VectorFunction as VectorAggrFunction; @@ -105,6 +106,10 @@ impl FunctionRegistry { .cloned() .collect() } + + pub fn is_aggr_func_exist(&self, name: &str) -> bool { + self.aggregate_functions.read().unwrap().contains_key(name) + } } pub static FUNCTION_REGISTRY: Lazy> = Lazy::new(|| { @@ -148,6 +153,9 @@ pub static FUNCTION_REGISTRY: Lazy> = Lazy::new(|| { // CountHash function CountHash::register(&function_registry); + // state function of supported aggregate functions + StateMergeHelper::register(&function_registry); + Arc::new(function_registry) }); diff --git a/src/query/src/dist_plan/analyzer.rs b/src/query/src/dist_plan/analyzer.rs index 4bf74275b1..d59ba9ac76 100644 --- a/src/query/src/dist_plan/analyzer.rs +++ b/src/query/src/dist_plan/analyzer.rs @@ -369,6 +369,9 @@ impl PlanRewriter { .collect::>() .join("\n") ); + if let Some(new_child_plan) = &transformer_actions.new_child_plan { + debug!("PlanRewriter: new child plan: {}", new_child_plan); + } if let Some(last_stage) = transformer_actions.extra_parent_plans.last() { // update the column requirements from the last stage // notice current plan's parent plan is where we need to apply the column requirements @@ -501,12 +504,12 @@ impl PlanRewriter { } fn expand(&mut self, mut on_node: LogicalPlan) -> DfResult { + // store schema before expand, new child plan might have a different schema, so not using it + let schema = on_node.schema().clone(); if let Some(new_child_plan) = self.new_child_plan.take() { // if there is a new child plan, use it as the new root on_node = new_child_plan; } - // store schema before expand - let schema = on_node.schema().clone(); let mut rewriter = EnforceDistRequirementRewriter::new( std::mem::take(&mut self.column_requirements), self.level, @@ -514,7 +517,7 @@ impl PlanRewriter { debug!("PlanRewriter: enforce column requirements for node: {on_node} with rewriter: {rewriter:?}"); on_node = on_node.rewrite(&mut rewriter)?.data; debug!( - "PlanRewriter: after enforced column requirements for node: {on_node} with rewriter: {rewriter:?}" + "PlanRewriter: after enforced column requirements with rewriter: {rewriter:?} for node:\n{on_node}" ); // add merge scan as the new root @@ -702,7 +705,10 @@ impl TreeNodeRewriter for PlanRewriter { // TODO(ruihang): avoid this clone if self.should_expand(&parent) { // TODO(ruihang): does this work for nodes with multiple children?; - debug!("PlanRewriter: should expand child:\n {node}\n Of Parent: {parent}"); + debug!( + "PlanRewriter: should expand child:\n {node}\n Of Parent: {}", + parent.display() + ); let node = self.expand(node); debug!( "PlanRewriter: expanded plan: {}", diff --git a/src/query/src/dist_plan/analyzer/test.rs b/src/query/src/dist_plan/analyzer/test.rs index 7e1a1e0334..db850cbfc8 100644 --- a/src/query/src/dist_plan/analyzer/test.rs +++ b/src/query/src/dist_plan/analyzer/test.rs @@ -456,10 +456,9 @@ fn expand_proj_step_aggr() { let expected = [ "Projection: min(t.number)", - " Projection: min(min(t.number)) AS min(t.number)", - " Aggregate: groupBy=[[]], aggr=[[min(min(t.number))]]", - " MergeScan [is_placeholder=false, remote_input=[", - "Aggregate: groupBy=[[]], aggr=[[min(t.number)]]", + " Aggregate: groupBy=[[]], aggr=[[__min_merge(__min_state(t.number)) AS min(t.number)]]", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[]], aggr=[[__min_state(t.number)]]", " Projection: t.number", // This Projection shouldn't add new column requirements " TableScan: t", "]]", @@ -502,10 +501,9 @@ fn expand_proj_alias_fake_part_col_aggr() { let expected = [ "Projection: pk1, pk2, min(t.number)", - " Projection: pk1, pk2, min(min(t.number)) AS min(t.number)", - " Aggregate: groupBy=[[pk1, pk2]], aggr=[[min(min(t.number))]]", - " MergeScan [is_placeholder=false, remote_input=[", - "Aggregate: groupBy=[[pk1, pk2]], aggr=[[min(t.number)]]", + " Aggregate: groupBy=[[pk1, pk2]], aggr=[[__min_merge(__min_state(t.number)) AS min(t.number)]]", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[pk1, pk2]], aggr=[[__min_state(t.number)]]", " Projection: t.number, pk1 AS pk2, pk3 AS pk1", " Projection: t.number, t.pk3 AS pk1, t.pk2 AS pk3", " TableScan: t", @@ -583,10 +581,9 @@ fn expand_part_col_aggr_step_aggr() { let expected = [ "Projection: min(max(t.number))", - " Projection: min(min(max(t.number))) AS min(max(t.number))", - " Aggregate: groupBy=[[]], aggr=[[min(min(max(t.number)))]]", - " MergeScan [is_placeholder=false, remote_input=[", - "Aggregate: groupBy=[[]], aggr=[[min(max(t.number))]]", + " Aggregate: groupBy=[[]], aggr=[[__min_merge(__min_state(max(t.number))) AS min(max(t.number))]]", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[]], aggr=[[__min_state(max(t.number))]]", " Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[max(t.number)]]", " TableScan: t", "]]", @@ -618,10 +615,9 @@ fn expand_step_aggr_step_aggr() { let expected = [ "Aggregate: groupBy=[[]], aggr=[[min(max(t.number))]]", " Projection: max(t.number)", - " Projection: max(max(t.number)) AS max(t.number)", - " Aggregate: groupBy=[[]], aggr=[[max(max(t.number))]]", - " MergeScan [is_placeholder=false, remote_input=[", - "Aggregate: groupBy=[[]], aggr=[[max(t.number)]]", + " Aggregate: groupBy=[[]], aggr=[[__max_merge(__max_state(t.number)) AS max(t.number)]]", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[]], aggr=[[__max_state(t.number)]]", " TableScan: t", "]]", ] @@ -695,10 +691,9 @@ fn expand_step_aggr_proj() { let expected = [ "Projection: min(t.number)", " Projection: t.pk1, min(t.number)", - " Projection: t.pk1, min(min(t.number)) AS min(t.number)", - " Aggregate: groupBy=[[t.pk1]], aggr=[[min(min(t.number))]]", - " MergeScan [is_placeholder=false, remote_input=[", - "Aggregate: groupBy=[[t.pk1]], aggr=[[min(t.number)]]", + " Aggregate: groupBy=[[t.pk1]], aggr=[[__min_merge(__min_state(t.number)) AS min(t.number)]]", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[t.pk1]], aggr=[[__min_state(t.number)]]", " TableScan: t", "]]", ] @@ -1109,10 +1104,42 @@ fn expand_step_aggr_limit() { let expected = [ "Limit: skip=0, fetch=10", " Projection: t.pk1, min(t.number)", - " Projection: t.pk1, min(min(t.number)) AS min(t.number)", - " Aggregate: groupBy=[[t.pk1]], aggr=[[min(min(t.number))]]", - " MergeScan [is_placeholder=false, remote_input=[", - "Aggregate: groupBy=[[t.pk1]], aggr=[[min(t.number)]]", + " Aggregate: groupBy=[[t.pk1]], aggr=[[__min_merge(__min_state(t.number)) AS min(t.number)]]", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[t.pk1]], aggr=[[__min_state(t.number)]]", + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} + +/// Test how avg get expanded +#[test] +fn expand_step_aggr_avg_limit() { + // use logging for better debugging + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "numbers".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .aggregate(vec![col("pk1")], vec![avg(col("number"))]) + .unwrap() + .limit(0, Some(10)) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + let expected = [ + "Limit: skip=0, fetch=10", + " Projection: t.pk1, avg(t.number)", + " Aggregate: groupBy=[[t.pk1]], aggr=[[__avg_merge(__avg_state(t.number)) AS avg(t.number)]]", + " MergeScan [is_placeholder=false, remote_input=[", + "Aggregate: groupBy=[[t.pk1]], aggr=[[__avg_state(CAST(t.number AS Float64))]]", " TableScan: t", "]]", ] diff --git a/src/query/src/dist_plan/commutativity.rs b/src/query/src/dist_plan/commutativity.rs index e8ece184fc..9327ba4792 100644 --- a/src/query/src/dist_plan/commutativity.rs +++ b/src/query/src/dist_plan/commutativity.rs @@ -15,14 +15,10 @@ use std::collections::HashSet; use std::sync::Arc; -use common_function::aggrs::approximate::hll::{HllState, HLL_MERGE_NAME, HLL_NAME}; -use common_function::aggrs::approximate::uddsketch::{ - UddSketchState, UDDSKETCH_MERGE_NAME, UDDSKETCH_STATE_NAME, -}; +use common_function::aggrs::aggr_wrapper::{aggr_state_func_name, StateMergeHelper}; +use common_function::function_registry::FUNCTION_REGISTRY; use common_telemetry::debug; -use datafusion::functions_aggregate::sum::sum_udaf; -use datafusion_common::Column; -use datafusion_expr::{Expr, LogicalPlan, Projection, UserDefinedLogicalNode}; +use datafusion_expr::{Expr, LogicalPlan, UserDefinedLogicalNode}; use promql::extension_plan::{ EmptyMetric, InstantManipulate, RangeManipulate, SeriesDivide, SeriesNormalize, }; @@ -31,6 +27,11 @@ use crate::dist_plan::analyzer::AliasMapping; use crate::dist_plan::merge_sort::{merge_sort_transformer, MergeSortLogicalPlan}; use crate::dist_plan::MergeScanLogicalPlan; +pub struct StepTransformAction { + extra_parent_plans: Vec, + new_child_plan: Option, +} + /// generate the upper aggregation plan that will execute on the frontend. /// Basically a logical plan resembling the following: /// Projection: @@ -42,111 +43,32 @@ use crate::dist_plan::MergeScanLogicalPlan; /// of the upper aggregation plan. pub fn step_aggr_to_upper_aggr( aggr_plan: &LogicalPlan, -) -> datafusion_common::Result<[LogicalPlan; 2]> { +) -> datafusion_common::Result { let LogicalPlan::Aggregate(input_aggr) = aggr_plan else { return Err(datafusion_common::DataFusionError::Plan( "step_aggr_to_upper_aggr only accepts Aggregate plan".to_string(), )); }; if !is_all_aggr_exprs_steppable(&input_aggr.aggr_expr) { - return Err(datafusion_common::DataFusionError::NotImplemented( - "Some aggregate expressions are not steppable".to_string(), - )); + return Err(datafusion_common::DataFusionError::NotImplemented(format!( + "Some aggregate expressions are not steppable in [{}]", + input_aggr + .aggr_expr + .iter() + .map(|e| e.to_string()) + .collect::>() + .join(", ") + ))); } - let mut upper_aggr_expr = vec![]; - for aggr_expr in &input_aggr.aggr_expr { - let Some(aggr_func) = get_aggr_func(aggr_expr) else { - return Err(datafusion_common::DataFusionError::NotImplemented( - "Aggregate function not found".to_string(), - )); - }; - let col_name = aggr_expr.qualified_name(); - let input_column = Expr::Column(datafusion_common::Column::new(col_name.0, col_name.1)); - let upper_func = match aggr_func.func.name() { - "sum" | "min" | "max" | "last_value" | "first_value" => { - // aggr_calc(aggr_merge(input_column))) as col_name - let mut new_aggr_func = aggr_func.clone(); - new_aggr_func.args = vec![input_column.clone()]; - new_aggr_func - } - "count" => { - // sum(input_column) as col_name - let mut new_aggr_func = aggr_func.clone(); - new_aggr_func.func = sum_udaf(); - new_aggr_func.args = vec![input_column.clone()]; - new_aggr_func - } - UDDSKETCH_STATE_NAME | UDDSKETCH_MERGE_NAME => { - // udd_merge(bucket_size, error_rate input_column) as col_name - let mut new_aggr_func = aggr_func.clone(); - new_aggr_func.func = Arc::new(UddSketchState::merge_udf_impl()); - new_aggr_func.args[2] = input_column.clone(); - new_aggr_func - } - HLL_NAME | HLL_MERGE_NAME => { - // hll_merge(input_column) as col_name - let mut new_aggr_func = aggr_func.clone(); - new_aggr_func.func = Arc::new(HllState::merge_udf_impl()); - new_aggr_func.args = vec![input_column.clone()]; - new_aggr_func - } - _ => { - return Err(datafusion_common::DataFusionError::NotImplemented(format!( - "Aggregate function {} is not supported for Step aggregation", - aggr_func.func.name() - ))) - } - }; - // deal with nested alias case - let mut new_aggr_expr = aggr_expr.clone(); - { - let new_aggr_func = get_aggr_func_mut(&mut new_aggr_expr).unwrap(); - *new_aggr_func = upper_func; - } + let step_aggr_plan = StateMergeHelper::split_aggr_node(input_aggr.clone())?; - upper_aggr_expr.push(new_aggr_expr); - } - let mut new_aggr = input_aggr.clone(); - // use lower aggregate plan as input, this will be replace by merge scan plan later - new_aggr.input = Arc::new(LogicalPlan::Aggregate(input_aggr.clone())); - - new_aggr.aggr_expr = upper_aggr_expr; - - // group by expr also need to be all ref by column to avoid duplicated computing - let mut new_group_expr = new_aggr.group_expr.clone(); - for expr in &mut new_group_expr { - if let Expr::Column(_) = expr { - // already a column, no need to change - continue; - } - let col_name = expr.qualified_name(); - let input_column = Expr::Column(datafusion_common::Column::new(col_name.0, col_name.1)); - *expr = input_column; - } - new_aggr.group_expr = new_group_expr.clone(); - - let mut new_projection_exprs = new_group_expr; - // the upper aggr expr need to be aliased to the input aggr expr's name, - // so that the parent plan can recognize it. - for (lower_aggr_expr, upper_aggr_expr) in - input_aggr.aggr_expr.iter().zip(new_aggr.aggr_expr.iter()) - { - let lower_col_name = lower_aggr_expr.qualified_name(); - let (table, col_name) = upper_aggr_expr.qualified_name(); - let aggr_out_column = Column::new(table, col_name); - let aliased_output_aggr_expr = - Expr::Column(aggr_out_column).alias_qualified(lower_col_name.0, lower_col_name.1); - new_projection_exprs.push(aliased_output_aggr_expr); - } - let upper_aggr_plan = LogicalPlan::Aggregate(new_aggr); - let upper_aggr_plan = upper_aggr_plan.recompute_schema()?; - // create a projection on top of the new aggregate plan - let new_projection = - Projection::try_new(new_projection_exprs, Arc::new(upper_aggr_plan.clone()))?; - let projection = LogicalPlan::Projection(new_projection); - // return the new logical plan - Ok([projection, upper_aggr_plan]) + // TODO(discord9): remove duplication + let ret = StepTransformAction { + extra_parent_plans: vec![step_aggr_plan.upper_merge.clone()], + new_child_plan: Some(step_aggr_plan.lower_state.clone()), + }; + Ok(ret) } /// Check if the given aggregate expression is steppable. @@ -154,25 +76,15 @@ pub fn step_aggr_to_upper_aggr( /// i.e. on datanode first call `state(input)` then /// on frontend call `calc(merge(state))` to get the final result. pub fn is_all_aggr_exprs_steppable(aggr_exprs: &[Expr]) -> bool { - let step_action = HashSet::from([ - "sum", - "count", - "min", - "max", - "first_value", - "last_value", - UDDSKETCH_STATE_NAME, - UDDSKETCH_MERGE_NAME, - HLL_NAME, - HLL_MERGE_NAME, - ]); aggr_exprs.iter().all(|expr| { if let Some(aggr_func) = get_aggr_func(expr) { if aggr_func.distinct { // Distinct aggregate functions are not steppable(yet). return false; } - step_action.contains(aggr_func.func.name()) + + // whether the corresponding state function exists in the registry + FUNCTION_REGISTRY.is_aggr_func_exist(&aggr_state_func_name(aggr_func.func.name())) } else { false } @@ -191,18 +103,6 @@ pub fn get_aggr_func(expr: &Expr) -> Option<&datafusion_expr::expr::AggregateFun } } -pub fn get_aggr_func_mut(expr: &mut Expr) -> Option<&mut datafusion_expr::expr::AggregateFunction> { - let mut expr_ref = expr; - while let Expr::Alias(alias) = expr_ref { - expr_ref = &mut alias.expr; - } - if let Expr::AggregateFunction(aggr_func) = expr_ref { - Some(aggr_func) - } else { - None - } -} - #[allow(dead_code)] pub enum Commutativity { Commutative, @@ -247,8 +147,8 @@ impl Categorizer { debug!("Before Step optimize: {plan}"); let ret = step_aggr_to_upper_aggr(plan); ret.ok().map(|s| TransformerAction { - extra_parent_plans: s.to_vec(), - new_child_plan: None, + extra_parent_plans: s.extra_parent_plans, + new_child_plan: s.new_child_plan, }) })), }; diff --git a/tests/cases/distributed/explain/step_aggr.result b/tests/cases/distributed/explain/step_aggr.result index df9d1891a3..3cefc4692e 100644 --- a/tests/cases/distributed/explain/step_aggr.result +++ b/tests/cases/distributed/explain/step_aggr.result @@ -53,16 +53,16 @@ FROM +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| Projection: sum(count(integers.i)) AS count(integers.i), sum(sum(integers.i)) AS sum(integers.i), uddsketch_calc(Float64(0.5), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))) AS uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))) AS hll_count(hll(integers.i))_| -|_|_Aggregate: groupBy=[[]], aggr=[[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]]_| +| logical_plan_| Projection: count(integers.i), sum(integers.i), uddsketch_calc(Float64(0.5), uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll(integers.i))_| +|_|_Aggregate: groupBy=[[]], aggr=[[__count_merge(__count_state(integers.i)) AS count(integers.i), __sum_merge(__sum_state(integers.i)) AS sum(integers.i), __uddsketch_state_merge(__uddsketch_state_state(Int64(128),Float64(0.01),integers.i)) AS uddsketch_state(Int64(128),Float64(0.01),integers.i), __hll_merge(__hll_state(integers.i)) AS hll(integers.i)]] | |_|_MergeScan [is_placeholder=false, remote_input=[_| -|_| Aggregate: groupBy=[[]], aggr=[[count(integers.i), sum(integers.i), uddsketch_state(Int64(128), Float64(0.01), CAST(integers.i AS Float64)), hll(CAST(integers.i AS Utf8))]]_| +|_| Aggregate: groupBy=[[]], aggr=[[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128), Float64(0.01), CAST(integers.i AS Float64)), __hll_state(CAST(integers.i AS Utf8))]]_| |_|_TableScan: integers_| |_| ]]_| -| physical_plan | ProjectionExec: expr=[sum(count(integers.i))@0 as count(integers.i), sum(sum(integers.i))@1 as sum(integers.i), uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))@2) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))@3) as hll_count(hll(integers.i))] | -|_|_AggregateExec: mode=Final, gby=[], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]_| +| physical_plan | ProjectionExec: expr=[count(integers.i)@0 as count(integers.i), sum(integers.i)@1 as sum(integers.i), uddsketch_calc(0.5, uddsketch_state(Int64(128),Float64(0.01),integers.i)@2) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll(integers.i)@3) as hll_count(hll(integers.i))]_| +|_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)]_| |_|_CoalescePartitionsExec_| -|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]_| +|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)]_| |_|_MergeScanExec: REDACTED |_|_| +-+-+ @@ -88,25 +88,101 @@ FROM +-+-+-+ | stage | node | plan_| +-+-+-+ -| 0_| 0_|_ProjectionExec: expr=[sum(count(integers.i))@0 as count(integers.i), sum(sum(integers.i))@1 as sum(integers.i), uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))@2) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))@3) as hll_count(hll(integers.i))] REDACTED -|_|_|_AggregateExec: mode=Final, gby=[], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))] REDACTED +| 0_| 0_|_ProjectionExec: expr=[count(integers.i)@0 as count(integers.i), sum(integers.i)@1 as sum(integers.i), uddsketch_calc(0.5, uddsketch_state(Int64(128),Float64(0.01),integers.i)@2) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll(integers.i)@3) as hll_count(hll(integers.i))] REDACTED +|_|_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED |_|_|_MergeScanExec: REDACTED |_|_|_| -| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED +| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED +| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED +| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED +|_|_|_| +|_|_| Total rows: 1_| ++-+-+-+ + +SELECT + avg(i) +FROM + integers; + ++-----------------+ +| avg(integers.i) | ++-----------------+ +| 3.0 | ++-----------------+ + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +EXPLAIN SELECT + avg(i) +FROM + integers; + ++-+-+ +| plan_type_| plan_| ++-+-+ +| logical_plan_| Aggregate: groupBy=[[]], aggr=[[__avg_merge(__avg_state(integers.i)) AS avg(integers.i)]]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[]], aggr=[[__avg_state(CAST(integers.i AS Float64))]]_| +|_|_TableScan: integers_| +|_| ]]_| +| physical_plan | AggregateExec: mode=Final, gby=[], aggr=[avg(integers.i)]_| +|_|_CoalescePartitionsExec_| +|_|_AggregateExec: mode=Partial, gby=[], aggr=[avg(integers.i)]_| +|_|_MergeScanExec: REDACTED +|_|_| ++-+-+ + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- might write to different partitions +-- SQLNESS REPLACE "partition_count":\{(.*?)\} "partition_count":REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +EXPLAIN ANALYZE SELECT + avg(i) +FROM + integers; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[avg(integers.i)] REDACTED +|_|_|_CoalescePartitionsExec REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[avg(integers.i)] REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[__avg_state(integers.i)] REDACTED +|_|_|_CoalescePartitionsExec REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__avg_state(integers.i)] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED +|_|_|_| +| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[__avg_state(integers.i)] REDACTED +|_|_|_CoalescePartitionsExec REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__avg_state(integers.i)] REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED +|_|_|_| +| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[__avg_state(integers.i)] REDACTED +|_|_|_CoalescePartitionsExec REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__avg_state(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| |_|_| Total rows: 1_| @@ -157,19 +233,19 @@ ORDER BY | plan_type_| plan_| +-+-+ | logical_plan_| Sort: integers.ts ASC NULLS LAST_| -|_|_Projection: integers.ts, sum(count(integers.i)) AS count(integers.i), sum(sum(integers.i)) AS sum(integers.i), uddsketch_calc(Float64(0.5), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))) AS uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))) AS hll_count(hll(integers.i))_| -|_|_Aggregate: groupBy=[[integers.ts]], aggr=[[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]]_| +|_|_Projection: integers.ts, count(integers.i), sum(integers.i), uddsketch_calc(Float64(0.5), uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll(integers.i))_| +|_|_Aggregate: groupBy=[[integers.ts]], aggr=[[__count_merge(__count_state(integers.i)) AS count(integers.i), __sum_merge(__sum_state(integers.i)) AS sum(integers.i), __uddsketch_state_merge(__uddsketch_state_state(Int64(128),Float64(0.01),integers.i)) AS uddsketch_state(Int64(128),Float64(0.01),integers.i), __hll_merge(__hll_state(integers.i)) AS hll(integers.i)]] | |_|_MergeScan [is_placeholder=false, remote_input=[_| -|_| Aggregate: groupBy=[[integers.ts]], aggr=[[count(integers.i), sum(integers.i), uddsketch_state(Int64(128), Float64(0.01), CAST(integers.i AS Float64)), hll(CAST(integers.i AS Utf8))]]_| +|_| Aggregate: groupBy=[[integers.ts]], aggr=[[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128), Float64(0.01), CAST(integers.i AS Float64)), __hll_state(CAST(integers.i AS Utf8))]]_| |_|_TableScan: integers_| |_| ]]_| | physical_plan | SortPreservingMergeExec: [ts@0 ASC NULLS LAST]_| |_|_SortExec: expr=[ts@0 ASC NULLS LAST], preserve_partitioning=[true]_| -|_|_ProjectionExec: expr=[ts@0 as ts, sum(count(integers.i))@1 as count(integers.i), sum(sum(integers.i))@2 as sum(integers.i), uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))@3) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))@4) as hll_count(hll(integers.i))] | -|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]_| +|_|_ProjectionExec: expr=[ts@0 as ts, count(integers.i)@1 as count(integers.i), sum(integers.i)@2 as sum(integers.i), uddsketch_calc(0.5, uddsketch_state(Int64(128),Float64(0.01),integers.i)@3) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll(integers.i)@4) as hll_count(hll(integers.i))]_| +|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)]_| |_|_CoalesceBatchesExec: target_batch_size=8192_| |_|_RepartitionExec: partitioning=REDACTED -|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]_| +|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)]_| |_|_MergeScanExec: REDACTED |_|_| +-+-+ @@ -202,29 +278,29 @@ ORDER BY +-+-+-+ | 0_| 0_|_SortPreservingMergeExec: [ts@0 ASC NULLS LAST] REDACTED |_|_|_SortExec: expr=[ts@0 ASC NULLS LAST], preserve_partitioning=[true] REDACTED -|_|_|_ProjectionExec: expr=[ts@0 as ts, sum(count(integers.i))@1 as count(integers.i), sum(sum(integers.i))@2 as sum(integers.i), uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))@3) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))@4) as hll_count(hll(integers.i))] REDACTED -|_|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))] REDACTED +|_|_|_ProjectionExec: expr=[ts@0 as ts, count(integers.i)@1 as count(integers.i), sum(integers.i)@2 as sum(integers.i), uddsketch_calc(0.5, uddsketch_state(Int64(128),Float64(0.01),integers.i)@3) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll(integers.i)@4) as hll_count(hll(integers.i))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED |_|_|_MergeScanExec: REDACTED |_|_|_| -| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED +| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED +| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED +| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| |_|_| Total rows: 4_| diff --git a/tests/cases/distributed/explain/step_aggr.sql b/tests/cases/distributed/explain/step_aggr.sql index faad0923e1..c3b10f358e 100644 --- a/tests/cases/distributed/explain/step_aggr.sql +++ b/tests/cases/distributed/explain/step_aggr.sql @@ -58,6 +58,35 @@ SELECT FROM integers; +SELECT + avg(i) +FROM + integers; + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +EXPLAIN SELECT + avg(i) +FROM + integers; + +-- SQLNESS REPLACE (metrics.*) REDACTED +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +-- might write to different partitions +-- SQLNESS REPLACE "partition_count":\{(.*?)\} "partition_count":REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +EXPLAIN ANALYZE SELECT + avg(i) +FROM + integers; + SELECT ts, count(i), diff --git a/tests/cases/distributed/explain/step_aggr_advance.result b/tests/cases/distributed/explain/step_aggr_advance.result index 9bc3684eee..2122493ba2 100644 --- a/tests/cases/distributed/explain/step_aggr_advance.result +++ b/tests/cases/distributed/explain/step_aggr_advance.result @@ -758,20 +758,20 @@ GROUP BY a, b; -+---------------+------------------------------------------------------------------------------------------------------------------------+ -| plan_type | plan | -+---------------+------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | Projection: min(min(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value) | -| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[min(min(aggr_optimize_not.greptime_value))]] | -| | MergeScan [is_placeholder=false, remote_input=[ | -| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[min(aggr_optimize_not.greptime_value)]] | -| | TableScan: aggr_optimize_not | -| | ]] | -| physical_plan | ProjectionExec: expr=[min(min(aggr_optimize_not.greptime_value))@2 as min(aggr_optimize_not.greptime_value)] | -| | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(min(aggr_optimize_not.greptime_value))] | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Projection: min(aggr_optimize_not.greptime_value) | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[__min_merge(__min_state(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value)]] | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[__min_state(aggr_optimize_not.greptime_value)]] | +| | TableScan: aggr_optimize_not | +| | ]] | +| physical_plan | ProjectionExec: expr=[min(aggr_optimize_not.greptime_value)@2 as min(aggr_optimize_not.greptime_value)] | +| | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] | | | MergeScanExec: REDACTED -| | | -+---------------+------------------------------------------------------------------------------------------------------------------------+ +| | | ++---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -- SQLNESS REPLACE (metrics.*) REDACTED -- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED @@ -792,20 +792,20 @@ GROUP BY +-+-+-+ | stage | node | plan_| +-+-+-+ -| 0_| 0_|_ProjectionExec: expr=[min(min(aggr_optimize_not.greptime_value))@2 as min(aggr_optimize_not.greptime_value)] REDACTED -|_|_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(min(aggr_optimize_not.greptime_value))] REDACTED +| 0_| 0_|_ProjectionExec: expr=[min(aggr_optimize_not.greptime_value)@2 as min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED |_|_|_MergeScanExec: REDACTED |_|_|_| -| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED |_|_|_| -| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED |_|_|_| |_|_| Total rows: 0_| @@ -824,20 +824,20 @@ GROUP BY a, b; -+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| plan_type | plan | -+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | Projection: min(min(aggr_optimize_not.greptime_value)) + max(max(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value) | -| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[min(min(aggr_optimize_not.greptime_value)), max(max(aggr_optimize_not.greptime_value))]] | -| | MergeScan [is_placeholder=false, remote_input=[ | -| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)]] | -| | TableScan: aggr_optimize_not | -| | ]] | -| physical_plan | ProjectionExec: expr=[min(min(aggr_optimize_not.greptime_value))@2 + max(max(aggr_optimize_not.greptime_value))@3 as min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value)] | -| | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(min(aggr_optimize_not.greptime_value)), max(max(aggr_optimize_not.greptime_value))] | ++---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Projection: min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value) | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[__min_merge(__min_state(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value), __max_merge(__max_state(aggr_optimize_not.greptime_value)) AS max(aggr_optimize_not.greptime_value)]] | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[__min_state(aggr_optimize_not.greptime_value), __max_state(aggr_optimize_not.greptime_value)]] | +| | TableScan: aggr_optimize_not | +| | ]] | +| physical_plan | ProjectionExec: expr=[min(aggr_optimize_not.greptime_value)@2 + max(aggr_optimize_not.greptime_value)@3 as min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value)] | +| | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] | | | MergeScanExec: REDACTED -| | | -+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ +| | | ++---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -- SQLNESS REPLACE (metrics.*) REDACTED -- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED @@ -858,20 +858,20 @@ GROUP BY +-+-+-+ | stage | node | plan_| +-+-+-+ -| 0_| 0_|_ProjectionExec: expr=[min(min(aggr_optimize_not.greptime_value))@2 + max(max(aggr_optimize_not.greptime_value))@3 as min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value)] REDACTED -|_|_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(min(aggr_optimize_not.greptime_value)), max(max(aggr_optimize_not.greptime_value))] REDACTED +| 0_| 0_|_ProjectionExec: expr=[min(aggr_optimize_not.greptime_value)@2 + max(aggr_optimize_not.greptime_value)@3 as min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] REDACTED |_|_|_MergeScanExec: REDACTED |_|_|_| -| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] REDACTED +| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value), __max_state(aggr_optimize_not.greptime_value)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value), __max_state(aggr_optimize_not.greptime_value)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED |_|_|_| -| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] REDACTED +| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value), __max_state(aggr_optimize_not.greptime_value)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value), __max_state(aggr_optimize_not.greptime_value)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED |_|_|_| |_|_| Total rows: 0_| @@ -900,21 +900,19 @@ FROM GROUP BY a; -+---------------+------------------------------------------------------------------------------------------------------------------------+ -| plan_type | plan | -+---------------+------------------------------------------------------------------------------------------------------------------------+ -| logical_plan | Projection: aggr_optimize_not.a, min(min(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value) | -| | Aggregate: groupBy=[[aggr_optimize_not.a]], aggr=[[min(min(aggr_optimize_not.greptime_value))]] | -| | MergeScan [is_placeholder=false, remote_input=[ | -| | Aggregate: groupBy=[[aggr_optimize_not.a]], aggr=[[min(aggr_optimize_not.greptime_value)]] | -| | Projection: aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.greptime_value | -| | TableScan: aggr_optimize_not | -| | ]] | -| physical_plan | ProjectionExec: expr=[a@0 as a, min(min(aggr_optimize_not.greptime_value))@1 as min(aggr_optimize_not.greptime_value)] | -| | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a], aggr=[min(min(aggr_optimize_not.greptime_value))] | -| | MergeScanExec: REDACTED -| | | -+---------------+------------------------------------------------------------------------------------------------------------------------+ ++---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------+ +| plan_type | plan | ++---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------+ +| logical_plan | Aggregate: groupBy=[[aggr_optimize_not.a]], aggr=[[__min_merge(__min_state(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value)]] | +| | MergeScan [is_placeholder=false, remote_input=[ | +| | Aggregate: groupBy=[[aggr_optimize_not.a]], aggr=[[__min_state(aggr_optimize_not.greptime_value)]] | +| | Projection: aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.greptime_value | +| | TableScan: aggr_optimize_not | +| | ]] | +| physical_plan | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] | +| | MergeScanExec: REDACTED +| | | ++---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------+ -- SQLNESS REPLACE (metrics.*) REDACTED -- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED @@ -945,20 +943,19 @@ GROUP BY +-+-+-+ | stage | node | plan_| +-+-+-+ -| 0_| 0_|_ProjectionExec: expr=[a@0 as a, min(min(aggr_optimize_not.greptime_value))@1 as min(aggr_optimize_not.greptime_value)] REDACTED -|_|_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a], aggr=[min(min(aggr_optimize_not.greptime_value))] REDACTED +| 0_| 0_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED |_|_|_MergeScanExec: REDACTED |_|_|_| -| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED |_|_|_| -| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED |_|_|_| |_|_| Total rows: 0_| diff --git a/tests/cases/distributed/explain/step_aggr_basic.result b/tests/cases/distributed/explain/step_aggr_basic.result index 973aa657fe..239645ee24 100644 --- a/tests/cases/distributed/explain/step_aggr_basic.result +++ b/tests/cases/distributed/explain/step_aggr_basic.result @@ -48,16 +48,14 @@ FROM +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| Projection: sum(count(integers.i)) AS count(integers.i)_| -|_|_Aggregate: groupBy=[[]], aggr=[[sum(count(integers.i))]]_| +| logical_plan_| Aggregate: groupBy=[[]], aggr=[[__count_merge(__count_state(integers.i)) AS count(integers.i)]]_| |_|_MergeScan [is_placeholder=false, remote_input=[_| -|_| Aggregate: groupBy=[[]], aggr=[[count(integers.i)]]_| +|_| Aggregate: groupBy=[[]], aggr=[[__count_state(integers.i)]]_| |_|_TableScan: integers_| |_| ]]_| -| physical_plan | ProjectionExec: expr=[sum(count(integers.i))@0 as count(integers.i)]_| -|_|_AggregateExec: mode=Final, gby=[], aggr=[sum(count(integers.i))]_| +| physical_plan | AggregateExec: mode=Final, gby=[], aggr=[count(integers.i)]_| |_|_CoalescePartitionsExec_| -|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(count(integers.i))]_| +|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i)]_| |_|_MergeScanExec: REDACTED |_|_| +-+-+ @@ -80,25 +78,24 @@ FROM +-+-+-+ | stage | node | plan_| +-+-+-+ -| 0_| 0_|_ProjectionExec: expr=[sum(count(integers.i))@0 as count(integers.i)] REDACTED -|_|_|_AggregateExec: mode=Final, gby=[], aggr=[sum(count(integers.i))] REDACTED +| 0_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(count(integers.i))] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i)] REDACTED |_|_|_MergeScanExec: REDACTED |_|_|_| -| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i)] REDACTED +| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(integers.i)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i)] REDACTED +| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(integers.i)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i)] REDACTED +| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(integers.i)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| |_|_| Total rows: 1_| @@ -145,19 +142,17 @@ ORDER BY | plan_type_| plan_| +-+-+ | logical_plan_| Sort: integers.ts ASC NULLS LAST, count(integers.i) ASC NULLS LAST_| -|_|_Projection: integers.ts, sum(count(integers.i)) AS count(integers.i)_| -|_|_Aggregate: groupBy=[[integers.ts]], aggr=[[sum(count(integers.i))]]_| +|_|_Aggregate: groupBy=[[integers.ts]], aggr=[[__count_merge(__count_state(integers.i)) AS count(integers.i)]] | |_|_MergeScan [is_placeholder=false, remote_input=[_| -|_| Aggregate: groupBy=[[integers.ts]], aggr=[[count(integers.i)]]_| +|_| Aggregate: groupBy=[[integers.ts]], aggr=[[__count_state(integers.i)]]_| |_|_TableScan: integers_| |_| ]]_| | physical_plan | SortPreservingMergeExec: [ts@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST]_| |_|_SortExec: expr=[ts@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true]_| -|_|_ProjectionExec: expr=[ts@0 as ts, sum(count(integers.i))@1 as count(integers.i)]_| -|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[sum(count(integers.i))]_| +|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i)]_| |_|_CoalesceBatchesExec: target_batch_size=8192_| |_|_RepartitionExec: partitioning=REDACTED -|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[sum(count(integers.i))]_| +|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[count(integers.i)]_| |_|_MergeScanExec: REDACTED |_|_| +-+-+ @@ -188,29 +183,28 @@ ORDER BY +-+-+-+ | 0_| 0_|_SortPreservingMergeExec: [ts@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST] REDACTED |_|_|_SortExec: expr=[ts@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED -|_|_|_ProjectionExec: expr=[ts@0 as ts, sum(count(integers.i))@1 as count(integers.i)] REDACTED -|_|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[sum(count(integers.i))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[sum(count(integers.i))] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[count(integers.i)] REDACTED |_|_|_MergeScanExec: REDACTED |_|_|_| -| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i)] REDACTED +| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[__count_state(integers.i)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[count(integers.i)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[__count_state(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i)] REDACTED +| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[__count_state(integers.i)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[count(integers.i)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[__count_state(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i)] REDACTED +| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[__count_state(integers.i)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[count(integers.i)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[__count_state(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| |_|_| Total rows: 4_| @@ -256,20 +250,14 @@ ORDER BY +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| Sort: time_window ASC NULLS LAST, count(integers.i) ASC NULLS LAST_| -|_|_Projection: date_bin(Utf8("1 hour"),integers.ts) AS time_window, sum(count(integers.i)) AS count(integers.i)_| -|_|_Aggregate: groupBy=[[date_bin(Utf8("1 hour"),integers.ts)]], aggr=[[sum(count(integers.i))]]_| +| logical_plan_| MergeSort: time_window ASC NULLS LAST, count(integers.i) ASC NULLS LAST_| |_|_MergeScan [is_placeholder=false, remote_input=[_| -|_| Aggregate: groupBy=[[date_bin(CAST(Utf8("1 hour") AS Interval(MonthDayNano)), integers.ts)]], aggr=[[count(integers.i)]]_| +|_| Sort: time_window ASC NULLS LAST, count(integers.i) ASC NULLS LAST_| +|_|_Projection: date_bin(Utf8("1 hour"),integers.ts) AS time_window, count(integers.i)_| +|_|_Aggregate: groupBy=[[date_bin(CAST(Utf8("1 hour") AS Interval(MonthDayNano)), integers.ts)]], aggr=[[count(integers.i)]] | |_|_TableScan: integers_| |_| ]]_| | physical_plan | SortPreservingMergeExec: [time_window@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST]_| -|_|_SortExec: expr=[time_window@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true]_| -|_|_ProjectionExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 as time_window, sum(count(integers.i))@1 as count(integers.i)]_| -|_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[sum(count(integers.i))] | -|_|_CoalesceBatchesExec: target_batch_size=8192_| -|_|_RepartitionExec: partitioning=REDACTED -|_|_AggregateExec: mode=Partial, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[sum(count(integers.i))]_| |_|_MergeScanExec: REDACTED |_|_| +-+-+ @@ -299,27 +287,30 @@ ORDER BY | stage | node | plan_| +-+-+-+ | 0_| 0_|_SortPreservingMergeExec: [time_window@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST] REDACTED -|_|_|_SortExec: expr=[time_window@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED -|_|_|_ProjectionExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 as time_window, sum(count(integers.i))@1 as count(integers.i)] REDACTED -|_|_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[sum(count(integers.i))] REDACTED -|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED -|_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[sum(count(integers.i))] REDACTED |_|_|_MergeScanExec: REDACTED |_|_|_| -| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED +| 1_| 0_|_ProjectionExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 as time_window, count(integers.i)@1 as count(integers.i)] REDACTED +|_|_|_SortPreservingMergeExec: [date_bin(Utf8("1 hour"),integers.ts)@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST] REDACTED +|_|_|_SortExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED |_|_|_AggregateExec: mode=Partial, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 3600000000000 }, ts@1) as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED +| 1_| 1_|_ProjectionExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 as time_window, count(integers.i)@1 as count(integers.i)] REDACTED +|_|_|_SortPreservingMergeExec: [date_bin(Utf8("1 hour"),integers.ts)@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST] REDACTED +|_|_|_SortExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED |_|_|_AggregateExec: mode=Partial, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 3600000000000 }, ts@1) as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED +| 1_| 2_|_ProjectionExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 as time_window, count(integers.i)@1 as count(integers.i)] REDACTED +|_|_|_SortPreservingMergeExec: [date_bin(Utf8("1 hour"),integers.ts)@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST] REDACTED +|_|_|_SortExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED |_|_|_AggregateExec: mode=Partial, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 3600000000000 }, ts@1) as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED @@ -375,20 +366,14 @@ ORDER BY +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| Sort: integers.ts + Int64(1) ASC NULLS LAST, integers.i / Int64(2) ASC NULLS LAST_| -|_|_Projection: integers.ts + Int64(1), integers.i / Int64(2), sum(count(integers.i)) AS count(integers.i)_| -|_|_Aggregate: groupBy=[[integers.ts + Int64(1), integers.i / Int64(2)]], aggr=[[sum(count(integers.i))]]_| +| logical_plan_| MergeSort: integers.ts + Int64(1) ASC NULLS LAST, integers.i / Int64(2) ASC NULLS LAST_| |_|_MergeScan [is_placeholder=false, remote_input=[_| -|_| Aggregate: groupBy=[[CAST(integers.ts AS Int64) + Int64(1), integers.i / Int64(2)]], aggr=[[count(integers.i)]]_| +|_| Sort: integers.ts + Int64(1) ASC NULLS LAST, integers.i / Int64(2) ASC NULLS LAST_| +|_|_Projection: integers.ts + Int64(1), integers.i / Int64(2), count(integers.i)_| +|_|_Aggregate: groupBy=[[CAST(integers.ts AS Int64) + Int64(1), integers.i / Int64(2)]], aggr=[[count(integers.i)]] | |_|_TableScan: integers_| |_| ]]_| | physical_plan | SortPreservingMergeExec: [integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST]_| -|_|_SortExec: expr=[integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST], preserve_partitioning=[true]_| -|_|_ProjectionExec: expr=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2), sum(count(integers.i))@2 as count(integers.i)]_| -|_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[sum(count(integers.i))] | -|_|_CoalesceBatchesExec: target_batch_size=8192_| -|_|_RepartitionExec: partitioning=REDACTED -|_|_AggregateExec: mode=Partial, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[sum(count(integers.i))]_| |_|_MergeScanExec: REDACTED |_|_| +-+-+ @@ -421,27 +406,27 @@ ORDER BY | stage | node | plan_| +-+-+-+ | 0_| 0_|_SortPreservingMergeExec: [integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST] REDACTED -|_|_|_SortExec: expr=[integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED -|_|_|_ProjectionExec: expr=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2), sum(count(integers.i))@2 as count(integers.i)] REDACTED -|_|_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[sum(count(integers.i))] REDACTED -|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED -|_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[sum(count(integers.i))] REDACTED |_|_|_MergeScanExec: REDACTED |_|_|_| -| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED +| 1_| 0_|_SortPreservingMergeExec: [integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST] REDACTED +|_|_|_SortExec: expr=[integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED |_|_|_AggregateExec: mode=Partial, gby=[CAST(ts@1 AS Int64) + 1 as integers.ts + Int64(1), i@0 / 2 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED +| 1_| 1_|_SortPreservingMergeExec: [integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST] REDACTED +|_|_|_SortExec: expr=[integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED |_|_|_AggregateExec: mode=Partial, gby=[CAST(ts@1 AS Int64) + 1 as integers.ts + Int64(1), i@0 / 2 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED +| 1_| 2_|_SortPreservingMergeExec: [integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST] REDACTED +|_|_|_SortExec: expr=[integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED |_|_|_AggregateExec: mode=Partial, gby=[CAST(ts@1 AS Int64) + 1 as integers.ts + Int64(1), i@0 / 2 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED @@ -507,16 +492,16 @@ FROM +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| Projection: uddsketch_calc(Float64(0.5), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state))) AS udd_result, hll_count(hll_merge(hll_merge(sink_table.hll_state))) AS hll_result_| -|_|_Aggregate: groupBy=[[]], aggr=[[uddsketch_merge(Int64(128), Float64(0.01), uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)), hll_merge(hll_merge(sink_table.hll_state))]]_| +| logical_plan_| Projection: uddsketch_calc(Float64(0.5), uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)) AS udd_result, hll_count(hll_merge(sink_table.hll_state)) AS hll_result_| +|_|_Aggregate: groupBy=[[]], aggr=[[__uddsketch_merge_merge(__uddsketch_merge_state(Int64(128),Float64(0.01),sink_table.udd_state)) AS uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), __hll_merge_merge(__hll_merge_state(sink_table.hll_state)) AS hll_merge(sink_table.hll_state)]] | |_|_MergeScan [is_placeholder=false, remote_input=[_| -|_| Aggregate: groupBy=[[]], aggr=[[uddsketch_merge(Int64(128), Float64(0.01), sink_table.udd_state), hll_merge(sink_table.hll_state)]]_| +|_| Aggregate: groupBy=[[]], aggr=[[__uddsketch_merge_state(Int64(128), Float64(0.01), sink_table.udd_state), __hll_merge_state(sink_table.hll_state)]]_| |_|_TableScan: sink_table_| |_| ]]_| -| physical_plan | ProjectionExec: expr=[uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state))@0) as udd_result, hll_count(hll_merge(hll_merge(sink_table.hll_state))@1) as hll_result] | -|_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)), hll_merge(hll_merge(sink_table.hll_state))]_| +| physical_plan | ProjectionExec: expr=[uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)@0) as udd_result, hll_count(hll_merge(sink_table.hll_state)@1) as hll_result]_| +|_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)]_| |_|_CoalescePartitionsExec_| -|_|_AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)), hll_merge(hll_merge(sink_table.hll_state))]_| +|_|_AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)]_| |_|_MergeScanExec: REDACTED |_|_| +-+-+ @@ -540,25 +525,25 @@ FROM +-+-+-+ | stage | node | plan_| +-+-+-+ -| 0_| 0_|_ProjectionExec: expr=[uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state))@0) as udd_result, hll_count(hll_merge(hll_merge(sink_table.hll_state))@1) as hll_result] REDACTED -|_|_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)), hll_merge(hll_merge(sink_table.hll_state))] REDACTED +| 0_| 0_|_ProjectionExec: expr=[uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)@0) as udd_result, hll_count(hll_merge(sink_table.hll_state)@1) as hll_result] REDACTED +|_|_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)), hll_merge(hll_merge(sink_table.hll_state))] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED |_|_|_MergeScanExec: REDACTED |_|_|_| -| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED +| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[__uddsketch_merge_state(Int64(128),Float64(0.01),sink_table.udd_state), __hll_merge_state(sink_table.hll_state)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__uddsketch_merge_state(Int64(128),Float64(0.01),sink_table.udd_state), __hll_merge_state(sink_table.hll_state)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED +| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[__uddsketch_merge_state(Int64(128),Float64(0.01),sink_table.udd_state), __hll_merge_state(sink_table.hll_state)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__uddsketch_merge_state(Int64(128),Float64(0.01),sink_table.udd_state), __hll_merge_state(sink_table.hll_state)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED +| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[__uddsketch_merge_state(Int64(128),Float64(0.01),sink_table.udd_state), __hll_merge_state(sink_table.hll_state)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__uddsketch_merge_state(Int64(128),Float64(0.01),sink_table.udd_state), __hll_merge_state(sink_table.hll_state)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| |_|_| Total rows: 1_| diff --git a/tests/cases/distributed/explain/step_aggr_massive.result b/tests/cases/distributed/explain/step_aggr_massive.result index e0ec1e22a2..92682c1e49 100644 --- a/tests/cases/distributed/explain/step_aggr_massive.result +++ b/tests/cases/distributed/explain/step_aggr_massive.result @@ -245,19 +245,13 @@ GROUP BY +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| Projection: base_table.env, base_table.service_name, base_table.city, base_table.page, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)) AS lcp_state, max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)) AS max_lcp, min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)) AS min_lcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)) AS fmp_state, max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)) AS max_fmp, min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)) AS min_fmp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)) AS fcp_state, max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)) AS max_fcp, min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)) AS min_fcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)) AS fp_state, max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)) AS max_fp, min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)) AS min_fp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)) AS tti_state, max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)) AS max_tti, min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)) AS min_tti, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)) AS fid_state, max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)) AS max_fid, min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)) AS min_fid, max(max(base_table.shard_key)) AS shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))_| -|_|_Aggregate: groupBy=[[base_table.env, base_table.service_name, base_table.city, base_table.page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))]], aggr=[[uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(base_table.shard_key))]]_| -|_|_MergeScan [is_placeholder=false, remote_input=[_| -|_| Aggregate: groupBy=[[base_table.env, base_table.service_name, base_table.city, base_table.page, arrow_cast(date_bin(CAST(Utf8("60 seconds") AS Interval(MonthDayNano)), base_table.time), Utf8("Timestamp(Second, None)"))]], aggr=[[uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END), max(base_table.shard_key)]]_| +| logical_plan_| MergeScan [is_placeholder=false, remote_input=[_| +|_| Projection: base_table.env, base_table.service_name, base_table.city, base_table.page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END) AS lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END) AS max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END) AS min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END) AS fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END) AS max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END) AS min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END) AS fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END) AS max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END) AS min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END) AS fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END) AS max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END) AS min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END) AS tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END) AS max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END) AS min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END) AS fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END) AS max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END) AS min_fid, max(base_table.shard_key) AS shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))_| +|_|_Aggregate: groupBy=[[base_table.env, base_table.service_name, base_table.city, base_table.page, arrow_cast(date_bin(CAST(Utf8("60 seconds") AS Interval(MonthDayNano)), base_table.time), Utf8("Timestamp(Second, None)"))]], aggr=[[uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END), max(base_table.shard_key)]] | |_|_Filter: (base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) OR base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) OR base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) OR base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) OR base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) OR base_table.fid > Int64(0) AND base_table.fid < Int64(3000000)) AND CAST(base_table.time AS Timestamp(Millisecond, Some("+00:00"))) >= CAST(now() AS Timestamp(Millisecond, Some("+00:00")))_| |_|_TableScan: base_table_| |_| ]]_| -| physical_plan | ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@5 as lcp_state, max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@6 as max_lcp, min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@7 as min_lcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@8 as fmp_state, max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@9 as max_fmp, min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@10 as min_fmp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@11 as fcp_state, max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@12 as max_fcp, min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@13 as min_fcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@14 as fp_state, max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@15 as max_fp, min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@16 as min_fp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@17 as tti_state, max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@18 as max_tti, min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@19 as min_tti, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@20 as fid_state, max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@21 as max_fid, min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@22 as min_fid, max(max(base_table.shard_key))@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] | -|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(base_table.shard_key))]_| -|_|_CoalesceBatchesExec: target_batch_size=8192_| -|_|_RepartitionExec: partitioning=REDACTED -|_|_AggregateExec: mode=Partial, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(base_table.shard_key))]_| -|_|_MergeScanExec: REDACTED +| physical_plan | MergeScanExec: REDACTED |_|_| +-+-+ @@ -456,152 +450,163 @@ GROUP BY +-+-+-+ | stage | node | plan_| +-+-+-+ -| 0_| 0_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@5 as lcp_state, max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@6 as max_lcp, min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@7 as min_lcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@8 as fmp_state, max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@9 as max_fmp, min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@10 as min_fmp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@11 as fcp_state, max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@12 as max_fcp, min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@13 as min_fcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@14 as fp_state, max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@15 as max_fp, min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@16 as min_fp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@17 as tti_state, max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@18 as max_tti, min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@19 as min_tti, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@20 as fid_state, max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@21 as max_fid, min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@22 as min_fid, max(max(base_table.shard_key))@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED -|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(base_table.shard_key))] REDACTED -|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED -|_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(base_table.shard_key))] REDACTED -|_|_|_MergeScanExec: REDACTED +| 0_| 0_|_MergeScanExec: REDACTED |_|_|_| -| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 0_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 1_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 2_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 3_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 3_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 4_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 4_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 5_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 5_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 6_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 6_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 7_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 7_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 8_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 8_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 9_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 9_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 10_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 10_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 11_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 11_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 12_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 12_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 13_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 13_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 14_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 14_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 15_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +| 1_| 15_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED |_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED @@ -626,10 +631,9 @@ where +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| Projection: count(*) AS count(*)_| -|_|_Aggregate: groupBy=[[]], aggr=[[sum(count(*)) AS count(*)]]_| +| logical_plan_| Aggregate: groupBy=[[]], aggr=[[__count_merge(__count_state(base_table.time)) AS count(*)]]_| |_|_MergeScan [is_placeholder=false, remote_input=[_| -|_| Aggregate: groupBy=[[]], aggr=[[count(base_table.time) AS count(*)]]_| +|_| Aggregate: groupBy=[[]], aggr=[[__count_state(base_table.time)]]_| |_|_Filter: CAST(base_table.time AS Timestamp(Millisecond, Some("+00:00"))) >= CAST(now() AS Timestamp(Millisecond, Some("+00:00")))_| |_|_TableScan: base_table_| |_| ]]_| @@ -665,84 +669,84 @@ where |_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED |_|_|_MergeScanExec: REDACTED |_|_|_| -| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 3_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 3_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 4_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 4_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 5_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 5_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 6_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 6_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 7_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 7_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 8_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 8_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 9_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 9_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 10_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 10_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 11_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 11_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 12_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 12_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 13_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 13_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 14_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 14_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 15_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED +| 1_| 15_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| |_|_| Total rows: 1_| diff --git a/tests/cases/standalone/common/aggregate/multi_regions.result b/tests/cases/standalone/common/aggregate/multi_regions.result index db66b6b6c0..af3ed5e456 100644 --- a/tests/cases/standalone/common/aggregate/multi_regions.result +++ b/tests/cases/standalone/common/aggregate/multi_regions.result @@ -60,20 +60,19 @@ select sum(val) from t; +-+-+-+ | stage | node | plan_| +-+-+-+ -| 0_| 0_|_ProjectionExec: expr=[sum(sum(t.val))@0 as sum(t.val)] REDACTED -|_|_|_AggregateExec: mode=Final, gby=[], aggr=[sum(sum(t.val))] REDACTED +| 0_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[sum(t.val)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(sum(t.val))] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(t.val)] REDACTED |_|_|_MergeScanExec: REDACTED |_|_|_| -| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[sum(t.val)] REDACTED +| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[__sum_state(t.val)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(t.val)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__sum_state(t.val)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[sum(t.val)] REDACTED +| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[__sum_state(t.val)] REDACTED |_|_|_CoalescePartitionsExec REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(t.val)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__sum_state(t.val)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| |_|_| Total rows: 1_| @@ -94,23 +93,23 @@ select sum(val) from t group by idc; +-+-+-+ | stage | node | plan_| +-+-+-+ -| 0_| 0_|_ProjectionExec: expr=[sum(sum(t.val))@1 as sum(t.val)] REDACTED -|_|_|_AggregateExec: mode=FinalPartitioned, gby=[idc@0 as idc], aggr=[sum(sum(t.val))] REDACTED +| 0_| 0_|_ProjectionExec: expr=[sum(t.val)@1 as sum(t.val)] REDACTED +|_|_|_AggregateExec: mode=FinalPartitioned, gby=[idc@0 as idc], aggr=[sum(t.val)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[idc@0 as idc], aggr=[sum(sum(t.val))] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[idc@0 as idc], aggr=[sum(t.val)] REDACTED |_|_|_MergeScanExec: REDACTED |_|_|_| -| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[idc@0 as idc], aggr=[sum(t.val)] REDACTED +| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[idc@0 as idc], aggr=[__sum_state(t.val)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[idc@1 as idc], aggr=[sum(t.val)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[idc@1 as idc], aggr=[__sum_state(t.val)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[idc@0 as idc], aggr=[sum(t.val)] REDACTED +| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[idc@0 as idc], aggr=[__sum_state(t.val)] REDACTED |_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED |_|_|_RepartitionExec: partitioning=REDACTED -|_|_|_AggregateExec: mode=Partial, gby=[idc@1 as idc], aggr=[sum(t.val)] REDACTED +|_|_|_AggregateExec: mode=Partial, gby=[idc@1 as idc], aggr=[__sum_state(t.val)] REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| |_|_| Total rows: 0_|