risingwave_frontend/expr/
subquery.rsuse std::hash::Hash;
use risingwave_common::types::DataType;
use super::{Expr, ExprImpl, ExprType};
use crate::binder::BoundQuery;
use crate::expr::{CorrelatedId, Depth};
#[derive(Clone, Debug, PartialEq, Eq)]
pub enum SubqueryKind {
Scalar,
UpdateSet,
Existential,
In(ExprImpl),
Some(ExprImpl, ExprType),
All(ExprImpl, ExprType),
Array,
}
#[derive(Clone)]
pub struct Subquery {
pub query: BoundQuery,
pub kind: SubqueryKind,
}
impl Subquery {
pub fn new(query: BoundQuery, kind: SubqueryKind) -> Self {
Self { query, kind }
}
pub fn is_correlated(&self, depth: Depth) -> bool {
self.query.is_correlated(depth)
}
pub fn collect_correlated_indices_by_depth_and_assign_id(
&mut self,
depth: Depth,
correlated_id: CorrelatedId,
) -> Vec<usize> {
let mut correlated_indices = self
.query
.collect_correlated_indices_by_depth_and_assign_id(depth, correlated_id);
correlated_indices.sort();
correlated_indices.dedup();
correlated_indices
}
}
impl PartialEq for Subquery {
fn eq(&self, _other: &Self) -> bool {
unreachable!("Subquery {:?} has not been unnested", self)
}
}
impl Hash for Subquery {
fn hash<H: std::hash::Hasher>(&self, _state: &mut H) {
unreachable!("Subquery {:?} has not been hashed", self)
}
}
impl Eq for Subquery {}
impl Expr for Subquery {
fn return_type(&self) -> DataType {
match self.kind {
SubqueryKind::Scalar => {
let types = self.query.data_types();
assert_eq!(types.len(), 1, "Subquery with more than one column");
types[0].clone()
}
SubqueryKind::UpdateSet => DataType::new_unnamed_struct(self.query.data_types()),
SubqueryKind::Array => {
let types = self.query.data_types();
assert_eq!(types.len(), 1, "Subquery with more than one column");
DataType::List(types[0].clone().into())
}
_ => DataType::Boolean,
}
}
fn to_expr_proto(&self) -> risingwave_pb::expr::ExprNode {
unreachable!("Subquery {:?} has not been unnested", self)
}
}
impl std::fmt::Debug for Subquery {
fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
f.debug_struct("Subquery")
.field("kind", &self.kind)
.field("query", &self.query)
.finish()
}
}