feat: register all aggregate function to auto step aggr fn (#6596)

* feat: support generic aggr push down

Signed-off-by: discord9 <discord9@163.com>

* typo

Signed-off-by: discord9 <discord9@163.com>

* fix: type ck in merge wrapper

Signed-off-by: discord9 <discord9@163.com>

* test: update sqlness

Signed-off-by: discord9 <discord9@163.com>

* feat: support all registried aggr func

Signed-off-by: discord9 <discord9@163.com>

* chore: per review

Signed-off-by: discord9 <discord9@163.com>

* chore: per review

Signed-off-by: discord9 <discord9@163.com>

---------

Signed-off-by: discord9 <discord9@163.com>
This commit is contained in:
discord9
2025-08-05 19:37:45 +08:00
committed by GitHub
parent 9871c22740
commit 875207d26c
12 changed files with 510 additions and 442 deletions

View File

@@ -26,6 +26,8 @@ use std::sync::Arc;
use arrow::array::StructArray;
use arrow_schema::Fields;
use common_telemetry::debug;
use datafusion::functions_aggregate::all_default_aggregate_functions;
use datafusion::optimizer::analyzer::type_coercion::TypeCoercion;
use datafusion::optimizer::AnalyzerRule;
use datafusion::physical_planner::create_aggregate_expr_and_maybe_filter;
@@ -39,6 +41,8 @@ use datafusion_expr::{
use datafusion_physical_expr::aggregate::AggregateFunctionExpr;
use datatypes::arrow::datatypes::{DataType, Field};
use crate::function_registry::FunctionRegistry;
/// Returns the name of the state function for the given aggregate function name.
/// The state function is used to compute the state of the aggregate function.
/// The state function's name is in the format `__<aggr_name>_state
@@ -65,9 +69,9 @@ pub struct StateMergeHelper;
#[derive(Debug, Clone)]
pub struct StepAggrPlan {
/// Upper merge plan, which is the aggregate plan that merges the states of the state function.
pub upper_merge: Arc<LogicalPlan>,
pub upper_merge: LogicalPlan,
/// Lower state plan, which is the aggregate plan that computes the state of the aggregate function.
pub lower_state: Arc<LogicalPlan>,
pub lower_state: LogicalPlan,
}
pub fn get_aggr_func(expr: &Expr) -> Option<&datafusion_expr::expr::AggregateFunction> {
@@ -83,6 +87,36 @@ pub fn get_aggr_func(expr: &Expr) -> Option<&datafusion_expr::expr::AggregateFun
}
impl StateMergeHelper {
/// Register all the `state` function of supported aggregate functions.
/// Note that can't register `merge` function here, as it needs to be created from the original aggregate function with given input types.
pub fn register(registry: &FunctionRegistry) {
let all_default = all_default_aggregate_functions();
let greptime_custom_aggr_functions = registry.aggregate_functions();
// if our custom aggregate function have the same name as the default aggregate function, we will override it.
let supported = all_default
.into_iter()
.chain(greptime_custom_aggr_functions.into_iter().map(Arc::new))
.collect::<Vec<_>>();
debug!(
"Registering state functions for supported: {:?}",
supported.iter().map(|f| f.name()).collect::<Vec<_>>()
);
let state_func = supported.into_iter().filter_map(|f| {
StateWrapper::new((*f).clone())
.inspect_err(
|e| common_telemetry::error!(e; "Failed to register state function for {:?}", f),
)
.ok()
.map(AggregateUDF::new_from_impl)
});
for func in state_func {
registry.register_aggr(func);
}
}
/// Split an aggregate plan into two aggregate plans, one for the state function and one for the merge function.
pub fn split_aggr_node(aggr_plan: Aggregate) -> datafusion_common::Result<StepAggrPlan> {
let aggr = {
@@ -166,18 +200,18 @@ impl StateMergeHelper {
let lower_plan = LogicalPlan::Aggregate(lower);
// update aggregate's output schema
let lower_plan = Arc::new(lower_plan.recompute_schema()?);
let lower_plan = lower_plan.recompute_schema()?;
let mut upper = aggr.clone();
let aggr_plan = LogicalPlan::Aggregate(aggr);
upper.aggr_expr = upper_aggr_exprs;
upper.input = lower_plan.clone();
upper.input = Arc::new(lower_plan.clone());
// upper schema's output schema should be the same as the original aggregate plan's output schema
let upper_check = upper.clone();
let upper_plan = Arc::new(LogicalPlan::Aggregate(upper_check).recompute_schema()?);
let upper_check = upper;
let upper_plan = LogicalPlan::Aggregate(upper_check).recompute_schema()?;
if *upper_plan.schema() != *aggr_plan.schema() {
return Err(datafusion_common::DataFusionError::Internal(format!(
"Upper aggregate plan's schema is not the same as the original aggregate plan's schema: \n[transformed]:{}\n[ original]{}",
"Upper aggregate plan's schema is not the same as the original aggregate plan's schema: \n[transformed]:{}\n[original]:{}",
upper_plan.schema(), aggr_plan.schema()
)));
}
@@ -407,15 +441,18 @@ impl AggregateUDFImpl for MergeWrapper {
&'a self,
acc_args: datafusion_expr::function::AccumulatorArgs<'b>,
) -> datafusion_common::Result<Box<dyn Accumulator>> {
if acc_args.schema.fields().len() != 1
|| !matches!(acc_args.schema.field(0).data_type(), DataType::Struct(_))
if acc_args.exprs.len() != 1
|| !matches!(
acc_args.exprs[0].data_type(acc_args.schema)?,
DataType::Struct(_)
)
{
return Err(datafusion_common::DataFusionError::Internal(format!(
"Expected one struct type as input, got: {:?}",
acc_args.schema
)));
}
let input_type = acc_args.schema.field(0).data_type();
let input_type = acc_args.exprs[0].data_type(acc_args.schema)?;
let DataType::Struct(fields) = input_type else {
return Err(datafusion_common::DataFusionError::Internal(format!(
"Expected a struct type for input, got: {:?}",
@@ -424,7 +461,7 @@ impl AggregateUDFImpl for MergeWrapper {
};
let inner_accum = self.original_phy_expr.create_accumulator()?;
Ok(Box::new(MergeAccum::new(inner_accum, fields)))
Ok(Box::new(MergeAccum::new(inner_accum, &fields)))
}
fn as_any(&self) -> &dyn std::any::Any {

View File

@@ -258,7 +258,7 @@ async fn test_sum_udaf() {
)
.recompute_schema()
.unwrap();
assert_eq!(res.lower_state.as_ref(), &expected_lower_plan);
assert_eq!(&res.lower_state, &expected_lower_plan);
let expected_merge_plan = LogicalPlan::Aggregate(
Aggregate::try_new(
@@ -297,7 +297,7 @@ async fn test_sum_udaf() {
)
.unwrap(),
);
assert_eq!(res.upper_merge.as_ref(), &expected_merge_plan);
assert_eq!(&res.upper_merge, &expected_merge_plan);
let phy_aggr_state_plan = DefaultPhysicalPlanner::default()
.create_physical_plan(&res.lower_state, &ctx.state())
@@ -405,7 +405,7 @@ async fn test_avg_udaf() {
let coerced_aggr_state_plan = TypeCoercion::new()
.analyze(expected_aggr_state_plan.clone(), &Default::default())
.unwrap();
assert_eq!(res.lower_state.as_ref(), &coerced_aggr_state_plan);
assert_eq!(&res.lower_state, &coerced_aggr_state_plan);
assert_eq!(
res.lower_state.schema().as_arrow(),
&arrow_schema::Schema::new(vec![Field::new(
@@ -456,7 +456,7 @@ async fn test_avg_udaf() {
)
.unwrap(),
);
assert_eq!(res.upper_merge.as_ref(), &expected_merge_plan);
assert_eq!(&res.upper_merge, &expected_merge_plan);
let phy_aggr_state_plan = DefaultPhysicalPlanner::default()
.create_physical_plan(&coerced_aggr_state_plan, &ctx.state())

View File

@@ -20,6 +20,7 @@ use datafusion_expr::AggregateUDF;
use once_cell::sync::Lazy;
use crate::admin::AdminFunction;
use crate::aggrs::aggr_wrapper::StateMergeHelper;
use crate::aggrs::approximate::ApproximateFunction;
use crate::aggrs::count_hash::CountHash;
use crate::aggrs::vector::VectorFunction as VectorAggrFunction;
@@ -105,6 +106,10 @@ impl FunctionRegistry {
.cloned()
.collect()
}
pub fn is_aggr_func_exist(&self, name: &str) -> bool {
self.aggregate_functions.read().unwrap().contains_key(name)
}
}
pub static FUNCTION_REGISTRY: Lazy<Arc<FunctionRegistry>> = Lazy::new(|| {
@@ -148,6 +153,9 @@ pub static FUNCTION_REGISTRY: Lazy<Arc<FunctionRegistry>> = Lazy::new(|| {
// CountHash function
CountHash::register(&function_registry);
// state function of supported aggregate functions
StateMergeHelper::register(&function_registry);
Arc::new(function_registry)
});

View File

@@ -369,6 +369,9 @@ impl PlanRewriter {
.collect::<Vec<_>>()
.join("\n")
);
if let Some(new_child_plan) = &transformer_actions.new_child_plan {
debug!("PlanRewriter: new child plan: {}", new_child_plan);
}
if let Some(last_stage) = transformer_actions.extra_parent_plans.last() {
// update the column requirements from the last stage
// notice current plan's parent plan is where we need to apply the column requirements
@@ -501,12 +504,12 @@ impl PlanRewriter {
}
fn expand(&mut self, mut on_node: LogicalPlan) -> DfResult<LogicalPlan> {
// store schema before expand, new child plan might have a different schema, so not using it
let schema = on_node.schema().clone();
if let Some(new_child_plan) = self.new_child_plan.take() {
// if there is a new child plan, use it as the new root
on_node = new_child_plan;
}
// store schema before expand
let schema = on_node.schema().clone();
let mut rewriter = EnforceDistRequirementRewriter::new(
std::mem::take(&mut self.column_requirements),
self.level,
@@ -514,7 +517,7 @@ impl PlanRewriter {
debug!("PlanRewriter: enforce column requirements for node: {on_node} with rewriter: {rewriter:?}");
on_node = on_node.rewrite(&mut rewriter)?.data;
debug!(
"PlanRewriter: after enforced column requirements for node: {on_node} with rewriter: {rewriter:?}"
"PlanRewriter: after enforced column requirements with rewriter: {rewriter:?} for node:\n{on_node}"
);
// add merge scan as the new root
@@ -702,7 +705,10 @@ impl TreeNodeRewriter for PlanRewriter {
// TODO(ruihang): avoid this clone
if self.should_expand(&parent) {
// TODO(ruihang): does this work for nodes with multiple children?;
debug!("PlanRewriter: should expand child:\n {node}\n Of Parent: {parent}");
debug!(
"PlanRewriter: should expand child:\n {node}\n Of Parent: {}",
parent.display()
);
let node = self.expand(node);
debug!(
"PlanRewriter: expanded plan: {}",

View File

@@ -456,10 +456,9 @@ fn expand_proj_step_aggr() {
let expected = [
"Projection: min(t.number)",
" Projection: min(min(t.number)) AS min(t.number)",
" Aggregate: groupBy=[[]], aggr=[[min(min(t.number))]]",
" MergeScan [is_placeholder=false, remote_input=[",
"Aggregate: groupBy=[[]], aggr=[[min(t.number)]]",
" Aggregate: groupBy=[[]], aggr=[[__min_merge(__min_state(t.number)) AS min(t.number)]]",
" MergeScan [is_placeholder=false, remote_input=[",
"Aggregate: groupBy=[[]], aggr=[[__min_state(t.number)]]",
" Projection: t.number", // This Projection shouldn't add new column requirements
" TableScan: t",
"]]",
@@ -502,10 +501,9 @@ fn expand_proj_alias_fake_part_col_aggr() {
let expected = [
"Projection: pk1, pk2, min(t.number)",
" Projection: pk1, pk2, min(min(t.number)) AS min(t.number)",
" Aggregate: groupBy=[[pk1, pk2]], aggr=[[min(min(t.number))]]",
" MergeScan [is_placeholder=false, remote_input=[",
"Aggregate: groupBy=[[pk1, pk2]], aggr=[[min(t.number)]]",
" Aggregate: groupBy=[[pk1, pk2]], aggr=[[__min_merge(__min_state(t.number)) AS min(t.number)]]",
" MergeScan [is_placeholder=false, remote_input=[",
"Aggregate: groupBy=[[pk1, pk2]], aggr=[[__min_state(t.number)]]",
" Projection: t.number, pk1 AS pk2, pk3 AS pk1",
" Projection: t.number, t.pk3 AS pk1, t.pk2 AS pk3",
" TableScan: t",
@@ -583,10 +581,9 @@ fn expand_part_col_aggr_step_aggr() {
let expected = [
"Projection: min(max(t.number))",
" Projection: min(min(max(t.number))) AS min(max(t.number))",
" Aggregate: groupBy=[[]], aggr=[[min(min(max(t.number)))]]",
" MergeScan [is_placeholder=false, remote_input=[",
"Aggregate: groupBy=[[]], aggr=[[min(max(t.number))]]",
" Aggregate: groupBy=[[]], aggr=[[__min_merge(__min_state(max(t.number))) AS min(max(t.number))]]",
" MergeScan [is_placeholder=false, remote_input=[",
"Aggregate: groupBy=[[]], aggr=[[__min_state(max(t.number))]]",
" Aggregate: groupBy=[[t.pk1, t.pk2]], aggr=[[max(t.number)]]",
" TableScan: t",
"]]",
@@ -618,10 +615,9 @@ fn expand_step_aggr_step_aggr() {
let expected = [
"Aggregate: groupBy=[[]], aggr=[[min(max(t.number))]]",
" Projection: max(t.number)",
" Projection: max(max(t.number)) AS max(t.number)",
" Aggregate: groupBy=[[]], aggr=[[max(max(t.number))]]",
" MergeScan [is_placeholder=false, remote_input=[",
"Aggregate: groupBy=[[]], aggr=[[max(t.number)]]",
" Aggregate: groupBy=[[]], aggr=[[__max_merge(__max_state(t.number)) AS max(t.number)]]",
" MergeScan [is_placeholder=false, remote_input=[",
"Aggregate: groupBy=[[]], aggr=[[__max_state(t.number)]]",
" TableScan: t",
"]]",
]
@@ -695,10 +691,9 @@ fn expand_step_aggr_proj() {
let expected = [
"Projection: min(t.number)",
" Projection: t.pk1, min(t.number)",
" Projection: t.pk1, min(min(t.number)) AS min(t.number)",
" Aggregate: groupBy=[[t.pk1]], aggr=[[min(min(t.number))]]",
" MergeScan [is_placeholder=false, remote_input=[",
"Aggregate: groupBy=[[t.pk1]], aggr=[[min(t.number)]]",
" Aggregate: groupBy=[[t.pk1]], aggr=[[__min_merge(__min_state(t.number)) AS min(t.number)]]",
" MergeScan [is_placeholder=false, remote_input=[",
"Aggregate: groupBy=[[t.pk1]], aggr=[[__min_state(t.number)]]",
" TableScan: t",
"]]",
]
@@ -1109,10 +1104,42 @@ fn expand_step_aggr_limit() {
let expected = [
"Limit: skip=0, fetch=10",
" Projection: t.pk1, min(t.number)",
" Projection: t.pk1, min(min(t.number)) AS min(t.number)",
" Aggregate: groupBy=[[t.pk1]], aggr=[[min(min(t.number))]]",
" MergeScan [is_placeholder=false, remote_input=[",
"Aggregate: groupBy=[[t.pk1]], aggr=[[min(t.number)]]",
" Aggregate: groupBy=[[t.pk1]], aggr=[[__min_merge(__min_state(t.number)) AS min(t.number)]]",
" MergeScan [is_placeholder=false, remote_input=[",
"Aggregate: groupBy=[[t.pk1]], aggr=[[__min_state(t.number)]]",
" TableScan: t",
"]]",
]
.join("\n");
assert_eq!(expected, result.to_string());
}
/// Test how avg get expanded
#[test]
fn expand_step_aggr_avg_limit() {
// use logging for better debugging
init_default_ut_logging();
let test_table = TestTable::table_with_name(0, "numbers".to_string());
let table_source = Arc::new(DefaultTableSource::new(Arc::new(
DfTableProviderAdapter::new(test_table),
)));
let plan = LogicalPlanBuilder::scan_with_filters("t", table_source, None, vec![])
.unwrap()
.aggregate(vec![col("pk1")], vec![avg(col("number"))])
.unwrap()
.limit(0, Some(10))
.unwrap()
.build()
.unwrap();
let config = ConfigOptions::default();
let result = DistPlannerAnalyzer {}.analyze(plan, &config).unwrap();
let expected = [
"Limit: skip=0, fetch=10",
" Projection: t.pk1, avg(t.number)",
" Aggregate: groupBy=[[t.pk1]], aggr=[[__avg_merge(__avg_state(t.number)) AS avg(t.number)]]",
" MergeScan [is_placeholder=false, remote_input=[",
"Aggregate: groupBy=[[t.pk1]], aggr=[[__avg_state(CAST(t.number AS Float64))]]",
" TableScan: t",
"]]",
]

View File

@@ -15,14 +15,10 @@
use std::collections::HashSet;
use std::sync::Arc;
use common_function::aggrs::approximate::hll::{HllState, HLL_MERGE_NAME, HLL_NAME};
use common_function::aggrs::approximate::uddsketch::{
UddSketchState, UDDSKETCH_MERGE_NAME, UDDSKETCH_STATE_NAME,
};
use common_function::aggrs::aggr_wrapper::{aggr_state_func_name, StateMergeHelper};
use common_function::function_registry::FUNCTION_REGISTRY;
use common_telemetry::debug;
use datafusion::functions_aggregate::sum::sum_udaf;
use datafusion_common::Column;
use datafusion_expr::{Expr, LogicalPlan, Projection, UserDefinedLogicalNode};
use datafusion_expr::{Expr, LogicalPlan, UserDefinedLogicalNode};
use promql::extension_plan::{
EmptyMetric, InstantManipulate, RangeManipulate, SeriesDivide, SeriesNormalize,
};
@@ -31,6 +27,11 @@ use crate::dist_plan::analyzer::AliasMapping;
use crate::dist_plan::merge_sort::{merge_sort_transformer, MergeSortLogicalPlan};
use crate::dist_plan::MergeScanLogicalPlan;
pub struct StepTransformAction {
extra_parent_plans: Vec<LogicalPlan>,
new_child_plan: Option<LogicalPlan>,
}
/// generate the upper aggregation plan that will execute on the frontend.
/// Basically a logical plan resembling the following:
/// Projection:
@@ -42,111 +43,32 @@ use crate::dist_plan::MergeScanLogicalPlan;
/// of the upper aggregation plan.
pub fn step_aggr_to_upper_aggr(
aggr_plan: &LogicalPlan,
) -> datafusion_common::Result<[LogicalPlan; 2]> {
) -> datafusion_common::Result<StepTransformAction> {
let LogicalPlan::Aggregate(input_aggr) = aggr_plan else {
return Err(datafusion_common::DataFusionError::Plan(
"step_aggr_to_upper_aggr only accepts Aggregate plan".to_string(),
));
};
if !is_all_aggr_exprs_steppable(&input_aggr.aggr_expr) {
return Err(datafusion_common::DataFusionError::NotImplemented(
"Some aggregate expressions are not steppable".to_string(),
));
return Err(datafusion_common::DataFusionError::NotImplemented(format!(
"Some aggregate expressions are not steppable in [{}]",
input_aggr
.aggr_expr
.iter()
.map(|e| e.to_string())
.collect::<Vec<_>>()
.join(", ")
)));
}
let mut upper_aggr_expr = vec![];
for aggr_expr in &input_aggr.aggr_expr {
let Some(aggr_func) = get_aggr_func(aggr_expr) else {
return Err(datafusion_common::DataFusionError::NotImplemented(
"Aggregate function not found".to_string(),
));
};
let col_name = aggr_expr.qualified_name();
let input_column = Expr::Column(datafusion_common::Column::new(col_name.0, col_name.1));
let upper_func = match aggr_func.func.name() {
"sum" | "min" | "max" | "last_value" | "first_value" => {
// aggr_calc(aggr_merge(input_column))) as col_name
let mut new_aggr_func = aggr_func.clone();
new_aggr_func.args = vec![input_column.clone()];
new_aggr_func
}
"count" => {
// sum(input_column) as col_name
let mut new_aggr_func = aggr_func.clone();
new_aggr_func.func = sum_udaf();
new_aggr_func.args = vec![input_column.clone()];
new_aggr_func
}
UDDSKETCH_STATE_NAME | UDDSKETCH_MERGE_NAME => {
// udd_merge(bucket_size, error_rate input_column) as col_name
let mut new_aggr_func = aggr_func.clone();
new_aggr_func.func = Arc::new(UddSketchState::merge_udf_impl());
new_aggr_func.args[2] = input_column.clone();
new_aggr_func
}
HLL_NAME | HLL_MERGE_NAME => {
// hll_merge(input_column) as col_name
let mut new_aggr_func = aggr_func.clone();
new_aggr_func.func = Arc::new(HllState::merge_udf_impl());
new_aggr_func.args = vec![input_column.clone()];
new_aggr_func
}
_ => {
return Err(datafusion_common::DataFusionError::NotImplemented(format!(
"Aggregate function {} is not supported for Step aggregation",
aggr_func.func.name()
)))
}
};
// deal with nested alias case
let mut new_aggr_expr = aggr_expr.clone();
{
let new_aggr_func = get_aggr_func_mut(&mut new_aggr_expr).unwrap();
*new_aggr_func = upper_func;
}
let step_aggr_plan = StateMergeHelper::split_aggr_node(input_aggr.clone())?;
upper_aggr_expr.push(new_aggr_expr);
}
let mut new_aggr = input_aggr.clone();
// use lower aggregate plan as input, this will be replace by merge scan plan later
new_aggr.input = Arc::new(LogicalPlan::Aggregate(input_aggr.clone()));
new_aggr.aggr_expr = upper_aggr_expr;
// group by expr also need to be all ref by column to avoid duplicated computing
let mut new_group_expr = new_aggr.group_expr.clone();
for expr in &mut new_group_expr {
if let Expr::Column(_) = expr {
// already a column, no need to change
continue;
}
let col_name = expr.qualified_name();
let input_column = Expr::Column(datafusion_common::Column::new(col_name.0, col_name.1));
*expr = input_column;
}
new_aggr.group_expr = new_group_expr.clone();
let mut new_projection_exprs = new_group_expr;
// the upper aggr expr need to be aliased to the input aggr expr's name,
// so that the parent plan can recognize it.
for (lower_aggr_expr, upper_aggr_expr) in
input_aggr.aggr_expr.iter().zip(new_aggr.aggr_expr.iter())
{
let lower_col_name = lower_aggr_expr.qualified_name();
let (table, col_name) = upper_aggr_expr.qualified_name();
let aggr_out_column = Column::new(table, col_name);
let aliased_output_aggr_expr =
Expr::Column(aggr_out_column).alias_qualified(lower_col_name.0, lower_col_name.1);
new_projection_exprs.push(aliased_output_aggr_expr);
}
let upper_aggr_plan = LogicalPlan::Aggregate(new_aggr);
let upper_aggr_plan = upper_aggr_plan.recompute_schema()?;
// create a projection on top of the new aggregate plan
let new_projection =
Projection::try_new(new_projection_exprs, Arc::new(upper_aggr_plan.clone()))?;
let projection = LogicalPlan::Projection(new_projection);
// return the new logical plan
Ok([projection, upper_aggr_plan])
// TODO(discord9): remove duplication
let ret = StepTransformAction {
extra_parent_plans: vec![step_aggr_plan.upper_merge.clone()],
new_child_plan: Some(step_aggr_plan.lower_state.clone()),
};
Ok(ret)
}
/// Check if the given aggregate expression is steppable.
@@ -154,25 +76,15 @@ pub fn step_aggr_to_upper_aggr(
/// i.e. on datanode first call `state(input)` then
/// on frontend call `calc(merge(state))` to get the final result.
pub fn is_all_aggr_exprs_steppable(aggr_exprs: &[Expr]) -> bool {
let step_action = HashSet::from([
"sum",
"count",
"min",
"max",
"first_value",
"last_value",
UDDSKETCH_STATE_NAME,
UDDSKETCH_MERGE_NAME,
HLL_NAME,
HLL_MERGE_NAME,
]);
aggr_exprs.iter().all(|expr| {
if let Some(aggr_func) = get_aggr_func(expr) {
if aggr_func.distinct {
// Distinct aggregate functions are not steppable(yet).
return false;
}
step_action.contains(aggr_func.func.name())
// whether the corresponding state function exists in the registry
FUNCTION_REGISTRY.is_aggr_func_exist(&aggr_state_func_name(aggr_func.func.name()))
} else {
false
}
@@ -191,18 +103,6 @@ pub fn get_aggr_func(expr: &Expr) -> Option<&datafusion_expr::expr::AggregateFun
}
}
pub fn get_aggr_func_mut(expr: &mut Expr) -> Option<&mut datafusion_expr::expr::AggregateFunction> {
let mut expr_ref = expr;
while let Expr::Alias(alias) = expr_ref {
expr_ref = &mut alias.expr;
}
if let Expr::AggregateFunction(aggr_func) = expr_ref {
Some(aggr_func)
} else {
None
}
}
#[allow(dead_code)]
pub enum Commutativity {
Commutative,
@@ -247,8 +147,8 @@ impl Categorizer {
debug!("Before Step optimize: {plan}");
let ret = step_aggr_to_upper_aggr(plan);
ret.ok().map(|s| TransformerAction {
extra_parent_plans: s.to_vec(),
new_child_plan: None,
extra_parent_plans: s.extra_parent_plans,
new_child_plan: s.new_child_plan,
})
})),
};

View File

@@ -53,16 +53,16 @@ FROM
+-+-+
| plan_type_| plan_|
+-+-+
| logical_plan_| Projection: sum(count(integers.i)) AS count(integers.i), sum(sum(integers.i)) AS sum(integers.i), uddsketch_calc(Float64(0.5), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))) AS uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))) AS hll_count(hll(integers.i))_|
|_|_Aggregate: groupBy=[[]], aggr=[[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]]_|
| logical_plan_| Projection: count(integers.i), sum(integers.i), uddsketch_calc(Float64(0.5), uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll(integers.i))_|
|_|_Aggregate: groupBy=[[]], aggr=[[__count_merge(__count_state(integers.i)) AS count(integers.i), __sum_merge(__sum_state(integers.i)) AS sum(integers.i), __uddsketch_state_merge(__uddsketch_state_state(Int64(128),Float64(0.01),integers.i)) AS uddsketch_state(Int64(128),Float64(0.01),integers.i), __hll_merge(__hll_state(integers.i)) AS hll(integers.i)]] |
|_|_MergeScan [is_placeholder=false, remote_input=[_|
|_| Aggregate: groupBy=[[]], aggr=[[count(integers.i), sum(integers.i), uddsketch_state(Int64(128), Float64(0.01), CAST(integers.i AS Float64)), hll(CAST(integers.i AS Utf8))]]_|
|_| Aggregate: groupBy=[[]], aggr=[[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128), Float64(0.01), CAST(integers.i AS Float64)), __hll_state(CAST(integers.i AS Utf8))]]_|
|_|_TableScan: integers_|
|_| ]]_|
| physical_plan | ProjectionExec: expr=[sum(count(integers.i))@0 as count(integers.i), sum(sum(integers.i))@1 as sum(integers.i), uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))@2) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))@3) as hll_count(hll(integers.i))] |
|_|_AggregateExec: mode=Final, gby=[], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]_|
| physical_plan | ProjectionExec: expr=[count(integers.i)@0 as count(integers.i), sum(integers.i)@1 as sum(integers.i), uddsketch_calc(0.5, uddsketch_state(Int64(128),Float64(0.01),integers.i)@2) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll(integers.i)@3) as hll_count(hll(integers.i))]_|
|_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)]_|
|_|_CoalescePartitionsExec_|
|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]_|
|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)]_|
|_|_MergeScanExec: REDACTED
|_|_|
+-+-+
@@ -88,25 +88,101 @@ FROM
+-+-+-+
| stage | node | plan_|
+-+-+-+
| 0_| 0_|_ProjectionExec: expr=[sum(count(integers.i))@0 as count(integers.i), sum(sum(integers.i))@1 as sum(integers.i), uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))@2) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))@3) as hll_count(hll(integers.i))] REDACTED
|_|_|_AggregateExec: mode=Final, gby=[], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))] REDACTED
| 0_| 0_|_ProjectionExec: expr=[count(integers.i)@0 as count(integers.i), sum(integers.i)@1 as sum(integers.i), uddsketch_calc(0.5, uddsketch_state(Int64(128),Float64(0.01),integers.i)@2) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll(integers.i)@3) as hll_count(hll(integers.i))] REDACTED
|_|_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
|_|_| Total rows: 1_|
+-+-+-+
SELECT
avg(i)
FROM
integers;
+-----------------+
| avg(integers.i) |
+-----------------+
| 3.0 |
+-----------------+
-- SQLNESS REPLACE (-+) -
-- SQLNESS REPLACE (\s\s+) _
-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED
-- SQLNESS REPLACE (Hash.*) REDACTED
-- SQLNESS REPLACE (peers.*) REDACTED
EXPLAIN SELECT
avg(i)
FROM
integers;
+-+-+
| plan_type_| plan_|
+-+-+
| logical_plan_| Aggregate: groupBy=[[]], aggr=[[__avg_merge(__avg_state(integers.i)) AS avg(integers.i)]]_|
|_|_MergeScan [is_placeholder=false, remote_input=[_|
|_| Aggregate: groupBy=[[]], aggr=[[__avg_state(CAST(integers.i AS Float64))]]_|
|_|_TableScan: integers_|
|_| ]]_|
| physical_plan | AggregateExec: mode=Final, gby=[], aggr=[avg(integers.i)]_|
|_|_CoalescePartitionsExec_|
|_|_AggregateExec: mode=Partial, gby=[], aggr=[avg(integers.i)]_|
|_|_MergeScanExec: REDACTED
|_|_|
+-+-+
-- SQLNESS REPLACE (metrics.*) REDACTED
-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED
-- SQLNESS REPLACE (-+) -
-- SQLNESS REPLACE (\s\s+) _
-- SQLNESS REPLACE (peers.*) REDACTED
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
-- might write to different partitions
-- SQLNESS REPLACE "partition_count":\{(.*?)\} "partition_count":REDACTED
-- SQLNESS REPLACE (Hash.*) REDACTED
EXPLAIN ANALYZE SELECT
avg(i)
FROM
integers;
+-+-+-+
| stage | node | plan_|
+-+-+-+
| 0_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[avg(integers.i)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[avg(integers.i)] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[__avg_state(integers.i)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__avg_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[__avg_state(integers.i)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__avg_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[__avg_state(integers.i)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__avg_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
|_|_| Total rows: 1_|
@@ -157,19 +233,19 @@ ORDER BY
| plan_type_| plan_|
+-+-+
| logical_plan_| Sort: integers.ts ASC NULLS LAST_|
|_|_Projection: integers.ts, sum(count(integers.i)) AS count(integers.i), sum(sum(integers.i)) AS sum(integers.i), uddsketch_calc(Float64(0.5), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))) AS uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))) AS hll_count(hll(integers.i))_|
|_|_Aggregate: groupBy=[[integers.ts]], aggr=[[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]]_|
|_|_Projection: integers.ts, count(integers.i), sum(integers.i), uddsketch_calc(Float64(0.5), uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll(integers.i))_|
|_|_Aggregate: groupBy=[[integers.ts]], aggr=[[__count_merge(__count_state(integers.i)) AS count(integers.i), __sum_merge(__sum_state(integers.i)) AS sum(integers.i), __uddsketch_state_merge(__uddsketch_state_state(Int64(128),Float64(0.01),integers.i)) AS uddsketch_state(Int64(128),Float64(0.01),integers.i), __hll_merge(__hll_state(integers.i)) AS hll(integers.i)]] |
|_|_MergeScan [is_placeholder=false, remote_input=[_|
|_| Aggregate: groupBy=[[integers.ts]], aggr=[[count(integers.i), sum(integers.i), uddsketch_state(Int64(128), Float64(0.01), CAST(integers.i AS Float64)), hll(CAST(integers.i AS Utf8))]]_|
|_| Aggregate: groupBy=[[integers.ts]], aggr=[[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128), Float64(0.01), CAST(integers.i AS Float64)), __hll_state(CAST(integers.i AS Utf8))]]_|
|_|_TableScan: integers_|
|_| ]]_|
| physical_plan | SortPreservingMergeExec: [ts@0 ASC NULLS LAST]_|
|_|_SortExec: expr=[ts@0 ASC NULLS LAST], preserve_partitioning=[true]_|
|_|_ProjectionExec: expr=[ts@0 as ts, sum(count(integers.i))@1 as count(integers.i), sum(sum(integers.i))@2 as sum(integers.i), uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))@3) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))@4) as hll_count(hll(integers.i))] |
|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]_|
|_|_ProjectionExec: expr=[ts@0 as ts, count(integers.i)@1 as count(integers.i), sum(integers.i)@2 as sum(integers.i), uddsketch_calc(0.5, uddsketch_state(Int64(128),Float64(0.01),integers.i)@3) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll(integers.i)@4) as hll_count(hll(integers.i))]_|
|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)]_|
|_|_CoalesceBatchesExec: target_batch_size=8192_|
|_|_RepartitionExec: partitioning=REDACTED
|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))]_|
|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)]_|
|_|_MergeScanExec: REDACTED
|_|_|
+-+-+
@@ -202,29 +278,29 @@ ORDER BY
+-+-+-+
| 0_| 0_|_SortPreservingMergeExec: [ts@0 ASC NULLS LAST] REDACTED
|_|_|_SortExec: expr=[ts@0 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_ProjectionExec: expr=[ts@0 as ts, sum(count(integers.i))@1 as count(integers.i), sum(sum(integers.i))@2 as sum(integers.i), uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i))@3) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll_merge(hll(integers.i))@4) as hll_count(hll(integers.i))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))] REDACTED
|_|_|_ProjectionExec: expr=[ts@0 as ts, count(integers.i)@1 as count(integers.i), sum(integers.i)@2 as sum(integers.i), uddsketch_calc(0.5, uddsketch_state(Int64(128),Float64(0.01),integers.i)@3) as uddsketch_calc(Float64(0.5),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_count(hll(integers.i)@4) as hll_count(hll(integers.i))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[sum(count(integers.i)), sum(sum(integers.i)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),integers.i)), hll_merge(hll(integers.i))] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[count(integers.i), sum(integers.i), uddsketch_state(Int64(128),Float64(0.01),integers.i), hll(integers.i)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[__count_state(integers.i), __sum_state(integers.i), __uddsketch_state_state(Int64(128),Float64(0.01),integers.i), __hll_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
|_|_| Total rows: 4_|

View File

@@ -58,6 +58,35 @@ SELECT
FROM
integers;
SELECT
avg(i)
FROM
integers;
-- SQLNESS REPLACE (-+) -
-- SQLNESS REPLACE (\s\s+) _
-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED
-- SQLNESS REPLACE (Hash.*) REDACTED
-- SQLNESS REPLACE (peers.*) REDACTED
EXPLAIN SELECT
avg(i)
FROM
integers;
-- SQLNESS REPLACE (metrics.*) REDACTED
-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED
-- SQLNESS REPLACE (-+) -
-- SQLNESS REPLACE (\s\s+) _
-- SQLNESS REPLACE (peers.*) REDACTED
-- SQLNESS REPLACE region=\d+\(\d+,\s+\d+\) region=REDACTED
-- might write to different partitions
-- SQLNESS REPLACE "partition_count":\{(.*?)\} "partition_count":REDACTED
-- SQLNESS REPLACE (Hash.*) REDACTED
EXPLAIN ANALYZE SELECT
avg(i)
FROM
integers;
SELECT
ts,
count(i),

View File

@@ -758,20 +758,20 @@ GROUP BY
a,
b;
+---------------+------------------------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+------------------------------------------------------------------------------------------------------------------------+
| logical_plan | Projection: min(min(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value) |
| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[min(min(aggr_optimize_not.greptime_value))]] |
| | MergeScan [is_placeholder=false, remote_input=[ |
| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[min(aggr_optimize_not.greptime_value)]] |
| | TableScan: aggr_optimize_not |
| | ]] |
| physical_plan | ProjectionExec: expr=[min(min(aggr_optimize_not.greptime_value))@2 as min(aggr_optimize_not.greptime_value)] |
| | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(min(aggr_optimize_not.greptime_value))] |
+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| logical_plan | Projection: min(aggr_optimize_not.greptime_value) |
| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[__min_merge(__min_state(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value)]] |
| | MergeScan [is_placeholder=false, remote_input=[ |
| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[__min_state(aggr_optimize_not.greptime_value)]] |
| | TableScan: aggr_optimize_not |
| | ]] |
| physical_plan | ProjectionExec: expr=[min(aggr_optimize_not.greptime_value)@2 as min(aggr_optimize_not.greptime_value)] |
| | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] |
| | MergeScanExec: REDACTED
| | |
+---------------+------------------------------------------------------------------------------------------------------------------------+
| | |
+---------------+---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-- SQLNESS REPLACE (metrics.*) REDACTED
-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED
@@ -792,20 +792,20 @@ GROUP BY
+-+-+-+
| stage | node | plan_|
+-+-+-+
| 0_| 0_|_ProjectionExec: expr=[min(min(aggr_optimize_not.greptime_value))@2 as min(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(min(aggr_optimize_not.greptime_value))] REDACTED
| 0_| 0_|_ProjectionExec: expr=[min(aggr_optimize_not.greptime_value)@2 as min(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 0_|
@@ -824,20 +824,20 @@ GROUP BY
a,
b;
+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| logical_plan | Projection: min(min(aggr_optimize_not.greptime_value)) + max(max(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value) |
| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[min(min(aggr_optimize_not.greptime_value)), max(max(aggr_optimize_not.greptime_value))]] |
| | MergeScan [is_placeholder=false, remote_input=[ |
| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)]] |
| | TableScan: aggr_optimize_not |
| | ]] |
| physical_plan | ProjectionExec: expr=[min(min(aggr_optimize_not.greptime_value))@2 + max(max(aggr_optimize_not.greptime_value))@3 as min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value)] |
| | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(min(aggr_optimize_not.greptime_value)), max(max(aggr_optimize_not.greptime_value))] |
+---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| logical_plan | Projection: min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value) |
| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[__min_merge(__min_state(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value), __max_merge(__max_state(aggr_optimize_not.greptime_value)) AS max(aggr_optimize_not.greptime_value)]] |
| | MergeScan [is_placeholder=false, remote_input=[ |
| | Aggregate: groupBy=[[aggr_optimize_not.a, aggr_optimize_not.b]], aggr=[[__min_state(aggr_optimize_not.greptime_value), __max_state(aggr_optimize_not.greptime_value)]] |
| | TableScan: aggr_optimize_not |
| | ]] |
| physical_plan | ProjectionExec: expr=[min(aggr_optimize_not.greptime_value)@2 + max(aggr_optimize_not.greptime_value)@3 as min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value)] |
| | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] |
| | MergeScanExec: REDACTED
| | |
+---------------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
| | |
+---------------+--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
-- SQLNESS REPLACE (metrics.*) REDACTED
-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED
@@ -858,20 +858,20 @@ GROUP BY
+-+-+-+
| stage | node | plan_|
+-+-+-+
| 0_| 0_|_ProjectionExec: expr=[min(min(aggr_optimize_not.greptime_value))@2 + max(max(aggr_optimize_not.greptime_value))@3 as min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(min(aggr_optimize_not.greptime_value)), max(max(aggr_optimize_not.greptime_value))] REDACTED
| 0_| 0_|_ProjectionExec: expr=[min(aggr_optimize_not.greptime_value)@2 + max(aggr_optimize_not.greptime_value)@3 as min(aggr_optimize_not.greptime_value) + max(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] REDACTED
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value), __max_state(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value), __max_state(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] REDACTED
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value), __max_state(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[min(aggr_optimize_not.greptime_value), max(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[__min_state(aggr_optimize_not.greptime_value), __max_state(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 0_|
@@ -900,21 +900,19 @@ FROM
GROUP BY
a;
+---------------+------------------------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+------------------------------------------------------------------------------------------------------------------------+
| logical_plan | Projection: aggr_optimize_not.a, min(min(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value) |
| | Aggregate: groupBy=[[aggr_optimize_not.a]], aggr=[[min(min(aggr_optimize_not.greptime_value))]] |
| | MergeScan [is_placeholder=false, remote_input=[ |
| | Aggregate: groupBy=[[aggr_optimize_not.a]], aggr=[[min(aggr_optimize_not.greptime_value)]] |
| | Projection: aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.greptime_value |
| | TableScan: aggr_optimize_not |
| | ]] |
| physical_plan | ProjectionExec: expr=[a@0 as a, min(min(aggr_optimize_not.greptime_value))@1 as min(aggr_optimize_not.greptime_value)] |
| | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a], aggr=[min(min(aggr_optimize_not.greptime_value))] |
| | MergeScanExec: REDACTED
| | |
+---------------+------------------------------------------------------------------------------------------------------------------------+
+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------+
| plan_type | plan |
+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------+
| logical_plan | Aggregate: groupBy=[[aggr_optimize_not.a]], aggr=[[__min_merge(__min_state(aggr_optimize_not.greptime_value)) AS min(aggr_optimize_not.greptime_value)]] |
| | MergeScan [is_placeholder=false, remote_input=[ |
| | Aggregate: groupBy=[[aggr_optimize_not.a]], aggr=[[__min_state(aggr_optimize_not.greptime_value)]] |
| | Projection: aggr_optimize_not.a, aggr_optimize_not.b, aggr_optimize_not.greptime_value |
| | TableScan: aggr_optimize_not |
| | ]] |
| physical_plan | AggregateExec: mode=SinglePartitioned, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] |
| | MergeScanExec: REDACTED
| | |
+---------------+----------------------------------------------------------------------------------------------------------------------------------------------------------+
-- SQLNESS REPLACE (metrics.*) REDACTED
-- SQLNESS REPLACE (RoundRobinBatch.*) REDACTED
@@ -945,20 +943,19 @@ GROUP BY
+-+-+-+
| stage | node | plan_|
+-+-+-+
| 0_| 0_|_ProjectionExec: expr=[a@0 as a, min(min(aggr_optimize_not.greptime_value))@1 as min(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a], aggr=[min(min(aggr_optimize_not.greptime_value))] REDACTED
| 0_| 0_|_AggregateExec: mode=SinglePartitioned, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[min(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[__min_state(aggr_optimize_not.greptime_value)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":{"count":0, "mem_ranges":0, "files":0, "file_ranges":0} REDACTED
|_|_|_|
|_|_| Total rows: 0_|

View File

@@ -48,16 +48,14 @@ FROM
+-+-+
| plan_type_| plan_|
+-+-+
| logical_plan_| Projection: sum(count(integers.i)) AS count(integers.i)_|
|_|_Aggregate: groupBy=[[]], aggr=[[sum(count(integers.i))]]_|
| logical_plan_| Aggregate: groupBy=[[]], aggr=[[__count_merge(__count_state(integers.i)) AS count(integers.i)]]_|
|_|_MergeScan [is_placeholder=false, remote_input=[_|
|_| Aggregate: groupBy=[[]], aggr=[[count(integers.i)]]_|
|_| Aggregate: groupBy=[[]], aggr=[[__count_state(integers.i)]]_|
|_|_TableScan: integers_|
|_| ]]_|
| physical_plan | ProjectionExec: expr=[sum(count(integers.i))@0 as count(integers.i)]_|
|_|_AggregateExec: mode=Final, gby=[], aggr=[sum(count(integers.i))]_|
| physical_plan | AggregateExec: mode=Final, gby=[], aggr=[count(integers.i)]_|
|_|_CoalescePartitionsExec_|
|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(count(integers.i))]_|
|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i)]_|
|_|_MergeScanExec: REDACTED
|_|_|
+-+-+
@@ -80,25 +78,24 @@ FROM
+-+-+-+
| stage | node | plan_|
+-+-+-+
| 0_| 0_|_ProjectionExec: expr=[sum(count(integers.i))@0 as count(integers.i)] REDACTED
|_|_|_AggregateExec: mode=Final, gby=[], aggr=[sum(count(integers.i))] REDACTED
| 0_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(count(integers.i))] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i)] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i)] REDACTED
| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(integers.i)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i)] REDACTED
| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(integers.i)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[count(integers.i)] REDACTED
| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(integers.i)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(integers.i)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
|_|_| Total rows: 1_|
@@ -145,19 +142,17 @@ ORDER BY
| plan_type_| plan_|
+-+-+
| logical_plan_| Sort: integers.ts ASC NULLS LAST, count(integers.i) ASC NULLS LAST_|
|_|_Projection: integers.ts, sum(count(integers.i)) AS count(integers.i)_|
|_|_Aggregate: groupBy=[[integers.ts]], aggr=[[sum(count(integers.i))]]_|
|_|_Aggregate: groupBy=[[integers.ts]], aggr=[[__count_merge(__count_state(integers.i)) AS count(integers.i)]] |
|_|_MergeScan [is_placeholder=false, remote_input=[_|
|_| Aggregate: groupBy=[[integers.ts]], aggr=[[count(integers.i)]]_|
|_| Aggregate: groupBy=[[integers.ts]], aggr=[[__count_state(integers.i)]]_|
|_|_TableScan: integers_|
|_| ]]_|
| physical_plan | SortPreservingMergeExec: [ts@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST]_|
|_|_SortExec: expr=[ts@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true]_|
|_|_ProjectionExec: expr=[ts@0 as ts, sum(count(integers.i))@1 as count(integers.i)]_|
|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[sum(count(integers.i))]_|
|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i)]_|
|_|_CoalesceBatchesExec: target_batch_size=8192_|
|_|_RepartitionExec: partitioning=REDACTED
|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[sum(count(integers.i))]_|
|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[count(integers.i)]_|
|_|_MergeScanExec: REDACTED
|_|_|
+-+-+
@@ -188,29 +183,28 @@ ORDER BY
+-+-+-+
| 0_| 0_|_SortPreservingMergeExec: [ts@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST] REDACTED
|_|_|_SortExec: expr=[ts@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_ProjectionExec: expr=[ts@0 as ts, sum(count(integers.i))@1 as count(integers.i)] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[sum(count(integers.i))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[sum(count(integers.i))] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@0 as ts], aggr=[count(integers.i)] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i)] REDACTED
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[__count_state(integers.i)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[count(integers.i)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[__count_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i)] REDACTED
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[__count_state(integers.i)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[count(integers.i)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[__count_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[count(integers.i)] REDACTED
| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[ts@0 as ts], aggr=[__count_state(integers.i)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[count(integers.i)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[ts@1 as ts], aggr=[__count_state(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
|_|_| Total rows: 4_|
@@ -256,20 +250,14 @@ ORDER BY
+-+-+
| plan_type_| plan_|
+-+-+
| logical_plan_| Sort: time_window ASC NULLS LAST, count(integers.i) ASC NULLS LAST_|
|_|_Projection: date_bin(Utf8("1 hour"),integers.ts) AS time_window, sum(count(integers.i)) AS count(integers.i)_|
|_|_Aggregate: groupBy=[[date_bin(Utf8("1 hour"),integers.ts)]], aggr=[[sum(count(integers.i))]]_|
| logical_plan_| MergeSort: time_window ASC NULLS LAST, count(integers.i) ASC NULLS LAST_|
|_|_MergeScan [is_placeholder=false, remote_input=[_|
|_| Aggregate: groupBy=[[date_bin(CAST(Utf8("1 hour") AS Interval(MonthDayNano)), integers.ts)]], aggr=[[count(integers.i)]]_|
|_| Sort: time_window ASC NULLS LAST, count(integers.i) ASC NULLS LAST_|
|_|_Projection: date_bin(Utf8("1 hour"),integers.ts) AS time_window, count(integers.i)_|
|_|_Aggregate: groupBy=[[date_bin(CAST(Utf8("1 hour") AS Interval(MonthDayNano)), integers.ts)]], aggr=[[count(integers.i)]] |
|_|_TableScan: integers_|
|_| ]]_|
| physical_plan | SortPreservingMergeExec: [time_window@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST]_|
|_|_SortExec: expr=[time_window@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true]_|
|_|_ProjectionExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 as time_window, sum(count(integers.i))@1 as count(integers.i)]_|
|_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[sum(count(integers.i))] |
|_|_CoalesceBatchesExec: target_batch_size=8192_|
|_|_RepartitionExec: partitioning=REDACTED
|_|_AggregateExec: mode=Partial, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[sum(count(integers.i))]_|
|_|_MergeScanExec: REDACTED
|_|_|
+-+-+
@@ -299,27 +287,30 @@ ORDER BY
| stage | node | plan_|
+-+-+-+
| 0_| 0_|_SortPreservingMergeExec: [time_window@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST] REDACTED
|_|_|_SortExec: expr=[time_window@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_ProjectionExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 as time_window, sum(count(integers.i))@1 as count(integers.i)] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[sum(count(integers.i))] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[sum(count(integers.i))] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED
| 1_| 0_|_ProjectionExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 as time_window, count(integers.i)@1 as count(integers.i)] REDACTED
|_|_|_SortPreservingMergeExec: [date_bin(Utf8("1 hour"),integers.ts)@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST] REDACTED
|_|_|_SortExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 3600000000000 }, ts@1) as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED
| 1_| 1_|_ProjectionExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 as time_window, count(integers.i)@1 as count(integers.i)] REDACTED
|_|_|_SortPreservingMergeExec: [date_bin(Utf8("1 hour"),integers.ts)@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST] REDACTED
|_|_|_SortExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 3600000000000 }, ts@1) as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED
| 1_| 2_|_ProjectionExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 as time_window, count(integers.i)@1 as count(integers.i)] REDACTED
|_|_|_SortPreservingMergeExec: [date_bin(Utf8("1 hour"),integers.ts)@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST] REDACTED
|_|_|_SortExec: expr=[date_bin(Utf8("1 hour"),integers.ts)@0 ASC NULLS LAST, count(integers.i)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("1 hour"),integers.ts)@0 as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 3600000000000 }, ts@1) as date_bin(Utf8("1 hour"),integers.ts)], aggr=[count(integers.i)] REDACTED
@@ -375,20 +366,14 @@ ORDER BY
+-+-+
| plan_type_| plan_|
+-+-+
| logical_plan_| Sort: integers.ts + Int64(1) ASC NULLS LAST, integers.i / Int64(2) ASC NULLS LAST_|
|_|_Projection: integers.ts + Int64(1), integers.i / Int64(2), sum(count(integers.i)) AS count(integers.i)_|
|_|_Aggregate: groupBy=[[integers.ts + Int64(1), integers.i / Int64(2)]], aggr=[[sum(count(integers.i))]]_|
| logical_plan_| MergeSort: integers.ts + Int64(1) ASC NULLS LAST, integers.i / Int64(2) ASC NULLS LAST_|
|_|_MergeScan [is_placeholder=false, remote_input=[_|
|_| Aggregate: groupBy=[[CAST(integers.ts AS Int64) + Int64(1), integers.i / Int64(2)]], aggr=[[count(integers.i)]]_|
|_| Sort: integers.ts + Int64(1) ASC NULLS LAST, integers.i / Int64(2) ASC NULLS LAST_|
|_|_Projection: integers.ts + Int64(1), integers.i / Int64(2), count(integers.i)_|
|_|_Aggregate: groupBy=[[CAST(integers.ts AS Int64) + Int64(1), integers.i / Int64(2)]], aggr=[[count(integers.i)]] |
|_|_TableScan: integers_|
|_| ]]_|
| physical_plan | SortPreservingMergeExec: [integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST]_|
|_|_SortExec: expr=[integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST], preserve_partitioning=[true]_|
|_|_ProjectionExec: expr=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2), sum(count(integers.i))@2 as count(integers.i)]_|
|_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[sum(count(integers.i))] |
|_|_CoalesceBatchesExec: target_batch_size=8192_|
|_|_RepartitionExec: partitioning=REDACTED
|_|_AggregateExec: mode=Partial, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[sum(count(integers.i))]_|
|_|_MergeScanExec: REDACTED
|_|_|
+-+-+
@@ -421,27 +406,27 @@ ORDER BY
| stage | node | plan_|
+-+-+-+
| 0_| 0_|_SortPreservingMergeExec: [integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST] REDACTED
|_|_|_SortExec: expr=[integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_ProjectionExec: expr=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2), sum(count(integers.i))@2 as count(integers.i)] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[sum(count(integers.i))] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[sum(count(integers.i))] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED
| 1_| 0_|_SortPreservingMergeExec: [integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST] REDACTED
|_|_|_SortExec: expr=[integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[CAST(ts@1 AS Int64) + 1 as integers.ts + Int64(1), i@0 / 2 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED
| 1_| 1_|_SortPreservingMergeExec: [integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST] REDACTED
|_|_|_SortExec: expr=[integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[CAST(ts@1 AS Int64) + 1 as integers.ts + Int64(1), i@0 / 2 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED
| 1_| 2_|_SortPreservingMergeExec: [integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST] REDACTED
|_|_|_SortExec: expr=[integers.ts + Int64(1)@0 ASC NULLS LAST, integers.i / Int64(2)@1 ASC NULLS LAST], preserve_partitioning=[true] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[integers.ts + Int64(1)@0 as integers.ts + Int64(1), integers.i / Int64(2)@1 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[CAST(ts@1 AS Int64) + 1 as integers.ts + Int64(1), i@0 / 2 as integers.i / Int64(2)], aggr=[count(integers.i)] REDACTED
@@ -507,16 +492,16 @@ FROM
+-+-+
| plan_type_| plan_|
+-+-+
| logical_plan_| Projection: uddsketch_calc(Float64(0.5), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state))) AS udd_result, hll_count(hll_merge(hll_merge(sink_table.hll_state))) AS hll_result_|
|_|_Aggregate: groupBy=[[]], aggr=[[uddsketch_merge(Int64(128), Float64(0.01), uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)), hll_merge(hll_merge(sink_table.hll_state))]]_|
| logical_plan_| Projection: uddsketch_calc(Float64(0.5), uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)) AS udd_result, hll_count(hll_merge(sink_table.hll_state)) AS hll_result_|
|_|_Aggregate: groupBy=[[]], aggr=[[__uddsketch_merge_merge(__uddsketch_merge_state(Int64(128),Float64(0.01),sink_table.udd_state)) AS uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), __hll_merge_merge(__hll_merge_state(sink_table.hll_state)) AS hll_merge(sink_table.hll_state)]] |
|_|_MergeScan [is_placeholder=false, remote_input=[_|
|_| Aggregate: groupBy=[[]], aggr=[[uddsketch_merge(Int64(128), Float64(0.01), sink_table.udd_state), hll_merge(sink_table.hll_state)]]_|
|_| Aggregate: groupBy=[[]], aggr=[[__uddsketch_merge_state(Int64(128), Float64(0.01), sink_table.udd_state), __hll_merge_state(sink_table.hll_state)]]_|
|_|_TableScan: sink_table_|
|_| ]]_|
| physical_plan | ProjectionExec: expr=[uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state))@0) as udd_result, hll_count(hll_merge(hll_merge(sink_table.hll_state))@1) as hll_result] |
|_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)), hll_merge(hll_merge(sink_table.hll_state))]_|
| physical_plan | ProjectionExec: expr=[uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)@0) as udd_result, hll_count(hll_merge(sink_table.hll_state)@1) as hll_result]_|
|_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)]_|
|_|_CoalescePartitionsExec_|
|_|_AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)), hll_merge(hll_merge(sink_table.hll_state))]_|
|_|_AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)]_|
|_|_MergeScanExec: REDACTED
|_|_|
+-+-+
@@ -540,25 +525,25 @@ FROM
+-+-+-+
| stage | node | plan_|
+-+-+-+
| 0_| 0_|_ProjectionExec: expr=[uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state))@0) as udd_result, hll_count(hll_merge(hll_merge(sink_table.hll_state))@1) as hll_result] REDACTED
|_|_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)), hll_merge(hll_merge(sink_table.hll_state))] REDACTED
| 0_| 0_|_ProjectionExec: expr=[uddsketch_calc(0.5, uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)@0) as udd_result, hll_count(hll_merge(sink_table.hll_state)@1) as hll_result] REDACTED
|_|_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state)), hll_merge(hll_merge(sink_table.hll_state))] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED
| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[__uddsketch_merge_state(Int64(128),Float64(0.01),sink_table.udd_state), __hll_merge_state(sink_table.hll_state)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__uddsketch_merge_state(Int64(128),Float64(0.01),sink_table.udd_state), __hll_merge_state(sink_table.hll_state)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED
| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[__uddsketch_merge_state(Int64(128),Float64(0.01),sink_table.udd_state), __hll_merge_state(sink_table.hll_state)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__uddsketch_merge_state(Int64(128),Float64(0.01),sink_table.udd_state), __hll_merge_state(sink_table.hll_state)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED
| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[__uddsketch_merge_state(Int64(128),Float64(0.01),sink_table.udd_state), __hll_merge_state(sink_table.hll_state)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[uddsketch_merge(Int64(128),Float64(0.01),sink_table.udd_state), hll_merge(sink_table.hll_state)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__uddsketch_merge_state(Int64(128),Float64(0.01),sink_table.udd_state), __hll_merge_state(sink_table.hll_state)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
|_|_| Total rows: 1_|

View File

@@ -245,19 +245,13 @@ GROUP BY
+-+-+
| plan_type_| plan_|
+-+-+
| logical_plan_| Projection: base_table.env, base_table.service_name, base_table.city, base_table.page, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)) AS lcp_state, max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)) AS max_lcp, min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)) AS min_lcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)) AS fmp_state, max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)) AS max_fmp, min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)) AS min_fmp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)) AS fcp_state, max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)) AS max_fcp, min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)) AS min_fcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)) AS fp_state, max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)) AS max_fp, min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)) AS min_fp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)) AS tti_state, max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)) AS max_tti, min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)) AS min_tti, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)) AS fid_state, max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)) AS max_fid, min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)) AS min_fid, max(max(base_table.shard_key)) AS shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))_|
|_|_Aggregate: groupBy=[[base_table.env, base_table.service_name, base_table.city, base_table.page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))]], aggr=[[uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), uddsketch_merge(Int64(128), Float64(0.01), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(base_table.shard_key))]]_|
|_|_MergeScan [is_placeholder=false, remote_input=[_|
|_| Aggregate: groupBy=[[base_table.env, base_table.service_name, base_table.city, base_table.page, arrow_cast(date_bin(CAST(Utf8("60 seconds") AS Interval(MonthDayNano)), base_table.time), Utf8("Timestamp(Second, None)"))]], aggr=[[uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END), max(base_table.shard_key)]]_|
| logical_plan_| MergeScan [is_placeholder=false, remote_input=[_|
|_| Projection: base_table.env, base_table.service_name, base_table.city, base_table.page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END) AS lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END) AS max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END) AS min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END) AS fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END) AS max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END) AS min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END) AS fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END) AS max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END) AS min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END) AS fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END) AS max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END) AS min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END) AS tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END) AS max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END) AS min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END) AS fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END) AS max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END) AS min_fid, max(base_table.shard_key) AS shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))_|
|_|_Aggregate: groupBy=[[base_table.env, base_table.service_name, base_table.city, base_table.page, arrow_cast(date_bin(CAST(Utf8("60 seconds") AS Interval(MonthDayNano)), base_table.time), Utf8("Timestamp(Second, None)"))]], aggr=[[uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE CAST(NULL AS Int64) END), uddsketch_state(Int64(128), Float64(0.01), CAST(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END AS Float64)), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE CAST(NULL AS Int64) END), max(base_table.shard_key)]] |
|_|_Filter: (base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) OR base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) OR base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) OR base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) OR base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) OR base_table.fid > Int64(0) AND base_table.fid < Int64(3000000)) AND CAST(base_table.time AS Timestamp(Millisecond, Some("+00:00"))) >= CAST(now() AS Timestamp(Millisecond, Some("+00:00")))_|
|_|_TableScan: base_table_|
|_| ]]_|
| physical_plan | ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@5 as lcp_state, max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@6 as max_lcp, min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@7 as min_lcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@8 as fmp_state, max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@9 as max_fmp, min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@10 as min_fmp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@11 as fcp_state, max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@12 as max_fcp, min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@13 as min_fcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@14 as fp_state, max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@15 as max_fp, min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@16 as min_fp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@17 as tti_state, max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@18 as max_tti, min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@19 as min_tti, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@20 as fid_state, max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@21 as max_fid, min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@22 as min_fid, max(max(base_table.shard_key))@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] |
|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(base_table.shard_key))]_|
|_|_CoalesceBatchesExec: target_batch_size=8192_|
|_|_RepartitionExec: partitioning=REDACTED
|_|_AggregateExec: mode=Partial, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(base_table.shard_key))]_|
|_|_MergeScanExec: REDACTED
| physical_plan | MergeScanExec: REDACTED
|_|_|
+-+-+
@@ -456,152 +450,163 @@ GROUP BY
+-+-+-+
| stage | node | plan_|
+-+-+-+
| 0_| 0_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@5 as lcp_state, max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@6 as max_lcp, min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END))@7 as min_lcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@8 as fmp_state, max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@9 as max_fmp, min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END))@10 as min_fmp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@11 as fcp_state, max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@12 as max_fcp, min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END))@13 as min_fcp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@14 as fp_state, max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@15 as max_fp, min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END))@16 as min_fp, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@17 as tti_state, max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@18 as max_tti, min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END))@19 as min_tti, uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@20 as fid_state, max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@21 as max_fid, min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END))@22 as min_fid, max(max(base_table.shard_key))@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(base_table.shard_key))] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), max(max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), min(min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), max(max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), min(min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), max(max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), min(min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), max(max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), min(min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), max(max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), min(min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END)), uddsketch_merge(Int64(128),Float64(0.01),uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), min(min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END)), max(max(base_table.shard_key))] REDACTED
|_|_|_MergeScanExec: REDACTED
| 0_| 0_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 0_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 1_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 2_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 2_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 3_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 3_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 4_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 4_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 5_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 5_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 6_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 6_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 7_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 7_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 8_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 8_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 9_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 9_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 10_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 10_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 11_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 11_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 12_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 12_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 13_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 13_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 14_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 14_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 15_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
| 1_| 15_|_ProjectionExec: expr=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@5 as lcp_state, max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@6 as max_lcp, min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END)@7 as min_lcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@8 as fmp_state, max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@9 as max_fmp, min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END)@10 as min_fmp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@11 as fcp_state, max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@12 as max_fcp, min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END)@13 as min_fcp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@14 as fp_state, max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@15 as max_fp, min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END)@16 as min_fp, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@17 as tti_state, max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@18 as max_tti, min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END)@19 as min_tti, uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@20 as fid_state, max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@21 as max_fid, min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END)@22 as min_fid, max(base_table.shard_key)@23 as shard_key, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[env@0 as env, service_name@1 as service_name, city@2 as city, page@3 as page, arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))@4 as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE NULL END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE NULL END), max(base_table.shard_key)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[env@7 as env, service_name@8 as service_name, city@9 as city, page@10 as page, CAST(date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 60000000000 }, time@6) AS Timestamp(Second, None)) as arrow_cast(date_bin(Utf8("60 seconds"),base_table.time),Utf8("Timestamp(Second, None)"))], aggr=[uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), max(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), min(CASE WHEN base_table.lcp > Int64(0) AND base_table.lcp < Int64(3000000) THEN base_table.lcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), max(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), min(CASE WHEN base_table.fmp > Int64(0) AND base_table.fmp < Int64(3000000) THEN base_table.fmp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), max(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), min(CASE WHEN base_table.fcp > Int64(0) AND base_table.fcp < Int64(3000000) THEN base_table.fcp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), max(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), min(CASE WHEN base_table.fp > Int64(0) AND base_table.fp < Int64(3000000) THEN base_table.fp ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), max(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), min(CASE WHEN base_table.tti > Int64(0) AND base_table.tti < Int64(3000000) THEN base_table.tti ELSE Int64(NULL) END), uddsketch_state(Int64(128),Float64(0.01),CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), min(CASE WHEN base_table.fid > Int64(0) AND base_table.fid < Int64(3000000) THEN base_table.fid ELSE Int64(NULL) END), max(base_table.shard_key)] REDACTED
|_|_|_ProjectionExec: expr=[CASE WHEN lcp@5 > 0 AND lcp@5 < 3000000 THEN lcp@5 END as __common_expr_1, CASE WHEN fmp@6 > 0 AND fmp@6 < 3000000 THEN fmp@6 END as __common_expr_2, CASE WHEN fcp@7 > 0 AND fcp@7 < 3000000 THEN fcp@7 END as __common_expr_3, CASE WHEN fp@8 > 0 AND fp@8 < 3000000 THEN fp@8 END as __common_expr_4, CASE WHEN tti@9 > 0 AND tti@9 < 3000000 THEN tti@9 END as __common_expr_5, CASE WHEN fid@10 > 0 AND fid@10 < 3000000 THEN fid@10 END as __common_expr_6, time@0 as time, env@1 as env, service_name@2 as service_name, city@3 as city, page@4 as page, shard_key@11 as shard_key] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_FilterExec: lcp@5 > 0 AND lcp@5 < 3000000 OR fmp@6 > 0 AND fmp@6 < 3000000 OR fcp@7 > 0 AND fcp@7 < 3000000 OR fp@8 > 0 AND fp@8 < 3000000 OR tti@9 > 0 AND tti@9 < 3000000 OR fid@10 > 0 AND fid@10 < 3000000 REDACTED
@@ -626,10 +631,9 @@ where
+-+-+
| plan_type_| plan_|
+-+-+
| logical_plan_| Projection: count(*) AS count(*)_|
|_|_Aggregate: groupBy=[[]], aggr=[[sum(count(*)) AS count(*)]]_|
| logical_plan_| Aggregate: groupBy=[[]], aggr=[[__count_merge(__count_state(base_table.time)) AS count(*)]]_|
|_|_MergeScan [is_placeholder=false, remote_input=[_|
|_| Aggregate: groupBy=[[]], aggr=[[count(base_table.time) AS count(*)]]_|
|_| Aggregate: groupBy=[[]], aggr=[[__count_state(base_table.time)]]_|
|_|_Filter: CAST(base_table.time AS Timestamp(Millisecond, Some("+00:00"))) >= CAST(now() AS Timestamp(Millisecond, Some("+00:00")))_|
|_|_TableScan: base_table_|
|_| ]]_|
@@ -665,84 +669,84 @@ where
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 2_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 3_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 3_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 4_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 4_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 5_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 5_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 6_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 6_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 7_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 7_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 8_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 8_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 9_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 9_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 10_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 10_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 11_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 11_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 12_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 12_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 13_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 13_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 14_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 14_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 15_|_AggregateExec: mode=Final, gby=[], aggr=[count(*)] REDACTED
| 1_| 15_|_AggregateExec: mode=Final, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[count(*)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__count_state(base_table.time)] REDACTED
|_|_|_UnorderedScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
|_|_| Total rows: 1_|

View File

@@ -60,20 +60,19 @@ select sum(val) from t;
+-+-+-+
| stage | node | plan_|
+-+-+-+
| 0_| 0_|_ProjectionExec: expr=[sum(sum(t.val))@0 as sum(t.val)] REDACTED
|_|_|_AggregateExec: mode=Final, gby=[], aggr=[sum(sum(t.val))] REDACTED
| 0_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[sum(t.val)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(sum(t.val))] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(t.val)] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[sum(t.val)] REDACTED
| 1_| 0_|_AggregateExec: mode=Final, gby=[], aggr=[__sum_state(t.val)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(t.val)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__sum_state(t.val)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[sum(t.val)] REDACTED
| 1_| 1_|_AggregateExec: mode=Final, gby=[], aggr=[__sum_state(t.val)] REDACTED
|_|_|_CoalescePartitionsExec REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[sum(t.val)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[], aggr=[__sum_state(t.val)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
|_|_| Total rows: 1_|
@@ -94,23 +93,23 @@ select sum(val) from t group by idc;
+-+-+-+
| stage | node | plan_|
+-+-+-+
| 0_| 0_|_ProjectionExec: expr=[sum(sum(t.val))@1 as sum(t.val)] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[idc@0 as idc], aggr=[sum(sum(t.val))] REDACTED
| 0_| 0_|_ProjectionExec: expr=[sum(t.val)@1 as sum(t.val)] REDACTED
|_|_|_AggregateExec: mode=FinalPartitioned, gby=[idc@0 as idc], aggr=[sum(t.val)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[idc@0 as idc], aggr=[sum(sum(t.val))] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[idc@0 as idc], aggr=[sum(t.val)] REDACTED
|_|_|_MergeScanExec: REDACTED
|_|_|_|
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[idc@0 as idc], aggr=[sum(t.val)] REDACTED
| 1_| 0_|_AggregateExec: mode=FinalPartitioned, gby=[idc@0 as idc], aggr=[__sum_state(t.val)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[idc@1 as idc], aggr=[sum(t.val)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[idc@1 as idc], aggr=[__sum_state(t.val)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[idc@0 as idc], aggr=[sum(t.val)] REDACTED
| 1_| 1_|_AggregateExec: mode=FinalPartitioned, gby=[idc@0 as idc], aggr=[__sum_state(t.val)] REDACTED
|_|_|_CoalesceBatchesExec: target_batch_size=8192 REDACTED
|_|_|_RepartitionExec: partitioning=REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[idc@1 as idc], aggr=[sum(t.val)] REDACTED
|_|_|_AggregateExec: mode=Partial, gby=[idc@1 as idc], aggr=[__sum_state(t.val)] REDACTED
|_|_|_SeqScan: region=REDACTED, "partition_count":REDACTED REDACTED
|_|_|_|
|_|_| Total rows: 0_|