risingwave_expr/window_function/
kind.rsuse anyhow::Context;
use enum_as_inner::EnumAsInner;
use parse_display::{Display, FromStr};
use risingwave_common::bail;
use crate::aggregate::AggType;
use crate::Result;
#[expect(clippy::large_enum_variant)]
#[derive(Debug, Display, FromStr , Clone, PartialEq, Eq, Hash, EnumAsInner)]
#[display(style = "snake_case")]
pub enum WindowFuncKind {
RowNumber,
Rank,
DenseRank,
Lag,
Lead,
#[display("{0}")]
Aggregate(AggType),
}
impl WindowFuncKind {
pub fn from_protobuf(
window_function_type: &risingwave_pb::expr::window_function::PbType,
) -> Result<Self> {
use risingwave_pb::expr::agg_call::PbKind as PbAggKind;
use risingwave_pb::expr::window_function::{PbGeneralType, PbType};
let kind = match window_function_type {
PbType::General(typ) => match PbGeneralType::try_from(*typ) {
Ok(PbGeneralType::Unspecified) => bail!("Unspecified window function type"),
Ok(PbGeneralType::RowNumber) => Self::RowNumber,
Ok(PbGeneralType::Rank) => Self::Rank,
Ok(PbGeneralType::DenseRank) => Self::DenseRank,
Ok(PbGeneralType::Lag) => Self::Lag,
Ok(PbGeneralType::Lead) => Self::Lead,
Err(_) => bail!("no such window function type"),
},
PbType::Aggregate(kind) => Self::Aggregate(AggType::from_protobuf_flatten(
PbAggKind::try_from(*kind).context("no such aggregate function type")?,
None,
None,
)?),
PbType::Aggregate2(agg_type) => Self::Aggregate(AggType::from_protobuf(agg_type)?),
};
Ok(kind)
}
}
impl WindowFuncKind {
pub fn is_numbering(&self) -> bool {
matches!(self, Self::RowNumber | Self::Rank | Self::DenseRank)
}
}