From 238ed003df5dca1bb65571f9ac213ef633956b93 Mon Sep 17 00:00:00 2001 From: discord9 <55937128+discord9@users.noreply.github.com> Date: Wed, 24 Sep 2025 14:57:01 +0800 Subject: [PATCH] fix: group by expr not as column in step aggr (#7008) * fix: group by expr not as column Signed-off-by: discord9 * test: dist analyzer date_bin Signed-off-by: discord9 * ???fix wip Signed-off-by: discord9 * fix: deduce using correct input fields Signed-off-by: discord9 * refactor: clearer wrapper Signed-off-by: discord9 * chore: update sqlness Signed-off-by: discord9 * chore: per review Signed-off-by: discord9 * chore: per review Signed-off-by: discord9 * chore: rm todo Signed-off-by: discord9 --------- Signed-off-by: discord9 --- src/common/function/src/aggrs/aggr_wrapper.rs | 40 ++- .../function/src/aggrs/aggr_wrapper/tests.rs | 23 ++ src/query/src/dist_plan/analyzer.rs | 37 +-- src/query/src/dist_plan/analyzer/test.rs | 41 +++ src/query/src/dist_plan/commutativity.rs | 33 ++- .../distributed/explain/step_aggr.result | 149 +++++++++- tests/cases/distributed/explain/step_aggr.sql | 62 +++- .../explain/step_aggr_advance.result | 274 ++++++++++++++++++ .../distributed/explain/step_aggr_advance.sql | 107 +++++++ .../explain/step_aggr_basic.result | 111 +++---- .../distributed/explain/step_aggr_basic.sql | 6 +- .../explain/step_aggr_massive.result | 101 +++---- 12 files changed, 830 insertions(+), 154 deletions(-) diff --git a/src/common/function/src/aggrs/aggr_wrapper.rs b/src/common/function/src/aggrs/aggr_wrapper.rs index 2d2a73ad3b..d76565b4dd 100644 --- a/src/common/function/src/aggrs/aggr_wrapper.rs +++ b/src/common/function/src/aggrs/aggr_wrapper.rs @@ -137,6 +137,15 @@ impl StateMergeHelper { let mut lower_aggr_exprs = vec![]; let mut upper_aggr_exprs = vec![]; + // group exprs for upper plan should refer to the output group expr as column from lower plan + // to avoid re-compute group exprs again. + let upper_group_exprs = aggr + .group_expr + .iter() + .map(|c| c.qualified_name()) + .map(|(r, c)| Expr::Column(Column::new(r, c))) + .collect(); + for aggr_expr in aggr.aggr_expr.iter() { let Some(aggr_func) = get_aggr_func(aggr_expr) else { return Err(datafusion_common::DataFusionError::NotImplemented(format!( @@ -198,10 +207,13 @@ impl StateMergeHelper { // update aggregate's output schema let lower_plan = lower_plan.recompute_schema()?; - let mut upper = aggr.clone(); + let upper = Aggregate::try_new( + Arc::new(lower_plan.clone()), + upper_group_exprs, + upper_aggr_exprs.clone(), + )?; let aggr_plan = LogicalPlan::Aggregate(aggr); - upper.aggr_expr = upper_aggr_exprs; - 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; let upper_plan = LogicalPlan::Aggregate(upper_check).recompute_schema()?; @@ -245,7 +257,19 @@ impl StateWrapper { &self, acc_args: &datafusion_expr::function::AccumulatorArgs, ) -> datafusion_common::Result { - self.inner.return_field(acc_args.schema.fields()) + let input_fields = acc_args + .exprs + .iter() + .map(|e| e.return_field(acc_args.schema)) + .collect::, _>>()?; + self.inner.return_field(&input_fields).inspect_err(|e| { + common_telemetry::error!( + "StateWrapper: {:#?}\nacc_args:{:?}\nerror:{:?}", + &self, + &acc_args, + e + ); + }) } } @@ -402,7 +426,7 @@ pub struct MergeWrapper { merge_signature: Signature, /// The original physical expression of the aggregate function, can't store the original aggregate function directly, as PhysicalExpr didn't implement Any original_phy_expr: Arc, - original_input_types: Vec, + return_type: DataType, } impl MergeWrapper { pub fn new( @@ -413,13 +437,14 @@ impl MergeWrapper { let name = aggr_merge_func_name(inner.name()); // the input type is actually struct type, which is the state fields of the original aggregate function. let merge_signature = Signature::user_defined(datafusion_expr::Volatility::Immutable); + let return_type = inner.return_type(&original_input_types)?; Ok(Self { inner, name, merge_signature, original_phy_expr, - original_input_types, + return_type, }) } @@ -471,8 +496,7 @@ impl AggregateUDFImpl for MergeWrapper { /// so return fixed return type instead of using `arg_types` to determine the return type. fn return_type(&self, _arg_types: &[DataType]) -> datafusion_common::Result { // The return type is the same as the original aggregate function's return type. - let ret_type = self.inner.return_type(&self.original_input_types)?; - Ok(ret_type) + Ok(self.return_type.clone()) } fn signature(&self) -> &Signature { &self.merge_signature diff --git a/src/common/function/src/aggrs/aggr_wrapper/tests.rs b/src/common/function/src/aggrs/aggr_wrapper/tests.rs index a262703b28..edc830dea5 100644 --- a/src/common/function/src/aggrs/aggr_wrapper/tests.rs +++ b/src/common/function/src/aggrs/aggr_wrapper/tests.rs @@ -23,6 +23,7 @@ use datafusion::catalog::{Session, TableProvider}; use datafusion::datasource::DefaultTableSource; use datafusion::execution::{RecordBatchStream, SendableRecordBatchStream, TaskContext}; use datafusion::functions_aggregate::average::avg_udaf; +use datafusion::functions_aggregate::count::count_udaf; use datafusion::functions_aggregate::sum::sum_udaf; use datafusion::optimizer::AnalyzerRule; use datafusion::optimizer::analyzer::type_coercion::TypeCoercion; @@ -555,6 +556,7 @@ async fn test_udaf_correct_eval_result() { input_schema: SchemaRef, input: Vec, expected_output: Option, + // extra check function on the final array result expected_fn: Option, distinct: bool, filter: Option>, @@ -585,6 +587,27 @@ async fn test_udaf_correct_eval_result() { order_by: vec![], null_treatment: None, }, + TestCase { + func: count_udaf(), + input_schema: Arc::new(arrow_schema::Schema::new(vec![Field::new( + "str_val", + DataType::Utf8, + true, + )])), + args: vec![Expr::Column(Column::new_unqualified("str_val"))], + input: vec![Arc::new(StringArray::from(vec![ + Some("hello"), + Some("world"), + None, + Some("what"), + ]))], + expected_output: Some(ScalarValue::Int64(Some(3))), + expected_fn: None, + distinct: false, + filter: None, + order_by: vec![], + null_treatment: None, + }, TestCase { func: avg_udaf(), input_schema: Arc::new(arrow_schema::Schema::new(vec![Field::new( diff --git a/src/query/src/dist_plan/analyzer.rs b/src/query/src/dist_plan/analyzer.rs index 50dfd6c56e..c7d73e70ae 100644 --- a/src/query/src/dist_plan/analyzer.rs +++ b/src/query/src/dist_plan/analyzer.rs @@ -325,7 +325,7 @@ impl PlanRewriter { } /// Return true if should stop and expand. The input plan is the parent node of current node - fn should_expand(&mut self, plan: &LogicalPlan) -> bool { + fn should_expand(&mut self, plan: &LogicalPlan) -> DfResult { debug!( "Check should_expand at level: {} with Stack:\n{}, ", self.level, @@ -335,20 +335,21 @@ impl PlanRewriter { .collect::>() .join("\n"), ); - if DFLogicalSubstraitConvertor - .encode(plan, DefaultSerializer) - .is_err() - { - return true; + if let Err(e) = DFLogicalSubstraitConvertor.encode(plan, DefaultSerializer) { + debug!( + "PlanRewriter: plan cannot be converted to substrait with error={e:?}, expanding now: {plan}" + ); + return Ok(true); } if self.expand_on_next_call { self.expand_on_next_call = false; - return true; + debug!("PlanRewriter: expand_on_next_call is true, expanding now"); + return Ok(true); } if self.expand_on_next_part_cond_trans_commutative { - let comm = Categorizer::check_plan(plan, self.partition_cols.clone()); + let comm = Categorizer::check_plan(plan, self.partition_cols.clone())?; match comm { Commutativity::PartialCommutative => { // a small difference is that for partial commutative, we still need to @@ -364,13 +365,16 @@ impl PlanRewriter { // again a new node that can be push down, we should just // do push down now and avoid further expansion self.expand_on_next_part_cond_trans_commutative = false; - return true; + debug!( + "PlanRewriter: meet a new conditional/transformed commutative plan, expanding now: {plan}" + ); + return Ok(true); } _ => (), } } - match Categorizer::check_plan(plan, self.partition_cols.clone()) { + match Categorizer::check_plan(plan, self.partition_cols.clone())? { Commutativity::Commutative => {} Commutativity::PartialCommutative => { if let Some(plan) = partial_commutative_transformer(plan) { @@ -391,9 +395,8 @@ impl PlanRewriter { } } Commutativity::TransformedCommutative { transformer } => { - if let Some(transformer) = transformer - && let Some(transformer_actions) = transformer(plan) - { + if let Some(transformer) = transformer { + let transformer_actions = transformer(plan)?; debug!( "PlanRewriter: transformed plan: {}\n from {plan}", transformer_actions @@ -424,11 +427,12 @@ impl PlanRewriter { Commutativity::NonCommutative | Commutativity::Unimplemented | Commutativity::Unsupported => { - return true; + debug!("PlanRewriter: meet a non-commutative plan, expanding now: {plan}"); + return Ok(true); } } - false + Ok(false) } /// Update the column requirements for the current plan, plan_level is the level of the plan @@ -838,8 +842,7 @@ impl TreeNodeRewriter for PlanRewriter { let parent = parent.clone(); - // TODO(ruihang): avoid this clone - if self.should_expand(&parent) { + 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: {}", diff --git a/src/query/src/dist_plan/analyzer/test.rs b/src/query/src/dist_plan/analyzer/test.rs index 18195ed0e0..722e888f14 100644 --- a/src/query/src/dist_plan/analyzer/test.rs +++ b/src/query/src/dist_plan/analyzer/test.rs @@ -15,6 +15,7 @@ use std::pin::Pin; use std::sync::Arc; +use arrow::datatypes::IntervalDayTime; use common_catalog::consts::{DEFAULT_CATALOG_NAME, DEFAULT_SCHEMA_NAME}; use common_error::ext::BoxedError; use common_recordbatch::adapter::RecordBatchMetrics; @@ -25,7 +26,9 @@ use datafusion::datasource::DefaultTableSource; use datafusion::functions_aggregate::expr_fn::avg; use datafusion::functions_aggregate::min_max::{max, min}; use datafusion_common::JoinType; +use datafusion_expr::expr::ScalarFunction; use datafusion_expr::{Expr, LogicalPlanBuilder, col, lit}; +use datafusion_functions::datetime::date_bin; use datafusion_sql::TableReference; use datatypes::data_type::ConcreteDataType; use datatypes::schema::{ColumnSchema, SchemaBuilder, SchemaRef}; @@ -1453,3 +1456,41 @@ fn transform_sort_subquery_alias() { .join("\n"); assert_eq!(expected, result.to_string()); } + +#[test] +fn date_bin_ts_group_by() { + init_default_ut_logging(); + let test_table = TestTable::table_with_name(0, "t".to_string()); + let table_source = Arc::new(DefaultTableSource::new(Arc::new( + DfTableProviderAdapter::new(test_table), + ))); + let date_bin_call = Expr::ScalarFunction(ScalarFunction::new_udf( + date_bin(), + vec![ + lit(datafusion_common::ScalarValue::IntervalDayTime(Some( + IntervalDayTime::new(0, 60 * 1000), // 1 minute in millis + ))), + col("ts"), + ], + )); + let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![]) + .unwrap() + .aggregate(vec![date_bin_call], vec![min(col("number"))]) + .unwrap() + .build() + .unwrap(); + + let config = ConfigOptions::default(); + let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap(); + + let expected = [ + r#"Projection: date_bin(IntervalDayTime("IntervalDayTime { days: 0, milliseconds: 60000 }"),t.ts), min(t.number)"#, + r#" Aggregate: groupBy=[[date_bin(IntervalDayTime("IntervalDayTime { days: 0, milliseconds: 60000 }"),t.ts)]], aggr=[[__min_merge(__min_state(t.number)) AS min(t.number)]]"#, + " MergeScan [is_placeholder=false, remote_input=[", + r#"Aggregate: groupBy=[[date_bin(IntervalDayTime("IntervalDayTime { days: 0, milliseconds: 60000 }"), t.ts)]], aggr=[[__min_state(t.number)]]"#, + " TableScan: t", + "]]", + ] + .join("\n"); + assert_eq!(expected, result.to_string()); +} diff --git a/src/query/src/dist_plan/commutativity.rs b/src/query/src/dist_plan/commutativity.rs index 9abbdcd2dd..2fbeaf004f 100644 --- a/src/query/src/dist_plan/commutativity.rs +++ b/src/query/src/dist_plan/commutativity.rs @@ -18,6 +18,7 @@ use std::sync::Arc; use common_function::aggrs::aggr_wrapper::{StateMergeHelper, aggr_state_func_name}; use common_function::function_registry::FUNCTION_REGISTRY; use common_telemetry::debug; +use datafusion::error::Result as DfResult; use datafusion_expr::{Expr, LogicalPlan, UserDefinedLogicalNode}; use promql::extension_plan::{ EmptyMetric, InstantManipulate, RangeManipulate, SeriesDivide, SeriesNormalize, @@ -121,15 +122,18 @@ pub enum Commutativity { pub struct Categorizer {} impl Categorizer { - pub fn check_plan(plan: &LogicalPlan, partition_cols: Option) -> Commutativity { + pub fn check_plan( + plan: &LogicalPlan, + partition_cols: Option, + ) -> DfResult { let partition_cols = partition_cols.unwrap_or_default(); - match plan { + let comm = match plan { LogicalPlan::Projection(proj) => { for expr in &proj.expr { let commutativity = Self::check_expr(expr); if !matches!(commutativity, Commutativity::Commutative) { - return commutativity; + return Ok(commutativity); } } Commutativity::Commutative @@ -142,24 +146,27 @@ impl Categorizer { let matches_partition = Self::check_partition(&aggr.group_expr, &partition_cols); if !matches_partition && is_all_steppable { debug!("Plan is steppable: {plan}"); - return Commutativity::TransformedCommutative { + return Ok(Commutativity::TransformedCommutative { transformer: Some(Arc::new(|plan: &LogicalPlan| { debug!("Before Step optimize: {plan}"); let ret = step_aggr_to_upper_aggr(plan); - ret.ok().map(|s| TransformerAction { + ret.inspect_err(|err| { + common_telemetry::error!("Failed to step aggregate plan: {err:?}"); + }) + .map(|s| TransformerAction { extra_parent_plans: s.extra_parent_plans, new_child_plan: s.new_child_plan, }) })), - }; + }); } if !matches_partition { - return Commutativity::NonCommutative; + return Ok(Commutativity::NonCommutative); } for expr in &aggr.aggr_expr { let commutativity = Self::check_expr(expr); if !matches!(commutativity, Commutativity::Commutative) { - return commutativity; + return Ok(commutativity); } } // all group by expressions are partition columns can push down, unless @@ -170,7 +177,7 @@ impl Categorizer { } LogicalPlan::Sort(_) => { if partition_cols.is_empty() { - return Commutativity::Commutative; + return Ok(Commutativity::Commutative); } // sort plan needs to consider column priority @@ -219,7 +226,9 @@ impl Categorizer { LogicalPlan::Ddl(_) => Commutativity::Unsupported, LogicalPlan::Copy(_) => Commutativity::Unsupported, LogicalPlan::RecursiveQuery(_) => Commutativity::Unsupported, - } + }; + + Ok(comm) } pub fn check_extension_plan( @@ -334,7 +343,7 @@ impl Categorizer { pub type Transformer = Arc Option>; /// Returns transformer action that need to be applied -pub type StageTransformer = Arc Option>; +pub type StageTransformer = Arc DfResult>; /// The Action that a transformer should take on the plan. pub struct TransformerAction { @@ -369,7 +378,7 @@ mod test { fetch: None, }); assert!(matches!( - Categorizer::check_plan(&plan, Some(Default::default())), + Categorizer::check_plan(&plan, Some(Default::default())).unwrap(), Commutativity::Commutative )); } diff --git a/tests/cases/distributed/explain/step_aggr.result b/tests/cases/distributed/explain/step_aggr.result index 89d61e62c3..611aadf736 100644 --- a/tests/cases/distributed/explain/step_aggr.result +++ b/tests/cases/distributed/explain/step_aggr.result @@ -14,13 +14,17 @@ Affected Rows: 0 INSERT INTO integers (host, i, ts) VALUES + ('220-A', 2, '2023-01-01 00:00:00'), + ('220-B', 3, '2023-01-01 00:00:00'), ('550-A', 1, '2023-01-01 00:00:00'), ('550-B', 5, '2023-01-01 00:00:00'), ('550-A', 2, '2023-01-01 01:00:00'), ('550-W', 3, '2023-01-01 02:00:00'), - ('550-W', 4, '2023-01-01 03:00:00'); + ('550-Z', 4, '2023-01-01 02:00:00'), + ('550-W', 5, '2023-01-01 03:00:00'), + ('550-Z', 6, '2023-01-01 03:00:00'); -Affected Rows: 5 +Affected Rows: 9 SELECT count(i), @@ -33,7 +37,7 @@ FROM +-------------------+-----------------+-----------------------------------------------------------------------------------+----------------------------+ | 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)) | +-------------------+-----------------+-----------------------------------------------------------------------------------+----------------------------+ -| 5 | 15 | 2.9742334234767016 | 5 | +| 9 | 31 | 2.9742334234767016 | 6 | +-------------------+-----------------+-----------------------------------------------------------------------------------+----------------------------+ -- SQLNESS REPLACE (-+) - @@ -122,11 +126,11 @@ SELECT FROM integers; -+-----------------+ -| avg(integers.i) | -+-----------------+ -| 3.0 | -+-----------------+ ++--------------------+ +| avg(integers.i) | ++--------------------+ +| 3.4444444444444446 | ++--------------------+ -- SQLNESS REPLACE (-+) - -- SQLNESS REPLACE (\s\s+) _ @@ -214,10 +218,10 @@ ORDER BY +---------------------+-------------------+-----------------+-----------------------------------------------------------------------------------+----------------------------+ | 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)) | +---------------------+-------------------+-----------------+-----------------------------------------------------------------------------------+----------------------------+ -| 2023-01-01T00:00:00 | 2 | 6 | 5.002829575110705 | 2 | +| 2023-01-01T00:00:00 | 4 | 11 | 2.9742334234767016 | 4 | | 2023-01-01T01:00:00 | 1 | 2 | 1.9936617014173446 | 1 | -| 2023-01-01T02:00:00 | 1 | 3 | 2.9742334234767016 | 1 | -| 2023-01-01T03:00:00 | 1 | 4 | 4.014835333028587 | 1 | +| 2023-01-01T02:00:00 | 2 | 7 | 4.014835333028587 | 2 | +| 2023-01-01T03:00:00 | 2 | 11 | 5.98951037117262 | 2 | +---------------------+-------------------+-----------------+-----------------------------------------------------------------------------------+----------------------------+ -- SQLNESS REPLACE (-+) - @@ -321,6 +325,129 @@ ORDER BY |_|_| Total rows: 4_| +-+-+-+ +SELECT + date_bin('2s'::INTERVAL, ts) as time_window, + count(i), + sum(i), + uddsketch_calc(0.5, uddsketch_state(128, 0.01, i)), + hll_count(hll(i)) +FROM + integers +GROUP BY + time_window +ORDER BY + time_window; + ++---------------------+-------------------+-----------------+-----------------------------------------------------------------------------------+----------------------------+ +| time_window | 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)) | ++---------------------+-------------------+-----------------+-----------------------------------------------------------------------------------+----------------------------+ +| 2023-01-01T00:00:00 | 4 | 11 | 2.9742334234767016 | 4 | +| 2023-01-01T01:00:00 | 1 | 2 | 1.9936617014173446 | 1 | +| 2023-01-01T02:00:00 | 2 | 7 | 4.014835333028587 | 2 | +| 2023-01-01T03:00:00 | 2 | 11 | 5.98951037117262 | 2 | ++---------------------+-------------------+-----------------+-----------------------------------------------------------------------------------+----------------------------+ + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +EXPLAIN +SELECT + date_bin('2s'::INTERVAL, ts) as time_window, + count(i), + sum(i), + uddsketch_calc(0.5, uddsketch_state(128, 0.01, i)), + hll_count(hll(i)) +FROM + integers +GROUP BY + time_window +ORDER BY + time_window; + ++-+-+ +| plan_type_| plan_| ++-+-+ +| logical_plan_| Sort: time_window ASC NULLS LAST_| +|_|_Projection: date_bin(Utf8("2 seconds"),integers.ts) AS time_window, 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=[[date_bin(Utf8("2 seconds"),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=[[date_bin(CAST(Utf8("2 seconds") AS Interval(MonthDayNano)), 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: [time_window@0 ASC NULLS LAST]_| +|_|_SortExec: expr=[time_window@0 ASC NULLS LAST], preserve_partitioning=[true]_| +|_|_ProjectionExec: expr=[date_bin(Utf8("2 seconds"),integers.ts)@0 as time_window, 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=[date_bin(Utf8("2 seconds"),integers.ts)@0 as date_bin(Utf8("2 seconds"),integers.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=[date_bin(Utf8("2 seconds"),integers.ts)@0 as date_bin(Utf8("2 seconds"),integers.ts)], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)]_| +|_|_CooperativeExec_| +|_|_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 + date_bin('2s'::INTERVAL, ts) as time_window, + count(i), + sum(i), + uddsketch_calc(0.5, uddsketch_state(128, 0.01, i)), + hll_count(hll(i)) +FROM + integers +GROUP BY + time_window +ORDER BY + time_window; + ++-+-+-+ +| stage | node | plan_| ++-+-+-+ +| 0_| 0_|_SortPreservingMergeExec: [time_window@0 ASC NULLS LAST] REDACTED +|_|_|_SortExec: expr=[time_window@0 ASC NULLS LAST], preserve_partitioning=[true] REDACTED +|_|_|_ProjectionExec: expr=[date_bin(Utf8("2 seconds"),integers.ts)@0 as time_window, 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=[date_bin(Utf8("2 seconds"),integers.ts)@0 as date_bin(Utf8("2 seconds"),integers.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=[date_bin(Utf8("2 seconds"),integers.ts)@0 as date_bin(Utf8("2 seconds"),integers.ts)], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED +|_|_|_CooperativeExec REDACTED +|_|_|_MergeScanExec: REDACTED +|_|_|_| +| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("2 seconds"),integers.ts)@0 as date_bin(Utf8("2 seconds"),integers.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=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 2000000000 }, ts@1) as date_bin(Utf8("2 seconds"),integers.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 +|_|_|_CooperativeExec REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED +|_|_|_| +| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("2 seconds"),integers.ts)@0 as date_bin(Utf8("2 seconds"),integers.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=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 2000000000 }, ts@1) as date_bin(Utf8("2 seconds"),integers.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 +|_|_|_CooperativeExec REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED +|_|_|_| +| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("2 seconds"),integers.ts)@0 as date_bin(Utf8("2 seconds"),integers.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=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 2000000000 }, ts@1) as date_bin(Utf8("2 seconds"),integers.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 +|_|_|_CooperativeExec REDACTED +|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED +|_|_|_| +|_|_| Total rows: 4_| ++-+-+-+ + DROP TABLE integers; Affected Rows: 0 diff --git a/tests/cases/distributed/explain/step_aggr.sql b/tests/cases/distributed/explain/step_aggr.sql index c3b10f358e..cb6cb3b883 100644 --- a/tests/cases/distributed/explain/step_aggr.sql +++ b/tests/cases/distributed/explain/step_aggr.sql @@ -12,11 +12,15 @@ CREATE TABLE integers( INSERT INTO integers (host, i, ts) VALUES + ('220-A', 2, '2023-01-01 00:00:00'), + ('220-B', 3, '2023-01-01 00:00:00'), ('550-A', 1, '2023-01-01 00:00:00'), ('550-B', 5, '2023-01-01 00:00:00'), ('550-A', 2, '2023-01-01 01:00:00'), ('550-W', 3, '2023-01-01 02:00:00'), - ('550-W', 4, '2023-01-01 03:00:00'); + ('550-Z', 4, '2023-01-01 02:00:00'), + ('550-W', 5, '2023-01-01 03:00:00'), + ('550-Z', 6, '2023-01-01 03:00:00'); SELECT count(i), @@ -142,4 +146,60 @@ GROUP BY ORDER BY ts; + +SELECT + date_bin('2s'::INTERVAL, ts) as time_window, + count(i), + sum(i), + uddsketch_calc(0.5, uddsketch_state(128, 0.01, i)), + hll_count(hll(i)) +FROM + integers +GROUP BY + time_window +ORDER BY + time_window; + +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +EXPLAIN +SELECT + date_bin('2s'::INTERVAL, ts) as time_window, + count(i), + sum(i), + uddsketch_calc(0.5, uddsketch_state(128, 0.01, i)), + hll_count(hll(i)) +FROM + integers +GROUP BY + time_window +ORDER BY + time_window; + +-- 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 + date_bin('2s'::INTERVAL, ts) as time_window, + count(i), + sum(i), + uddsketch_calc(0.5, uddsketch_state(128, 0.01, i)), + hll_count(hll(i)) +FROM + integers +GROUP BY + time_window +ORDER BY + time_window; + DROP TABLE integers; diff --git a/tests/cases/distributed/explain/step_aggr_advance.result b/tests/cases/distributed/explain/step_aggr_advance.result index 316e282e24..a5f7a4aff9 100644 --- a/tests/cases/distributed/explain/step_aggr_advance.result +++ b/tests/cases/distributed/explain/step_aggr_advance.result @@ -1037,3 +1037,277 @@ drop table aggr_optimize_not; Affected Rows: 0 +-- +-- Additional test cases for step aggregation pushdown +-- +CREATE TABLE step_aggr_extended ( + pk_col_1 STRING, + pk_col_2 BIGINT, + val_col_1 BIGINT, + val_col_2 STRING, + val_col_3 BIGINT, + ts TIMESTAMP TIME INDEX, + PRIMARY KEY(pk_col_1, pk_col_2) +) PARTITION ON COLUMNS (pk_col_1) ( + pk_col_1 < 'f', + pk_col_1 >= 'f' +); + +Affected Rows: 0 + +INSERT INTO step_aggr_extended VALUES + ('a', 1, 100, 'v1', 10, 1672531200000), + ('a', 2, 200, 'v2', NULL, 1672531201000), + ('g', 1, 300, 'v1', 30, 1672531202000), + ('g', 2, 400, 'v2', 40, 1672531203000), + ('a', 3, 100, 'v3', 10, 1672531204000), + ('g', 3, 300, 'v3', 30, 1672531205000), + ('h', 4, 500, NULL, 50, 1672531206000); + +Affected Rows: 7 + +-- Case 12: GROUP BY includes a mix of partition key and non-partition key. +-- `pk_col_1` is a partition key, `pk_col_2` is not. +-- This should pushdown entire aggregation to datanodes since it's partitioned by `pk_col_1`. +-- Expected: Full pushdown of aggregation to datanodes. +SELECT pk_col_1, pk_col_2, sum(val_col_1) FROM step_aggr_extended GROUP BY pk_col_1, pk_col_2 ORDER BY pk_col_1, pk_col_2; + ++----------+----------+-----------------------------------+ +| pk_col_1 | pk_col_2 | sum(step_aggr_extended.val_col_1) | ++----------+----------+-----------------------------------+ +| a | 1 | 100 | +| a | 2 | 200 | +| a | 3 | 100 | +| g | 1 | 300 | +| g | 2 | 400 | +| g | 3 | 300 | +| h | 4 | 500 | ++----------+----------+-----------------------------------+ + +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN SELECT pk_col_1, pk_col_2, sum(val_col_1) FROM step_aggr_extended GROUP BY pk_col_1, pk_col_2 ORDER BY pk_col_1, pk_col_2; + ++-+-+ +| plan_type_| plan_| ++-+-+ +| logical_plan_| MergeSort: step_aggr_extended.pk_col_1 ASC NULLS LAST, step_aggr_extended.pk_col_2 ASC NULLS LAST_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Sort: step_aggr_extended.pk_col_1 ASC NULLS LAST, step_aggr_extended.pk_col_2 ASC NULLS LAST_| +|_|_Projection: step_aggr_extended.pk_col_1, step_aggr_extended.pk_col_2, sum(step_aggr_extended.val_col_1)_| +|_|_Aggregate: groupBy=[[step_aggr_extended.pk_col_1, step_aggr_extended.pk_col_2]], aggr=[[sum(step_aggr_extended.val_col_1)]] | +|_|_TableScan: step_aggr_extended_| +|_| ]]_| +| physical_plan | SortPreservingMergeExec: [pk_col_1@0 ASC NULLS LAST, pk_col_2@1 ASC NULLS LAST]_| +|_|_CooperativeExec_| +|_|_CooperativeExec_| +|_|_MergeScanExec: REDACTED +|_|_| ++-+-+ + +-- Case 13: COUNT(DISTINCT) aggregation. +-- `DISTINCT` aggregation is more complex and requires a two-phase distinct calculation in a distributed environment. Currently not supported for pushdown. +-- Expected: datanode only do table scan, actual aggregation happens on frontend. +SELECT COUNT(DISTINCT val_col_1) FROM step_aggr_extended; + ++----------------------------------------------+ +| count(DISTINCT step_aggr_extended.val_col_1) | ++----------------------------------------------+ +| 5 | ++----------------------------------------------+ + +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN SELECT COUNT(DISTINCT val_col_1) FROM step_aggr_extended; + ++-+-+ +| plan_type_| plan_| ++-+-+ +| logical_plan_| Projection: count(alias1) AS count(DISTINCT step_aggr_extended.val_col_1)_| +|_|_Aggregate: groupBy=[[]], aggr=[[count(alias1)]]_| +|_|_Aggregate: groupBy=[[step_aggr_extended.val_col_1 AS alias1]], aggr=[[]]_| +|_|_Projection: step_aggr_extended.val_col_1_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| TableScan: step_aggr_extended_| +|_| ]]_| +| physical_plan | ProjectionExec: expr=[count(alias1)@0 as count(DISTINCT step_aggr_extended.val_col_1)]_| +|_|_AggregateExec: mode=Final, gby=[], aggr=[count(alias1)]_| +|_|_CoalescePartitionsExec_| +|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(alias1)]_| +|_|_AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[]_| +|_|_CoalesceBatchesExec: target_batch_size=8192_| +|_|_RepartitionExec: partitioning=REDACTED +|_|_AggregateExec: mode=Partial, gby=[val_col_1@0 as alias1], aggr=[]_| +|_|_ProjectionExec: expr=[val_col_1@2 as val_col_1]_| +|_|_CooperativeExec_| +|_|_MergeScanExec: REDACTED +|_|_| ++-+-+ + +-- Case 14: Aggregation with a HAVING clause. +-- The `HAVING` clause filters results after aggregation. +-- Expected: The `HAVING` filter should be applied on the frontend after the final aggregation is complete, not pushed down to datanodes. +SELECT pk_col_2, sum(val_col_1) FROM step_aggr_extended GROUP BY pk_col_2 HAVING sum(val_col_1) > 300 ORDER BY pk_col_2; + ++----------+-----------------------------------+ +| pk_col_2 | sum(step_aggr_extended.val_col_1) | ++----------+-----------------------------------+ +| 1 | 400 | +| 2 | 600 | +| 3 | 400 | +| 4 | 500 | ++----------+-----------------------------------+ + +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN SELECT pk_col_2, sum(val_col_1) FROM step_aggr_extended GROUP BY pk_col_2 HAVING sum(val_col_1) > 300 ORDER BY pk_col_2; + ++-+-+ +| plan_type_| plan_| ++-+-+ +| logical_plan_| Sort: step_aggr_extended.pk_col_2 ASC NULLS LAST_| +|_|_Filter: sum(step_aggr_extended.val_col_1) > Int64(300)_| +|_|_Aggregate: groupBy=[[step_aggr_extended.pk_col_2]], aggr=[[__sum_merge(__sum_state(step_aggr_extended.val_col_1)) AS sum(step_aggr_extended.val_col_1)]] | +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[step_aggr_extended.pk_col_2]], aggr=[[__sum_state(step_aggr_extended.val_col_1)]]_| +|_|_TableScan: step_aggr_extended_| +|_| ]]_| +| physical_plan | SortPreservingMergeExec: [pk_col_2@0 ASC NULLS LAST]_| +|_|_SortExec: expr=[pk_col_2@0 ASC NULLS LAST], preserve_partitioning=[true]_| +|_|_CoalesceBatchesExec: target_batch_size=8192_| +|_|_FilterExec: sum(step_aggr_extended.val_col_1)@1 > 300_| +|_|_AggregateExec: mode=FinalPartitioned, gby=[pk_col_2@0 as pk_col_2], aggr=[sum(step_aggr_extended.val_col_1)]_| +|_|_CoalesceBatchesExec: target_batch_size=8192_| +|_|_RepartitionExec: partitioning=REDACTED +|_|_AggregateExec: mode=Partial, gby=[pk_col_2@0 as pk_col_2], aggr=[sum(step_aggr_extended.val_col_1)]_| +|_|_CooperativeExec_| +|_|_MergeScanExec: REDACTED +|_|_| ++-+-+ + +-- Case 15: Aggregation on a column with NULL values. +-- `SUM` should ignore NULLs. `COUNT(val_col_2)` should count non-nulls, `COUNT(*)` should count all rows. +-- Expected: Correct aggregation results, proving NULLs are handled properly in a distributed context. +SELECT SUM(val_col_3), COUNT(val_col_2), COUNT(val_col_3), COUNT(*) FROM step_aggr_extended; + ++-----------------------------------+-------------------------------------+-------------------------------------+----------+ +| sum(step_aggr_extended.val_col_3) | count(step_aggr_extended.val_col_2) | count(step_aggr_extended.val_col_3) | count(*) | ++-----------------------------------+-------------------------------------+-------------------------------------+----------+ +| 170 | 6 | 6 | 7 | ++-----------------------------------+-------------------------------------+-------------------------------------+----------+ + +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN SELECT SUM(val_col_3), COUNT(val_col_2), COUNT(val_col_3), COUNT(*) FROM step_aggr_extended; + ++-+-+ +| plan_type_| plan_| ++-+-+ +| logical_plan_| Projection: sum(step_aggr_extended.val_col_3), count(step_aggr_extended.val_col_2), count(step_aggr_extended.val_col_3), count(Int64(1)) AS count(*)_| +|_|_Aggregate: groupBy=[[]], aggr=[[__sum_merge(__sum_state(step_aggr_extended.val_col_3)) AS sum(step_aggr_extended.val_col_3), __count_merge(__count_state(step_aggr_extended.val_col_2)) AS count(step_aggr_extended.val_col_2), __count_merge(__count_state(step_aggr_extended.val_col_3)) AS count(step_aggr_extended.val_col_3), __count_merge(__count_state(step_aggr_extended.ts)) AS count(Int64(1))]] | +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[]], aggr=[[__sum_state(step_aggr_extended.val_col_3), __count_state(step_aggr_extended.val_col_2), __count_state(step_aggr_extended.val_col_3), __count_state(step_aggr_extended.ts)]]_| +|_|_TableScan: step_aggr_extended_| +|_| ]]_| +| physical_plan | ProjectionExec: expr=[sum(step_aggr_extended.val_col_3)@0 as sum(step_aggr_extended.val_col_3), count(step_aggr_extended.val_col_2)@1 as count(step_aggr_extended.val_col_2), count(step_aggr_extended.val_col_3)@2 as count(step_aggr_extended.val_col_3), count(Int64(1))@3 as count(*)]_| +|_|_AggregateExec: mode=Final, gby=[], aggr=[sum(step_aggr_extended.val_col_3), count(step_aggr_extended.val_col_2), count(step_aggr_extended.val_col_3), count(Int64(1))]_| +|_|_CoalescePartitionsExec_| +|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(step_aggr_extended.val_col_3), count(step_aggr_extended.val_col_2), count(step_aggr_extended.val_col_3), count(Int64(1))]_| +|_|_CooperativeExec_| +|_|_MergeScanExec: REDACTED +|_|_| ++-+-+ + +-- Case 16: Aggregation on STRING columns. +-- `MIN` and `MAX` can operate on strings. +-- Expected: Correct lexicographical min/max results. +SELECT MIN(pk_col_1), MAX(val_col_2) FROM step_aggr_extended; + ++----------------------------------+-----------------------------------+ +| min(step_aggr_extended.pk_col_1) | max(step_aggr_extended.val_col_2) | ++----------------------------------+-----------------------------------+ +| a | v3 | ++----------------------------------+-----------------------------------+ + +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN SELECT MIN(pk_col_1), MAX(val_col_2) FROM step_aggr_extended; + ++-+-+ +| plan_type_| plan_| ++-+-+ +| logical_plan_| Aggregate: groupBy=[[]], aggr=[[__min_merge(__min_state(step_aggr_extended.pk_col_1)) AS min(step_aggr_extended.pk_col_1), __max_merge(__max_state(step_aggr_extended.val_col_2)) AS max(step_aggr_extended.val_col_2)]] | +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[]], aggr=[[__min_state(step_aggr_extended.pk_col_1), __max_state(step_aggr_extended.val_col_2)]]_| +|_|_TableScan: step_aggr_extended_| +|_| ]]_| +| physical_plan | AggregateExec: mode=Final, gby=[], aggr=[min(step_aggr_extended.pk_col_1), max(step_aggr_extended.val_col_2)]_| +|_|_CoalescePartitionsExec_| +|_|_AggregateExec: mode=Partial, gby=[], aggr=[min(step_aggr_extended.pk_col_1), max(step_aggr_extended.val_col_2)]_| +|_|_CooperativeExec_| +|_|_MergeScanExec: REDACTED +|_|_| ++-+-+ + +-- Case 17: Aggregation on an empty input set. +-- `WHERE` clause filters out all rows. +-- Expected: Aggregation should return correct default values (e.g., COUNT is 0, SUM is NULL). +SELECT SUM(val_col_1), COUNT(*) FROM step_aggr_extended WHERE pk_col_1 = 'non_existent'; + ++-----------------------------------+----------+ +| sum(step_aggr_extended.val_col_1) | count(*) | ++-----------------------------------+----------+ +| | 0 | ++-----------------------------------+----------+ + +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN SELECT SUM(val_col_1), COUNT(*) FROM step_aggr_extended WHERE pk_col_1 = 'non_existent'; + ++-+-+ +| plan_type_| plan_| ++-+-+ +| logical_plan_| Projection: sum(step_aggr_extended.val_col_1), count(Int64(1)) AS count(*)_| +|_|_Aggregate: groupBy=[[]], aggr=[[__sum_merge(__sum_state(step_aggr_extended.val_col_1)) AS sum(step_aggr_extended.val_col_1), __count_merge(__count_state(step_aggr_extended.ts)) AS count(Int64(1))]] | +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[]], aggr=[[__sum_state(step_aggr_extended.val_col_1), __count_state(step_aggr_extended.ts)]]_| +|_|_Filter: step_aggr_extended.pk_col_1 = Utf8("non_existent")_| +|_|_TableScan: step_aggr_extended_| +|_| ]]_| +| physical_plan | ProjectionExec: expr=[sum(step_aggr_extended.val_col_1)@0 as sum(step_aggr_extended.val_col_1), count(Int64(1))@1 as count(*)]_| +|_|_AggregateExec: mode=Final, gby=[], aggr=[sum(step_aggr_extended.val_col_1), count(Int64(1))]_| +|_|_CoalescePartitionsExec_| +|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(step_aggr_extended.val_col_1), count(Int64(1))]_| +|_|_CooperativeExec_| +|_|_MergeScanExec: REDACTED +|_|_| ++-+-+ + +DROP TABLE step_aggr_extended; + +Affected Rows: 0 + diff --git a/tests/cases/distributed/explain/step_aggr_advance.sql b/tests/cases/distributed/explain/step_aggr_advance.sql index aa8554bdb8..e9a6a3363e 100644 --- a/tests/cases/distributed/explain/step_aggr_advance.sql +++ b/tests/cases/distributed/explain/step_aggr_advance.sql @@ -305,3 +305,110 @@ GROUP BY drop table aggr_optimize_not_count; drop table aggr_optimize_not; + +-- +-- Additional test cases for step aggregation pushdown +-- +CREATE TABLE step_aggr_extended ( + pk_col_1 STRING, + pk_col_2 BIGINT, + val_col_1 BIGINT, + val_col_2 STRING, + val_col_3 BIGINT, + ts TIMESTAMP TIME INDEX, + PRIMARY KEY(pk_col_1, pk_col_2) +) PARTITION ON COLUMNS (pk_col_1) ( + pk_col_1 < 'f', + pk_col_1 >= 'f' +); + +INSERT INTO step_aggr_extended VALUES + ('a', 1, 100, 'v1', 10, 1672531200000), + ('a', 2, 200, 'v2', NULL, 1672531201000), + ('g', 1, 300, 'v1', 30, 1672531202000), + ('g', 2, 400, 'v2', 40, 1672531203000), + ('a', 3, 100, 'v3', 10, 1672531204000), + ('g', 3, 300, 'v3', 30, 1672531205000), + ('h', 4, 500, NULL, 50, 1672531206000); + + +-- Case 12: GROUP BY includes a mix of partition key and non-partition key. +-- `pk_col_1` is a partition key, `pk_col_2` is not. +-- This should pushdown entire aggregation to datanodes since it's partitioned by `pk_col_1`. +-- Expected: Full pushdown of aggregation to datanodes. +SELECT pk_col_1, pk_col_2, sum(val_col_1) FROM step_aggr_extended GROUP BY pk_col_1, pk_col_2 ORDER BY pk_col_1, pk_col_2; + +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN SELECT pk_col_1, pk_col_2, sum(val_col_1) FROM step_aggr_extended GROUP BY pk_col_1, pk_col_2 ORDER BY pk_col_1, pk_col_2; + +-- Case 13: COUNT(DISTINCT) aggregation. +-- `DISTINCT` aggregation is more complex and requires a two-phase distinct calculation in a distributed environment. Currently not supported for pushdown. +-- Expected: datanode only do table scan, actual aggregation happens on frontend. +SELECT COUNT(DISTINCT val_col_1) FROM step_aggr_extended; + +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN SELECT COUNT(DISTINCT val_col_1) FROM step_aggr_extended; + +-- Case 14: Aggregation with a HAVING clause. +-- The `HAVING` clause filters results after aggregation. +-- Expected: The `HAVING` filter should be applied on the frontend after the final aggregation is complete, not pushed down to datanodes. +SELECT pk_col_2, sum(val_col_1) FROM step_aggr_extended GROUP BY pk_col_2 HAVING sum(val_col_1) > 300 ORDER BY pk_col_2; + +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN SELECT pk_col_2, sum(val_col_1) FROM step_aggr_extended GROUP BY pk_col_2 HAVING sum(val_col_1) > 300 ORDER BY pk_col_2; + +-- Case 15: Aggregation on a column with NULL values. +-- `SUM` should ignore NULLs. `COUNT(val_col_2)` should count non-nulls, `COUNT(*)` should count all rows. +-- Expected: Correct aggregation results, proving NULLs are handled properly in a distributed context. +SELECT SUM(val_col_3), COUNT(val_col_2), COUNT(val_col_3), COUNT(*) FROM step_aggr_extended; + +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN SELECT SUM(val_col_3), COUNT(val_col_2), COUNT(val_col_3), COUNT(*) FROM step_aggr_extended; + +-- Case 16: Aggregation on STRING columns. +-- `MIN` and `MAX` can operate on strings. +-- Expected: Correct lexicographical min/max results. +SELECT MIN(pk_col_1), MAX(val_col_2) FROM step_aggr_extended; + +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN SELECT MIN(pk_col_1), MAX(val_col_2) FROM step_aggr_extended; + +-- Case 17: Aggregation on an empty input set. +-- `WHERE` clause filters out all rows. +-- Expected: Aggregation should return correct default values (e.g., COUNT is 0, SUM is NULL). +SELECT SUM(val_col_1), COUNT(*) FROM step_aggr_extended WHERE pk_col_1 = 'non_existent'; + +-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED +-- SQLNESS REPLACE (peers.*) REDACTED +-- SQLNESS REPLACE (Hash.*) REDACTED +-- SQLNESS REPLACE (-+) - +-- SQLNESS REPLACE (\s\s+) _ +-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED +EXPLAIN SELECT SUM(val_col_1), COUNT(*) FROM step_aggr_extended WHERE pk_col_1 = 'non_existent'; + +DROP TABLE step_aggr_extended; diff --git a/tests/cases/distributed/explain/step_aggr_basic.result b/tests/cases/distributed/explain/step_aggr_basic.result index 7691e02c37..975f71e703 100644 --- a/tests/cases/distributed/explain/step_aggr_basic.result +++ b/tests/cases/distributed/explain/step_aggr_basic.result @@ -14,13 +14,17 @@ Affected Rows: 0 INSERT INTO integers (host, i, ts) VALUES + ('220-A', 2, '2023-01-01 00:00:00'), + ('220-B', 3, '2023-01-01 00:00:00'), ('550-A', 1, '2023-01-01 00:00:00'), ('550-B', 5, '2023-01-01 00:00:00'), ('550-A', 2, '2023-01-01 01:00:00'), ('550-W', 3, '2023-01-01 02:00:00'), - ('550-W', 4, '2023-01-01 03:00:00'); + ('550-Z', 4, '2023-01-01 02:00:00'), + ('550-W', 5, '2023-01-01 03:00:00'), + ('550-Z', 6, '2023-01-01 03:00:00'); -Affected Rows: 5 +Affected Rows: 9 -- count SELECT @@ -31,7 +35,7 @@ FROM +-------------------+ | count(integers.i) | +-------------------+ -| 5 | +| 9 | +-------------------+ -- SQLNESS REPLACE (-+) - @@ -120,10 +124,10 @@ ORDER BY +---------------------+-------------------+ | ts | count(integers.i) | +---------------------+-------------------+ -| 2023-01-01T00:00:00 | 2 | +| 2023-01-01T00:00:00 | 4 | | 2023-01-01T01:00:00 | 1 | -| 2023-01-01T02:00:00 | 1 | -| 2023-01-01T03:00:00 | 1 | +| 2023-01-01T02:00:00 | 2 | +| 2023-01-01T03:00:00 | 2 | +---------------------+-------------------+ -- SQLNESS REPLACE (-+) - @@ -234,10 +238,10 @@ ORDER BY +---------------------+-------------------+ | time_window | count(integers.i) | +---------------------+-------------------+ -| 2023-01-01T00:00:00 | 2 | +| 2023-01-01T00:00:00 | 4 | | 2023-01-01T01:00:00 | 1 | -| 2023-01-01T02:00:00 | 1 | -| 2023-01-01T03:00:00 | 1 | +| 2023-01-01T02:00:00 | 2 | +| 2023-01-01T03:00:00 | 2 | +---------------------+-------------------+ -- SQLNESS REPLACE (-+) - @@ -260,15 +264,20 @@ ORDER BY +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| MergeSort: time_window ASC NULLS LAST, count(integers.i) ASC NULLS LAST_| -|_|_MergeScan [is_placeholder=false, remote_input=[_| -|_| Sort: time_window ASC NULLS LAST, count(integers.i) ASC NULLS LAST_| +| 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, count(integers.i)_| -|_|_Aggregate: groupBy=[[date_bin(CAST(Utf8("1 hour") AS Interval(MonthDayNano)), integers.ts)]], aggr=[[count(integers.i)]] | +|_|_Aggregate: groupBy=[[date_bin(Utf8("1 hour"),integers.ts)]], aggr=[[__count_merge(__count_state(integers.i)) AS count(integers.i)]]_| +|_|_MergeScan [is_placeholder=false, remote_input=[_| +|_| Aggregate: groupBy=[[date_bin(CAST(Utf8("1 hour") AS Interval(MonthDayNano)), integers.ts)]], aggr=[[__count_state(integers.i)]]_| |_|_TableScan: integers_| |_| ]]_| | physical_plan | SortPreservingMergeExec: [time_window@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST]_| -|_|_CooperativeExec_| +|_|_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, 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=[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=[count(integers.i)]_| |_|_CooperativeExec_| |_|_MergeScanExec: REDACTED |_|_| @@ -299,37 +308,33 @@ ORDER BY | stage | node | plan_| +-+-+-+ | 0_| 0_|_SortPreservingMergeExec: [time_window@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST] REDACTED -|_|_|_CooperativeExec 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, 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=[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=[count(integers.i)] REDACTED |_|_|_CooperativeExec REDACTED |_|_|_MergeScanExec: 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 +| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[__count_state(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 +|_|_|_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_state(integers.i)] REDACTED |_|_|_CooperativeExec REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED 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 +| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[__count_state(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 +|_|_|_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_state(integers.i)] REDACTED |_|_|_CooperativeExec REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED 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 +| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[__count_state(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 +|_|_|_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_state(integers.i)] REDACTED |_|_|_CooperativeExec REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| @@ -354,10 +359,13 @@ ORDER BY | integers.ts + Int64(1) | integers.i / Int64(2) | count(integers.i) | +------------------------+-----------------------+-------------------+ | 1672531200001 | 0 | 1 | +| 1672531200001 | 1 | 2 | | 1672531200001 | 2 | 1 | | 1672534800001 | 1 | 1 | | 1672538400001 | 1 | 1 | +| 1672538400001 | 2 | 1 | | 1672542000001 | 2 | 1 | +| 1672542000001 | 3 | 1 | +------------------------+-----------------------+-------------------+ -- SQLNESS REPLACE (-+) - @@ -383,15 +391,18 @@ ORDER BY +-+-+ | plan_type_| plan_| +-+-+ -| logical_plan_| MergeSort: integers.ts + Int64(1) ASC NULLS LAST, integers.i / Int64(2) ASC NULLS LAST_| +| logical_plan_| Sort: integers.ts + Int64(1) ASC NULLS LAST, integers.i / Int64(2) ASC NULLS LAST_| +|_|_Aggregate: groupBy=[[integers.ts + Int64(1), integers.i / Int64(2)]], aggr=[[__count_merge(__count_state(integers.i)) AS count(integers.i)]]_| |_|_MergeScan [is_placeholder=false, remote_input=[_| -|_| 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)]] | +|_| Aggregate: groupBy=[[CAST(integers.ts AS Int64) + Int64(1), integers.i / Int64(2)]], aggr=[[__count_state(integers.i)]]_| |_|_TableScan: integers_| |_| ]]_| | physical_plan | SortPreservingMergeExec: [integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST]_| -|_|_CooperativeExec_| +|_|_SortExec: expr=[integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST], preserve_partitioning=[true]_| +|_|_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)] | +|_|_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=[count(integers.i)]_| |_|_CooperativeExec_| |_|_MergeScanExec: REDACTED |_|_| @@ -425,38 +436,36 @@ 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 -|_|_|_CooperativeExec 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=[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 |_|_|_CooperativeExec REDACTED |_|_|_MergeScanExec: 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 +| 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_state(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 +|_|_|_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_state(integers.i)] REDACTED |_|_|_CooperativeExec REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_state(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 +|_|_|_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_state(integers.i)] REDACTED |_|_|_CooperativeExec REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_state(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 +|_|_|_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_state(integers.i)] REDACTED |_|_|_CooperativeExec REDACTED |_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED |_|_|_| -|_|_| Total rows: 5_| +|_|_| Total rows: 8_| +-+-+-+ -- test udd/hll_merege pushdown @@ -487,7 +496,7 @@ GROUP BY time_window, host; -Affected Rows: 5 +Affected Rows: 9 SELECT uddsketch_calc(0.5, uddsketch_merge(128, 0.01, udd_state)) as udd_result, @@ -498,7 +507,7 @@ FROM +--------------------+------------+ | udd_result | hll_result | +--------------------+------------+ -| 2.9742334234767016 | 5 | +| 2.9742334234767016 | 6 | +--------------------+------------+ -- SQLNESS REPLACE (-+) - diff --git a/tests/cases/distributed/explain/step_aggr_basic.sql b/tests/cases/distributed/explain/step_aggr_basic.sql index a2afebd0bf..9a0d226a13 100644 --- a/tests/cases/distributed/explain/step_aggr_basic.sql +++ b/tests/cases/distributed/explain/step_aggr_basic.sql @@ -12,11 +12,15 @@ CREATE TABLE integers( INSERT INTO integers (host, i, ts) VALUES + ('220-A', 2, '2023-01-01 00:00:00'), + ('220-B', 3, '2023-01-01 00:00:00'), ('550-A', 1, '2023-01-01 00:00:00'), ('550-B', 5, '2023-01-01 00:00:00'), ('550-A', 2, '2023-01-01 01:00:00'), ('550-W', 3, '2023-01-01 02:00:00'), - ('550-W', 4, '2023-01-01 03:00:00'); + ('550-Z', 4, '2023-01-01 02:00:00'), + ('550-W', 5, '2023-01-01 03:00:00'), + ('550-Z', 6, '2023-01-01 03:00:00'); -- count SELECT diff --git a/tests/cases/distributed/explain/step_aggr_massive.result b/tests/cases/distributed/explain/step_aggr_massive.result index 5afaeb3376..734f40e650 100644 --- a/tests/cases/distributed/explain/step_aggr_massive.result +++ b/tests/cases/distributed/explain/step_aggr_massive.result @@ -245,13 +245,19 @@ GROUP BY +-+-+ | plan_type_| plan_| +-+-+ -| 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)]] | +| logical_plan_| 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(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))]], aggr=[[__uddsketch_state_merge(__uddsketch_state_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 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_merge(__max_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)) AS max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_merge(__min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)) AS min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_merge(__uddsketch_state_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 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_merge(__max_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)) AS max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_merge(__min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)) AS min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_merge(__uddsketch_state_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 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_merge(__max_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)) AS max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_merge(__min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)) AS min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_merge(__uddsketch_state_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 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_merge(__max_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)) AS max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_merge(__min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)) AS min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_merge(__uddsketch_state_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 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_merge(__max_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)) AS max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_merge(__min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)) AS min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_merge(__uddsketch_state_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 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_merge(__max_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)) AS max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_merge(__min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)) AS min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_merge(__max_state(base_table.shard_key)) AS 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END), __min_state(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_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END), __min_state(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_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END), __min_state(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_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END), __min_state(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_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END), __min_state(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_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END), __max_state(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 | CooperativeExec_| +| physical_plan | 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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)"))]_| +|_|_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)]_| +|_|_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_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)]_| +|_|_CooperativeExec_| |_|_MergeScanExec: REDACTED |_|_| +-+-+ @@ -451,179 +457,168 @@ GROUP BY +-+-+-+ | stage | node | plan_| +-+-+-+ -| 0_| 0_|_CooperativeExec REDACTED +| 0_| 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 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 +|_|_|_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_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 +|_|_|_CooperativeExec REDACTED |_|_|_MergeScanExec: 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 |_|_|_CooperativeExec REDACTED |_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED 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 +| 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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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 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 +|_|_|_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_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_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __min_state(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __min_state(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __min_state(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __min_state(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __min_state(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), __uddsketch_state_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_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __min_state(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), __max_state(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