risingwave_frontend/expr/
literal.rs1use risingwave_common::types::{DataType, Datum, ToText, literal_type_match};
16use risingwave_common::util::value_encoding::{DatumFromProtoExt, DatumToProtoExt};
17use risingwave_pb::expr::expr_node::RexNode;
18
19use super::Expr;
20use crate::expr::ExprType;
21#[derive(Clone, Eq, PartialEq, Hash)]
22pub struct Literal {
23 data: Datum,
24 data_type: Option<DataType>,
26}
27
28impl std::fmt::Debug for Literal {
29 fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
30 if f.alternate() {
31 f.debug_struct("Literal")
32 .field("data", &self.data)
33 .field("data_type", &self.data_type)
34 .finish()
35 } else {
36 let data_type = self.return_type();
37 match &self.data {
38 None => write!(f, "null"),
39 Some(v) => match data_type {
40 DataType::Boolean => write!(f, "{}", v.as_bool()),
41 DataType::Int16
42 | DataType::Int32
43 | DataType::Int64
44 | DataType::Serial
45 | DataType::Decimal
46 | DataType::Float32
47 | DataType::Float64 => write!(f, "{}", v.as_scalar_ref_impl().to_text()),
48 DataType::Varchar
49 | DataType::Bytea
50 | DataType::Date
51 | DataType::Timestamp
52 | DataType::Timestamptz
53 | DataType::Time
54 | DataType::Interval
55 | DataType::Jsonb
56 | DataType::Int256
57 | DataType::Struct(_)
58 | DataType::Map(_) => write!(
59 f,
60 "'{}'",
61 v.as_scalar_ref_impl().to_text_with_type(&data_type)
62 ),
63 DataType::List { .. } => write!(f, "{}", v.as_list().display_for_explain()),
64 DataType::Vector(_) => todo!("VECTOR_PLACEHOLDER"),
65 },
66 }?;
67 write!(f, ":{:?}", data_type)
68 }
69 }
70}
71
72impl Literal {
73 pub fn new(data: Datum, data_type: DataType) -> Self {
74 assert!(
75 literal_type_match(&data_type, data.as_ref()),
76 "data_type: {:?}, data: {:?}",
77 data_type,
78 data
79 );
80 Literal {
81 data,
82 data_type: Some(data_type),
83 }
84 }
85
86 pub fn new_untyped(data: Option<String>) -> Self {
87 Literal {
88 data: data.map(Into::into),
89 data_type: None,
90 }
91 }
92
93 pub fn get_data(&self) -> &Datum {
94 &self.data
95 }
96
97 pub fn get_data_type(&self) -> &Option<DataType> {
98 &self.data_type
99 }
100
101 pub fn is_untyped(&self) -> bool {
102 self.data_type.is_none()
103 }
104
105 pub(super) fn from_expr_proto(
106 proto: &risingwave_pb::expr::ExprNode,
107 ) -> crate::error::Result<Self> {
108 let data_type = proto.get_return_type()?;
109 Ok(Self {
110 data: value_encoding_to_literal(&proto.rex_node, &data_type.into())?,
111 data_type: Some(data_type.into()),
112 })
113 }
114}
115
116impl Expr for Literal {
117 fn return_type(&self) -> DataType {
118 self.data_type.clone().unwrap_or(DataType::Varchar)
119 }
120
121 fn to_expr_proto(&self) -> risingwave_pb::expr::ExprNode {
122 use risingwave_pb::expr::*;
123 ExprNode {
124 function_type: ExprType::Unspecified as i32,
125 return_type: Some(self.return_type().to_protobuf()),
126 rex_node: Some(literal_to_value_encoding(self.get_data())),
127 }
128 }
129}
130
131pub fn literal_to_value_encoding(d: &Datum) -> RexNode {
133 RexNode::Constant(d.to_protobuf())
134}
135
136fn value_encoding_to_literal(
138 proto: &Option<RexNode>,
139 ty: &DataType,
140) -> crate::error::Result<Datum> {
141 if let Some(rex_node) = proto {
142 if let RexNode::Constant(prost_datum) = rex_node {
143 let datum = Datum::from_protobuf(prost_datum, ty)?;
144 Ok(datum)
145 } else {
146 unreachable!()
147 }
148 } else {
149 Ok(None)
150 }
151}
152
153#[cfg(test)]
154mod tests {
155 use risingwave_common::array::{ListValue, StructValue};
156 use risingwave_common::types::{DataType, Datum, ScalarImpl, StructType};
157 use risingwave_common::util::value_encoding::DatumFromProtoExt;
158 use risingwave_pb::expr::expr_node::RexNode;
159
160 use crate::expr::literal::literal_to_value_encoding;
161
162 #[test]
163 fn test_struct_to_value_encoding() {
164 let value = StructValue::new(vec![
165 Some(ScalarImpl::Utf8("".into())),
166 Some(2.into()),
167 Some(3.into()),
168 ]);
169 let data = Some(ScalarImpl::Struct(value.clone()));
170 let node = literal_to_value_encoding(&data);
171 if let RexNode::Constant(prost) = node {
172 let data2 = Datum::from_protobuf(
173 &prost,
174 &StructType::unnamed(vec![DataType::Varchar, DataType::Int32, DataType::Int32])
175 .into(),
176 )
177 .unwrap()
178 .unwrap();
179 assert_eq!(ScalarImpl::Struct(value), data2);
180 }
181 }
182
183 #[test]
184 fn test_list_to_value_encoding() {
185 let value = ListValue::from_iter(["1", "2", ""]);
186 let data = Some(ScalarImpl::List(value.clone()));
187 let node = literal_to_value_encoding(&data);
188 if let RexNode::Constant(prost) = node {
189 let data2 = Datum::from_protobuf(&prost, &DataType::List(Box::new(DataType::Varchar)))
190 .unwrap()
191 .unwrap();
192 assert_eq!(ScalarImpl::List(value), data2);
193 }
194 }
195}