risingwave_frontend/optimizer/plan_node/
stream_local_approx_percentile.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use fixedbitset::FixedBitSet;
use pretty_xmlish::{Pretty, XmlNode};
use risingwave_common::catalog::{Field, Schema};
use risingwave_common::types::DataType;
use risingwave_pb::stream_plan::stream_node::PbNodeBody;
use risingwave_pb::stream_plan::LocalApproxPercentileNode;

use crate::expr::{ExprRewriter, ExprVisitor, InputRef, InputRefDisplay, Literal};
use crate::optimizer::plan_node::expr_visitable::ExprVisitable;
use crate::optimizer::plan_node::generic::{GenericPlanRef, PhysicalPlanRef};
use crate::optimizer::plan_node::stream::StreamPlanRef;
use crate::optimizer::plan_node::utils::{childless_record, watermark_pretty, Distill};
use crate::optimizer::plan_node::{
    ExprRewritable, PlanAggCall, PlanBase, PlanTreeNodeUnary, Stream, StreamNode,
};
use crate::optimizer::property::FunctionalDependencySet;
use crate::stream_fragmenter::BuildFragmentGraphState;
use crate::PlanRef;

// Does not contain `core` because no other plan nodes share
// common fields and schema, even GlobalApproxPercentile.
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
pub struct StreamLocalApproxPercentile {
    pub base: PlanBase<Stream>,
    input: PlanRef,
    quantile: Literal,
    relative_error: Literal,
    percentile_col: InputRef,
}

impl StreamLocalApproxPercentile {
    pub fn new(input: PlanRef, approx_percentile_agg_call: &PlanAggCall) -> Self {
        let schema = Schema::new(vec![
            Field::with_name(DataType::Int16, "sign"),
            Field::with_name(DataType::Int32, "bucket_id"),
            Field::with_name(DataType::Int32, "count"),
        ]);
        // FIXME(kwannoel): How does watermark work with FixedBitSet
        let watermark_columns = FixedBitSet::with_capacity(3);
        let functional_dependency = FunctionalDependencySet::with_key(3, &[]);
        let base = PlanBase::new_stream(
            input.ctx(),
            schema,
            input.stream_key().map(|k| k.to_vec()),
            functional_dependency,
            input.distribution().clone(),
            input.append_only(),
            input.emit_on_window_close(),
            watermark_columns,
            input.columns_monotonicity().clone(),
        );
        Self {
            base,
            input,
            quantile: approx_percentile_agg_call.direct_args[0].clone(),
            relative_error: approx_percentile_agg_call.direct_args[1].clone(),
            percentile_col: approx_percentile_agg_call.inputs[0].clone(),
        }
    }
}

impl Distill for StreamLocalApproxPercentile {
    fn distill<'a>(&self) -> XmlNode<'a> {
        let mut out = Vec::with_capacity(5);
        out.push((
            "percentile_col",
            Pretty::display(&InputRefDisplay {
                input_ref: &self.percentile_col,
                input_schema: self.input.schema(),
            }),
        ));
        out.push(("quantile", Pretty::debug(&self.quantile)));
        out.push(("relative_error", Pretty::debug(&self.relative_error)));
        if let Some(ow) = watermark_pretty(self.base.watermark_columns(), self.schema()) {
            out.push(("output_watermarks", ow));
        }
        childless_record("StreamLocalApproxPercentile", out)
    }
}

impl PlanTreeNodeUnary for StreamLocalApproxPercentile {
    fn input(&self) -> PlanRef {
        self.input.clone()
    }

    fn clone_with_input(&self, input: PlanRef) -> Self {
        Self {
            base: self.base.clone(),
            input,
            quantile: self.quantile.clone(),
            relative_error: self.relative_error.clone(),
            percentile_col: self.percentile_col.clone(),
        }
    }
}

impl_plan_tree_node_for_unary! {StreamLocalApproxPercentile}

impl StreamNode for StreamLocalApproxPercentile {
    fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> PbNodeBody {
        let relative_error = self.relative_error.get_data().as_ref().unwrap();
        let relative_error = relative_error.as_float64().into_inner();
        let base = (1.0 + relative_error) / (1.0 - relative_error);
        let percentile_index = self.percentile_col.index() as u32;
        let body = LocalApproxPercentileNode {
            base,
            percentile_index,
        };
        PbNodeBody::LocalApproxPercentile(body)
    }
}

impl ExprRewritable for StreamLocalApproxPercentile {
    fn has_rewritable_expr(&self) -> bool {
        false
    }

    fn rewrite_exprs(&self, _rewriter: &mut dyn ExprRewriter) -> PlanRef {
        unimplemented!()
    }
}

impl ExprVisitable for StreamLocalApproxPercentile {
    fn visit_exprs(&self, _v: &mut dyn ExprVisitor) {}
}