risingwave_frontend/expr/
window_function.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
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
// 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 itertools::Itertools;
use risingwave_common::bail_not_implemented;
use risingwave_common::types::DataType;
use risingwave_expr::aggregate::AggType;
use risingwave_expr::window_function::{Frame, WindowFuncKind};

use super::{Expr, ExprImpl, OrderBy, RwResult};
use crate::error::{ErrorCode, RwError};
use crate::expr::infer_type;

/// A window function performs a calculation across a set of table rows that are somehow related to
/// the current row, according to the window spec `OVER (PARTITION BY .. ORDER BY ..)`.
/// One output row is calculated for each row in the input table.
///
/// Window functions are permitted only in the `SELECT` list and the `ORDER BY` clause of the query.
/// They are forbidden elsewhere, such as in `GROUP BY`, `HAVING` and `WHERE` clauses.
#[derive(Clone, Eq, PartialEq, Hash)]
pub struct WindowFunction {
    pub kind: WindowFuncKind,
    pub args: Vec<ExprImpl>,
    pub return_type: DataType,
    pub partition_by: Vec<ExprImpl>,
    pub order_by: OrderBy,
    pub frame: Option<Frame>,
}

impl WindowFunction {
    /// Create a `WindowFunction` expr with the return type inferred from `func_type` and types of
    /// `inputs`.
    pub fn new(
        kind: WindowFuncKind,
        partition_by: Vec<ExprImpl>,
        order_by: OrderBy,
        mut args: Vec<ExprImpl>,
        frame: Option<Frame>,
    ) -> RwResult<Self> {
        let return_type = Self::infer_return_type(&kind, &mut args)?;
        Ok(Self {
            kind,
            args,
            return_type,
            partition_by,
            order_by,
            frame,
        })
    }

    fn infer_return_type(kind: &WindowFuncKind, args: &mut [ExprImpl]) -> RwResult<DataType> {
        use WindowFuncKind::*;
        match (kind, args) {
            (RowNumber, []) => Ok(DataType::Int64),
            (Rank, []) => Ok(DataType::Int64),
            (DenseRank, []) => Ok(DataType::Int64),

            (Lag | Lead, [value]) => Ok(value.return_type()),
            (Lag | Lead, [value, offset]) => {
                if !offset.return_type().is_int() {
                    return Err(ErrorCode::InvalidInputSyntax(format!(
                        "the `offset` of `{kind}` function should be integer"
                    ))
                    .into());
                }
                if !offset.is_const() {
                    bail_not_implemented!(
                        "non-const `offset` of `{kind}` function is not supported yet"
                    );
                }
                Ok(value.return_type())
            }
            (Lag | Lead, [_value, _offset, _default]) => {
                bail_not_implemented!(
                    "`{kind}` window function with `default` argument is not supported yet"
                );
            }

            (Aggregate(agg_type), args) => Ok(match agg_type {
                AggType::Builtin(kind) => infer_type((*kind).into(), args)?,
                AggType::UserDefined(udf) => udf.return_type.as_ref().unwrap().into(),
                AggType::WrapScalar(expr) => expr.return_type.as_ref().unwrap().into(),
            }),

            (_, args) => {
                let args = args
                    .iter()
                    .map(|e| format!("{}", e.return_type()))
                    .join(", ");
                Err(RwError::from(ErrorCode::InvalidInputSyntax(format!(
                    "Invalid window function: {kind}({args})"
                ))))
            }
        }
    }
}

impl std::fmt::Debug for WindowFunction {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        if f.alternate() {
            let mut builder = f.debug_struct("WindowFunction");
            builder
                .field("kind", &self.kind)
                .field("return_type", &self.return_type)
                .field("args", &self.args)
                .field("partition_by", &self.partition_by)
                .field("order_by", &format_args!("{}", self.order_by));
            if let Some(frame) = &self.frame {
                builder.field("frame", &format_args!("{}", frame));
            } else {
                builder.field("frame", &"None".to_string());
            }
            builder.finish()
        } else {
            write!(f, "{}() OVER(", self.kind)?;

            let mut delim = "";
            if !self.partition_by.is_empty() {
                delim = " ";
                write!(
                    f,
                    "PARTITION BY {:?}",
                    self.partition_by.iter().format(", ")
                )?;
            }
            if !self.order_by.sort_exprs.is_empty() {
                write!(f, "{delim}{}", self.order_by)?;
            }
            if let Some(frame) = &self.frame {
                write!(f, "{delim}{}", frame)?;
            }
            f.write_str(")")?;

            Ok(())
        }
    }
}

impl Expr for WindowFunction {
    fn return_type(&self) -> DataType {
        self.return_type.clone()
    }

    fn to_expr_proto(&self) -> risingwave_pb::expr::ExprNode {
        unreachable!("Window function should not be converted to ExprNode")
    }
}