risingwave_sqlparser/ast/
mod.rs

1// Licensed under the Apache License, Version 2.0 (the "License");
2// you may not use this file except in compliance with the License.
3// You may obtain a copy of the License at
4//
5//     http://www.apache.org/licenses/LICENSE-2.0
6//
7// Unless required by applicable law or agreed to in writing, software
8// distributed under the License is distributed on an "AS IS" BASIS,
9// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
10// See the License for the specific language governing permissions and
11// limitations under the License.
12
13//! SQL Abstract Syntax Tree (AST) types
14mod analyze;
15mod data_type;
16pub(crate) mod ddl;
17mod legacy_source;
18mod operator;
19mod query;
20mod statement;
21mod value;
22
23#[cfg(not(feature = "std"))]
24use alloc::{
25    boxed::Box,
26    string::{String, ToString},
27    vec::Vec,
28};
29use core::fmt;
30use core::fmt::Display;
31use std::collections::HashSet;
32use std::sync::Arc;
33
34use itertools::Itertools;
35#[cfg(feature = "serde")]
36use serde::{Deserialize, Serialize};
37use winnow::ModalResult;
38
39pub use self::data_type::{DataType, StructField};
40pub use self::ddl::{
41    AlterColumnOperation, AlterConnectionOperation, AlterDatabaseOperation, AlterFragmentOperation,
42    AlterFunctionOperation, AlterSchemaOperation, AlterSecretOperation, AlterTableOperation,
43    ColumnDef, ColumnOption, ColumnOptionDef, ReferentialAction, SourceWatermark, TableConstraint,
44    WebhookSourceInfo,
45};
46pub use self::legacy_source::{CompatibleFormatEncode, get_delimiter};
47pub use self::operator::{BinaryOperator, QualifiedOperator, UnaryOperator};
48pub use self::query::{
49    Corresponding, Cte, CteInner, Distinct, Fetch, Join, JoinConstraint, JoinOperator, LateralView,
50    OrderByExpr, Query, Select, SelectItem, SetExpr, SetOperator, TableAlias, TableFactor,
51    TableWithJoins, Top, Values, With,
52};
53pub use self::statement::*;
54pub use self::value::{
55    ConnectionRefValue, CstyleEscapedString, DateTimeField, DollarQuotedString, JsonPredicateType,
56    SecretRefAsType, SecretRefValue, TrimWhereField, Value,
57};
58pub use crate::ast::analyze::AnalyzeTarget;
59pub use crate::ast::ddl::{
60    AlterIndexOperation, AlterSinkOperation, AlterSourceOperation, AlterSubscriptionOperation,
61    AlterViewOperation,
62};
63use crate::keywords::Keyword;
64use crate::parser::{IncludeOption, IncludeOptionItem, Parser, ParserError, StrError};
65use crate::tokenizer::Tokenizer;
66
67pub type RedactSqlOptionKeywordsRef = Arc<HashSet<String>>;
68
69task_local::task_local! {
70    pub static REDACT_SQL_OPTION_KEYWORDS: RedactSqlOptionKeywordsRef;
71}
72
73pub struct DisplaySeparated<'a, T>
74where
75    T: fmt::Display,
76{
77    slice: &'a [T],
78    sep: &'static str,
79}
80
81impl<T> fmt::Display for DisplaySeparated<'_, T>
82where
83    T: fmt::Display,
84{
85    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
86        let mut delim = "";
87        for t in self.slice {
88            write!(f, "{}", delim)?;
89            delim = self.sep;
90            write!(f, "{}", t)?;
91        }
92        Ok(())
93    }
94}
95
96pub fn display_separated<'a, T>(slice: &'a [T], sep: &'static str) -> DisplaySeparated<'a, T>
97where
98    T: fmt::Display,
99{
100    DisplaySeparated { slice, sep }
101}
102
103pub fn display_comma_separated<T>(slice: &[T]) -> DisplaySeparated<'_, T>
104where
105    T: fmt::Display,
106{
107    DisplaySeparated { slice, sep: ", " }
108}
109
110/// An identifier, decomposed into its value or character data and the quote style.
111#[derive(Debug, Clone, PartialEq, Eq, Hash)]
112#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
113pub struct Ident {
114    /// The value of the identifier without quotes.
115    pub(crate) value: String,
116    /// The starting quote if any. Valid quote characters are the single quote,
117    /// double quote, backtick, and opening square bracket.
118    pub(crate) quote_style: Option<char>,
119}
120
121impl Ident {
122    /// Create a new identifier with the given value and no quotes.
123    /// the given value must not be a empty string.
124    pub fn new_unchecked<S>(value: S) -> Self
125    where
126        S: Into<String>,
127    {
128        Ident {
129            value: value.into(),
130            quote_style: None,
131        }
132    }
133
134    /// Create a new quoted identifier with the given quote and value.
135    /// the given value must not be a empty string and the given quote must be in ['\'', '"', '`',
136    /// '['].
137    pub fn with_quote_unchecked<S>(quote: char, value: S) -> Self
138    where
139        S: Into<String>,
140    {
141        Ident {
142            value: value.into(),
143            quote_style: Some(quote),
144        }
145    }
146
147    /// Create a new quoted identifier with the given quote and value.
148    /// returns ParserError when the given string is empty or the given quote is illegal.
149    pub fn with_quote_check<S>(quote: char, value: S) -> Result<Ident, ParserError>
150    where
151        S: Into<String>,
152    {
153        let value_str = value.into();
154        if value_str.is_empty() {
155            return Err(ParserError::ParserError(format!(
156                "zero-length delimited identifier at or near \"{value_str}\""
157            )));
158        }
159
160        if !(quote == '\'' || quote == '"' || quote == '`' || quote == '[') {
161            return Err(ParserError::ParserError(
162                "unexpected quote style".to_owned(),
163            ));
164        }
165
166        Ok(Ident {
167            value: value_str,
168            quote_style: Some(quote),
169        })
170    }
171
172    /// Value after considering quote style
173    /// In certain places, double quotes can force case-sensitive, but not always
174    /// e.g. session variables.
175    pub fn real_value(&self) -> String {
176        match self.quote_style {
177            Some('"') => self.value.clone(),
178            _ => self.value.to_lowercase(),
179        }
180    }
181
182    pub fn quote_style(&self) -> Option<char> {
183        self.quote_style
184    }
185}
186
187impl From<&str> for Ident {
188    fn from(value: &str) -> Self {
189        Ident {
190            value: value.to_owned(),
191            quote_style: None,
192        }
193    }
194}
195
196impl ParseTo for Ident {
197    fn parse_to(parser: &mut Parser<'_>) -> ModalResult<Self> {
198        parser.parse_identifier()
199    }
200}
201
202impl fmt::Display for Ident {
203    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
204        match self.quote_style {
205            Some(q) if q == '"' || q == '\'' || q == '`' => write!(f, "{}{}{}", q, self.value, q),
206            Some('[') => write!(f, "[{}]", self.value),
207            None => f.write_str(&self.value),
208            _ => panic!("unexpected quote style"),
209        }
210    }
211}
212
213/// A name of a table, view, custom type, etc., possibly multi-part, i.e. db.schema.obj
214///
215/// Is is ensured to be non-empty.
216#[derive(Debug, Clone, PartialEq, Eq, Hash)]
217#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
218pub struct ObjectName(pub Vec<Ident>);
219
220impl ObjectName {
221    pub fn real_value(&self) -> String {
222        self.0
223            .iter()
224            .map(|ident| ident.real_value())
225            .collect::<Vec<_>>()
226            .join(".")
227    }
228
229    pub fn from_test_str(s: &str) -> Self {
230        ObjectName::from(vec![s.into()])
231    }
232
233    pub fn base_name(&self) -> String {
234        self.0
235            .iter()
236            .last()
237            .expect("should have base name")
238            .real_value()
239    }
240}
241
242impl fmt::Display for ObjectName {
243    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
244        write!(f, "{}", display_separated(&self.0, "."))
245    }
246}
247
248impl ParseTo for ObjectName {
249    fn parse_to(p: &mut Parser<'_>) -> ModalResult<Self> {
250        p.parse_object_name()
251    }
252}
253
254impl From<Vec<Ident>> for ObjectName {
255    fn from(value: Vec<Ident>) -> Self {
256        Self(value)
257    }
258}
259
260/// For array type `ARRAY[..]` or `[..]`
261#[derive(Debug, Clone, PartialEq, Eq, Hash)]
262#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
263pub struct Array {
264    /// The list of expressions between brackets
265    pub elem: Vec<Expr>,
266
267    /// `true` for  `ARRAY[..]`, `false` for `[..]`
268    pub named: bool,
269}
270
271impl fmt::Display for Array {
272    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
273        write!(
274            f,
275            "{}[{}]",
276            if self.named { "ARRAY" } else { "" },
277            display_comma_separated(&self.elem)
278        )
279    }
280}
281
282/// An escape character, to represent '' or a single character.
283#[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)]
284#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
285pub struct EscapeChar(Option<char>);
286
287impl EscapeChar {
288    pub fn escape(ch: char) -> Self {
289        Self(Some(ch))
290    }
291
292    pub fn empty() -> Self {
293        Self(None)
294    }
295}
296
297impl fmt::Display for EscapeChar {
298    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
299        match self.0 {
300            Some(ch) => write!(f, "{}", ch),
301            None => f.write_str(""),
302        }
303    }
304}
305
306/// An SQL expression of any type.
307///
308/// The parser does not distinguish between expressions of different types
309/// (e.g. boolean vs string), so the caller must handle expressions of
310/// inappropriate type, like `WHERE 1` or `SELECT 1=1`, as necessary.
311#[derive(Debug, Clone, PartialEq, Eq, Hash)]
312#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
313pub enum Expr {
314    /// Identifier e.g. table name or column name
315    Identifier(Ident),
316    /// Multi-part identifier, e.g. `table_alias.column` or `schema.table.col`
317    CompoundIdentifier(Vec<Ident>),
318    /// Struct-field identifier.
319    /// Expr is an arbitrary expression, returning either a table or a column.
320    /// Idents are consecutive field accesses.
321    /// e.g. `(table.v1).v2` or `(table).v1.v2`
322    ///
323    /// It must contain parentheses to be distinguished from a [`Expr::CompoundIdentifier`].
324    /// See also <https://www.postgresql.org/docs/current/rowtypes.html#ROWTYPES-ACCESSING>
325    ///
326    /// The left parentheses must be put at the beginning of the expression.
327    /// The first parenthesized part is the `expr` part, and the rest are flattened into `idents`.
328    /// e.g., `((v1).v2.v3).v4` is equivalent to `(v1).v2.v3.v4`.
329    FieldIdentifier(Box<Expr>, Vec<Ident>),
330    /// `IS NULL` operator
331    IsNull(Box<Expr>),
332    /// `IS NOT NULL` operator
333    IsNotNull(Box<Expr>),
334    /// `IS TRUE` operator
335    IsTrue(Box<Expr>),
336    /// `IS NOT TRUE` operator
337    IsNotTrue(Box<Expr>),
338    /// `IS FALSE` operator
339    IsFalse(Box<Expr>),
340    /// `IS NOT FALSE` operator
341    IsNotFalse(Box<Expr>),
342    /// `IS UNKNOWN` operator
343    IsUnknown(Box<Expr>),
344    /// `IS NOT UNKNOWN` operator
345    IsNotUnknown(Box<Expr>),
346    /// `IS DISTINCT FROM` operator
347    IsDistinctFrom(Box<Expr>, Box<Expr>),
348    /// `IS NOT DISTINCT FROM` operator
349    IsNotDistinctFrom(Box<Expr>, Box<Expr>),
350    /// ```text
351    /// IS [ NOT ] JSON [ VALUE | ARRAY | OBJECT | SCALAR ]
352    /// [ { WITH | WITHOUT } UNIQUE [ KEYS ] ]
353    /// ```
354    IsJson {
355        expr: Box<Expr>,
356        negated: bool,
357        item_type: JsonPredicateType,
358        unique_keys: bool,
359    },
360    /// `[ NOT ] IN (val1, val2, ...)`
361    InList {
362        expr: Box<Expr>,
363        list: Vec<Expr>,
364        negated: bool,
365    },
366    /// `[ NOT ] IN (SELECT ...)`
367    InSubquery {
368        expr: Box<Expr>,
369        subquery: Box<Query>,
370        negated: bool,
371    },
372    /// `<expr> [ NOT ] BETWEEN <low> AND <high>`
373    Between {
374        expr: Box<Expr>,
375        negated: bool,
376        low: Box<Expr>,
377        high: Box<Expr>,
378    },
379    /// LIKE
380    Like {
381        negated: bool,
382        expr: Box<Expr>,
383        pattern: Box<Expr>,
384        escape_char: Option<EscapeChar>,
385    },
386    /// ILIKE (case-insensitive LIKE)
387    ILike {
388        negated: bool,
389        expr: Box<Expr>,
390        pattern: Box<Expr>,
391        escape_char: Option<EscapeChar>,
392    },
393    /// `<expr> [ NOT ] SIMILAR TO <pat> ESCAPE <esc_text>`
394    SimilarTo {
395        negated: bool,
396        expr: Box<Expr>,
397        pattern: Box<Expr>,
398        escape_char: Option<EscapeChar>,
399    },
400    /// Binary operation e.g. `1 + 1` or `foo > bar`
401    BinaryOp {
402        left: Box<Expr>,
403        op: BinaryOperator,
404        right: Box<Expr>,
405    },
406    /// Some operation e.g. `foo > Some(bar)`, It will be wrapped in the right side of BinaryExpr
407    SomeOp(Box<Expr>),
408    /// ALL operation e.g. `foo > ALL(bar)`, It will be wrapped in the right side of BinaryExpr
409    AllOp(Box<Expr>),
410    /// Unary operation e.g. `NOT foo`
411    UnaryOp {
412        op: UnaryOperator,
413        expr: Box<Expr>,
414    },
415    /// CAST an expression to a different data type e.g. `CAST(foo AS VARCHAR)`
416    Cast {
417        expr: Box<Expr>,
418        data_type: DataType,
419    },
420    /// TRY_CAST an expression to a different data type e.g. `TRY_CAST(foo AS VARCHAR)`
421    //  this differs from CAST in the choice of how to implement invalid conversions
422    TryCast {
423        expr: Box<Expr>,
424        data_type: DataType,
425    },
426    /// AT TIME ZONE converts `timestamp without time zone` to/from `timestamp with time zone` with
427    /// explicitly specified zone
428    AtTimeZone {
429        timestamp: Box<Expr>,
430        time_zone: Box<Expr>,
431    },
432    /// `EXTRACT(DateTimeField FROM <expr>)`
433    Extract {
434        field: String,
435        expr: Box<Expr>,
436    },
437    /// `SUBSTRING(<expr> [FROM <expr>] [FOR <expr>])`
438    Substring {
439        expr: Box<Expr>,
440        substring_from: Option<Box<Expr>>,
441        substring_for: Option<Box<Expr>>,
442    },
443    /// `POSITION(<expr> IN <expr>)`
444    Position {
445        substring: Box<Expr>,
446        string: Box<Expr>,
447    },
448    /// `OVERLAY(<expr> PLACING <expr> FROM <expr> [ FOR <expr> ])`
449    Overlay {
450        expr: Box<Expr>,
451        new_substring: Box<Expr>,
452        start: Box<Expr>,
453        count: Option<Box<Expr>>,
454    },
455    /// `TRIM([BOTH | LEADING | TRAILING] [<expr>] FROM <expr>)`\
456    /// Or\
457    /// `TRIM([BOTH | LEADING | TRAILING] [FROM] <expr> [, <expr>])`
458    Trim {
459        expr: Box<Expr>,
460        // ([BOTH | LEADING | TRAILING], <expr>)
461        trim_where: Option<TrimWhereField>,
462        trim_what: Option<Box<Expr>>,
463    },
464    /// `expr COLLATE collation`
465    Collate {
466        expr: Box<Expr>,
467        collation: ObjectName,
468    },
469    /// Nested expression e.g. `(foo > bar)` or `(1)`
470    Nested(Box<Expr>),
471    /// A literal value, such as string, number, date or NULL
472    Value(Value),
473    /// Parameter Symbol e.g. `$1`, `$1::int`
474    Parameter {
475        index: u64,
476    },
477    /// A constant of form `<data_type> 'value'`.
478    /// This can represent ANSI SQL `DATE`, `TIME`, and `TIMESTAMP` literals (such as `DATE
479    /// '2020-01-01'`), as well as constants of other types (a non-standard PostgreSQL extension).
480    TypedString {
481        data_type: DataType,
482        value: String,
483    },
484    /// Scalar function call e.g. `LEFT(foo, 5)`
485    Function(Function),
486    /// `CASE [<operand>] WHEN <condition> THEN <result> ... [ELSE <result>] END`
487    ///
488    /// Note we only recognize a complete single expression as `<condition>`,
489    /// not `< 0` nor `1, 2, 3` as allowed in a `<simple when clause>` per
490    /// <https://jakewheat.github.io/sql-overview/sql-2011-foundation-grammar.html#simple-when-clause>
491    Case {
492        operand: Option<Box<Expr>>,
493        conditions: Vec<Expr>,
494        results: Vec<Expr>,
495        else_result: Option<Box<Expr>>,
496    },
497    /// An exists expression `EXISTS(SELECT ...)`, used in expressions like
498    /// `WHERE EXISTS (SELECT ...)`.
499    Exists(Box<Query>),
500    /// A parenthesized subquery `(SELECT ...)`, used in expression like
501    /// `SELECT (subquery) AS x` or `WHERE (subquery) = x`
502    Subquery(Box<Query>),
503    /// The `GROUPING SETS` expr.
504    GroupingSets(Vec<Vec<Expr>>),
505    /// The `CUBE` expr.
506    Cube(Vec<Vec<Expr>>),
507    /// The `ROLLUP` expr.
508    Rollup(Vec<Vec<Expr>>),
509    /// The `ROW` expr. The `ROW` keyword can be omitted,
510    Row(Vec<Expr>),
511    /// An array constructor `ARRAY[[2,3,4],[5,6,7]]`
512    Array(Array),
513    /// An array constructing subquery `ARRAY(SELECT 2 UNION SELECT 3)`
514    ArraySubquery(Box<Query>),
515    /// A subscript expression `arr[1]` or `map['a']`
516    Index {
517        obj: Box<Expr>,
518        index: Box<Expr>,
519    },
520    /// A slice expression `arr[1:3]`
521    ArrayRangeIndex {
522        obj: Box<Expr>,
523        start: Option<Box<Expr>>,
524        end: Option<Box<Expr>>,
525    },
526    LambdaFunction {
527        args: Vec<Ident>,
528        body: Box<Expr>,
529    },
530    Map {
531        entries: Vec<(Expr, Expr)>,
532    },
533}
534
535impl fmt::Display for Expr {
536    #[expect(clippy::disallowed_methods, reason = "use zip_eq")]
537    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
538        match self {
539            Expr::Identifier(s) => write!(f, "{}", s),
540            Expr::CompoundIdentifier(s) => write!(f, "{}", display_separated(s, ".")),
541            Expr::FieldIdentifier(ast, s) => write!(f, "({}).{}", ast, display_separated(s, ".")),
542            Expr::IsNull(ast) => write!(f, "{} IS NULL", ast),
543            Expr::IsNotNull(ast) => write!(f, "{} IS NOT NULL", ast),
544            Expr::IsTrue(ast) => write!(f, "{} IS TRUE", ast),
545            Expr::IsNotTrue(ast) => write!(f, "{} IS NOT TRUE", ast),
546            Expr::IsFalse(ast) => write!(f, "{} IS FALSE", ast),
547            Expr::IsNotFalse(ast) => write!(f, "{} IS NOT FALSE", ast),
548            Expr::IsUnknown(ast) => write!(f, "{} IS UNKNOWN", ast),
549            Expr::IsNotUnknown(ast) => write!(f, "{} IS NOT UNKNOWN", ast),
550            Expr::IsJson {
551                expr,
552                negated,
553                item_type,
554                unique_keys,
555            } => write!(
556                f,
557                "{} IS {}JSON{}{}",
558                expr,
559                if *negated { "NOT " } else { "" },
560                item_type,
561                if *unique_keys {
562                    " WITH UNIQUE KEYS"
563                } else {
564                    ""
565                },
566            ),
567            Expr::InList {
568                expr,
569                list,
570                negated,
571            } => write!(
572                f,
573                "{} {}IN ({})",
574                expr,
575                if *negated { "NOT " } else { "" },
576                display_comma_separated(list)
577            ),
578            Expr::InSubquery {
579                expr,
580                subquery,
581                negated,
582            } => write!(
583                f,
584                "{} {}IN ({})",
585                expr,
586                if *negated { "NOT " } else { "" },
587                subquery
588            ),
589            Expr::Between {
590                expr,
591                negated,
592                low,
593                high,
594            } => write!(
595                f,
596                "{} {}BETWEEN {} AND {}",
597                expr,
598                if *negated { "NOT " } else { "" },
599                low,
600                high
601            ),
602            Expr::Like {
603                negated,
604                expr,
605                pattern,
606                escape_char,
607            } => match escape_char {
608                Some(ch) => write!(
609                    f,
610                    "{} {}LIKE {} ESCAPE '{}'",
611                    expr,
612                    if *negated { "NOT " } else { "" },
613                    pattern,
614                    ch
615                ),
616                _ => write!(
617                    f,
618                    "{} {}LIKE {}",
619                    expr,
620                    if *negated { "NOT " } else { "" },
621                    pattern
622                ),
623            },
624            Expr::ILike {
625                negated,
626                expr,
627                pattern,
628                escape_char,
629            } => match escape_char {
630                Some(ch) => write!(
631                    f,
632                    "{} {}ILIKE {} ESCAPE '{}'",
633                    expr,
634                    if *negated { "NOT " } else { "" },
635                    pattern,
636                    ch
637                ),
638                _ => write!(
639                    f,
640                    "{} {}ILIKE {}",
641                    expr,
642                    if *negated { "NOT " } else { "" },
643                    pattern
644                ),
645            },
646            Expr::SimilarTo {
647                negated,
648                expr,
649                pattern,
650                escape_char,
651            } => match escape_char {
652                Some(ch) => write!(
653                    f,
654                    "{} {}SIMILAR TO {} ESCAPE '{}'",
655                    expr,
656                    if *negated { "NOT " } else { "" },
657                    pattern,
658                    ch
659                ),
660                _ => write!(
661                    f,
662                    "{} {}SIMILAR TO {}",
663                    expr,
664                    if *negated { "NOT " } else { "" },
665                    pattern
666                ),
667            },
668            Expr::BinaryOp { left, op, right } => write!(f, "{} {} {}", left, op, right),
669            Expr::SomeOp(expr) => write!(f, "SOME({})", expr),
670            Expr::AllOp(expr) => write!(f, "ALL({})", expr),
671            Expr::UnaryOp { op, expr } => {
672                if op == &UnaryOperator::PGPostfixFactorial {
673                    write!(f, "{}{}", expr, op)
674                } else {
675                    write!(f, "{} {}", op, expr)
676                }
677            }
678            Expr::Cast { expr, data_type } => write!(f, "CAST({} AS {})", expr, data_type),
679            Expr::TryCast { expr, data_type } => write!(f, "TRY_CAST({} AS {})", expr, data_type),
680            Expr::AtTimeZone {
681                timestamp,
682                time_zone,
683            } => write!(f, "{} AT TIME ZONE {}", timestamp, time_zone),
684            Expr::Extract { field, expr } => write!(f, "EXTRACT({} FROM {})", field, expr),
685            Expr::Collate { expr, collation } => write!(f, "{} COLLATE {}", expr, collation),
686            Expr::Nested(ast) => write!(f, "({})", ast),
687            Expr::Value(v) => write!(f, "{}", v),
688            Expr::Parameter { index } => write!(f, "${}", index),
689            Expr::TypedString { data_type, value } => {
690                write!(f, "{}", data_type)?;
691                write!(f, " '{}'", &value::escape_single_quote_string(value))
692            }
693            Expr::Function(fun) => write!(f, "{}", fun),
694            Expr::Case {
695                operand,
696                conditions,
697                results,
698                else_result,
699            } => {
700                write!(f, "CASE")?;
701                if let Some(operand) = operand {
702                    write!(f, " {}", operand)?;
703                }
704                for (c, r) in conditions.iter().zip_eq(results) {
705                    write!(f, " WHEN {} THEN {}", c, r)?;
706                }
707
708                if let Some(else_result) = else_result {
709                    write!(f, " ELSE {}", else_result)?;
710                }
711                write!(f, " END")
712            }
713            Expr::Exists(s) => write!(f, "EXISTS ({})", s),
714            Expr::Subquery(s) => write!(f, "({})", s),
715            Expr::GroupingSets(sets) => {
716                write!(f, "GROUPING SETS (")?;
717                let mut sep = "";
718                for set in sets {
719                    write!(f, "{}", sep)?;
720                    sep = ", ";
721                    write!(f, "({})", display_comma_separated(set))?;
722                }
723                write!(f, ")")
724            }
725            Expr::Cube(sets) => {
726                write!(f, "CUBE (")?;
727                let mut sep = "";
728                for set in sets {
729                    write!(f, "{}", sep)?;
730                    sep = ", ";
731                    if set.len() == 1 {
732                        write!(f, "{}", set[0])?;
733                    } else {
734                        write!(f, "({})", display_comma_separated(set))?;
735                    }
736                }
737                write!(f, ")")
738            }
739            Expr::Rollup(sets) => {
740                write!(f, "ROLLUP (")?;
741                let mut sep = "";
742                for set in sets {
743                    write!(f, "{}", sep)?;
744                    sep = ", ";
745                    if set.len() == 1 {
746                        write!(f, "{}", set[0])?;
747                    } else {
748                        write!(f, "({})", display_comma_separated(set))?;
749                    }
750                }
751                write!(f, ")")
752            }
753            Expr::Substring {
754                expr,
755                substring_from,
756                substring_for,
757            } => {
758                write!(f, "SUBSTRING({}", expr)?;
759                if let Some(from_part) = substring_from {
760                    write!(f, " FROM {}", from_part)?;
761                }
762                if let Some(from_part) = substring_for {
763                    write!(f, " FOR {}", from_part)?;
764                }
765
766                write!(f, ")")
767            }
768            Expr::Position { substring, string } => {
769                write!(f, "POSITION({} IN {})", substring, string)
770            }
771            Expr::Overlay {
772                expr,
773                new_substring,
774                start,
775                count,
776            } => {
777                write!(f, "OVERLAY({}", expr)?;
778                write!(f, " PLACING {}", new_substring)?;
779                write!(f, " FROM {}", start)?;
780
781                if let Some(count_expr) = count {
782                    write!(f, " FOR {}", count_expr)?;
783                }
784
785                write!(f, ")")
786            }
787            Expr::IsDistinctFrom(a, b) => write!(f, "{} IS DISTINCT FROM {}", a, b),
788            Expr::IsNotDistinctFrom(a, b) => write!(f, "{} IS NOT DISTINCT FROM {}", a, b),
789            Expr::Trim {
790                expr,
791                trim_where,
792                trim_what,
793            } => {
794                write!(f, "TRIM(")?;
795                if let Some(ident) = trim_where {
796                    write!(f, "{} ", ident)?;
797                }
798                if let Some(trim_char) = trim_what {
799                    write!(f, "{} ", trim_char)?;
800                }
801                write!(f, "FROM {})", expr)
802            }
803            Expr::Row(exprs) => write!(
804                f,
805                "ROW({})",
806                exprs
807                    .iter()
808                    .map(|v| v.to_string())
809                    .collect::<Vec<String>>()
810                    .as_slice()
811                    .join(", ")
812            ),
813            Expr::Index { obj, index } => {
814                write!(f, "{}[{}]", obj, index)?;
815                Ok(())
816            }
817            Expr::ArrayRangeIndex { obj, start, end } => {
818                let start_str = match start {
819                    None => "".to_owned(),
820                    Some(start) => format!("{}", start),
821                };
822                let end_str = match end {
823                    None => "".to_owned(),
824                    Some(end) => format!("{}", end),
825                };
826                write!(f, "{}[{}:{}]", obj, start_str, end_str)?;
827                Ok(())
828            }
829            Expr::Array(exprs) => write!(f, "{}", exprs),
830            Expr::ArraySubquery(s) => write!(f, "ARRAY ({})", s),
831            Expr::LambdaFunction { args, body } => {
832                write!(
833                    f,
834                    "|{}| {}",
835                    args.iter().map(ToString::to_string).join(", "),
836                    body
837                )
838            }
839            Expr::Map { entries } => {
840                write!(
841                    f,
842                    "MAP {{{}}}",
843                    entries
844                        .iter()
845                        .map(|(k, v)| format!("{}: {}", k, v))
846                        .join(", ")
847                )
848            }
849        }
850    }
851}
852
853/// A window specification (i.e. `OVER (PARTITION BY .. ORDER BY .. etc.)`)
854#[derive(Debug, Clone, PartialEq, Eq, Hash)]
855#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
856pub struct WindowSpec {
857    pub partition_by: Vec<Expr>,
858    pub order_by: Vec<OrderByExpr>,
859    pub window_frame: Option<WindowFrame>,
860}
861
862impl fmt::Display for WindowSpec {
863    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
864        let mut delim = "";
865        if !self.partition_by.is_empty() {
866            delim = " ";
867            write!(
868                f,
869                "PARTITION BY {}",
870                display_comma_separated(&self.partition_by)
871            )?;
872        }
873        if !self.order_by.is_empty() {
874            f.write_str(delim)?;
875            delim = " ";
876            write!(f, "ORDER BY {}", display_comma_separated(&self.order_by))?;
877        }
878        if let Some(window_frame) = &self.window_frame {
879            f.write_str(delim)?;
880            window_frame.fmt(f)?;
881        }
882        Ok(())
883    }
884}
885
886/// Specifies the data processed by a window function, e.g.
887/// `RANGE UNBOUNDED PRECEDING` or `ROWS BETWEEN 5 PRECEDING AND CURRENT ROW`.
888///
889/// Note: The parser does not validate the specified bounds; the caller should
890/// reject invalid bounds like `ROWS UNBOUNDED FOLLOWING` before execution.
891#[derive(Debug, Clone, PartialEq, Eq, Hash)]
892#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
893pub struct WindowFrame {
894    pub units: WindowFrameUnits,
895    pub bounds: WindowFrameBounds,
896    pub exclusion: Option<WindowFrameExclusion>,
897}
898
899#[derive(Debug, Clone, PartialEq, Eq, Hash)]
900#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
901pub enum WindowFrameUnits {
902    Rows,
903    Range,
904    Groups,
905    Session,
906}
907
908#[derive(Debug, Clone, PartialEq, Eq, Hash)]
909#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
910pub enum WindowFrameBounds {
911    Bounds {
912        start: WindowFrameBound,
913        /// The right bound of the `BETWEEN .. AND` clause. The end bound of `None`
914        /// indicates the shorthand form (e.g. `ROWS 1 PRECEDING`), which must
915        /// behave the same as `end_bound = WindowFrameBound::CurrentRow`.
916        end: Option<WindowFrameBound>,
917    },
918    Gap(Box<Expr>),
919}
920
921impl fmt::Display for WindowFrame {
922    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
923        write!(f, "{} ", self.units)?;
924        match &self.bounds {
925            WindowFrameBounds::Bounds { start, end } => {
926                if let Some(end) = end {
927                    write!(f, "BETWEEN {} AND {}", start, end)
928                } else {
929                    write!(f, "{}", start)
930                }
931            }
932            WindowFrameBounds::Gap(gap) => {
933                write!(f, "WITH GAP {}", gap)
934            }
935        }
936    }
937}
938
939impl fmt::Display for WindowFrameUnits {
940    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
941        f.write_str(match self {
942            WindowFrameUnits::Rows => "ROWS",
943            WindowFrameUnits::Range => "RANGE",
944            WindowFrameUnits::Groups => "GROUPS",
945            WindowFrameUnits::Session => "SESSION",
946        })
947    }
948}
949
950/// Specifies [WindowFrame]'s `start_bound` and `end_bound`
951#[derive(Debug, Clone, PartialEq, Eq, Hash)]
952#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
953pub enum WindowFrameBound {
954    /// `CURRENT ROW`
955    CurrentRow,
956    /// `<offset> PRECEDING` or `UNBOUNDED PRECEDING`
957    Preceding(Option<Box<Expr>>),
958    /// `<offset> FOLLOWING` or `UNBOUNDED FOLLOWING`.
959    Following(Option<Box<Expr>>),
960}
961
962impl fmt::Display for WindowFrameBound {
963    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
964        match self {
965            WindowFrameBound::CurrentRow => f.write_str("CURRENT ROW"),
966            WindowFrameBound::Preceding(None) => f.write_str("UNBOUNDED PRECEDING"),
967            WindowFrameBound::Following(None) => f.write_str("UNBOUNDED FOLLOWING"),
968            WindowFrameBound::Preceding(Some(n)) => write!(f, "{} PRECEDING", n),
969            WindowFrameBound::Following(Some(n)) => write!(f, "{} FOLLOWING", n),
970        }
971    }
972}
973
974/// Frame exclusion option of [WindowFrame].
975#[derive(Debug, Copy, Clone, PartialEq, Eq, Hash)]
976#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
977pub enum WindowFrameExclusion {
978    CurrentRow,
979    Group,
980    Ties,
981    NoOthers,
982}
983
984impl fmt::Display for WindowFrameExclusion {
985    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
986        match self {
987            WindowFrameExclusion::CurrentRow => f.write_str("EXCLUDE CURRENT ROW"),
988            WindowFrameExclusion::Group => f.write_str("EXCLUDE GROUP"),
989            WindowFrameExclusion::Ties => f.write_str("EXCLUDE TIES"),
990            WindowFrameExclusion::NoOthers => f.write_str("EXCLUDE NO OTHERS"),
991        }
992    }
993}
994
995#[derive(Debug, Clone, PartialEq, Eq, Hash)]
996#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
997pub enum AddDropSync {
998    ADD,
999    DROP,
1000    SYNC,
1001}
1002
1003impl fmt::Display for AddDropSync {
1004    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1005        match self {
1006            AddDropSync::SYNC => f.write_str("SYNC PARTITIONS"),
1007            AddDropSync::DROP => f.write_str("DROP PARTITIONS"),
1008            AddDropSync::ADD => f.write_str("ADD PARTITIONS"),
1009        }
1010    }
1011}
1012
1013#[derive(Debug, Clone, PartialEq, Eq, Hash)]
1014#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
1015pub enum ShowObject {
1016    Table { schema: Option<Ident> },
1017    InternalTable { schema: Option<Ident> },
1018    Database,
1019    Schema,
1020    View { schema: Option<Ident> },
1021    MaterializedView { schema: Option<Ident> },
1022    Source { schema: Option<Ident> },
1023    Sink { schema: Option<Ident> },
1024    Subscription { schema: Option<Ident> },
1025    Columns { table: ObjectName },
1026    Connection { schema: Option<Ident> },
1027    Secret { schema: Option<Ident> },
1028    Function { schema: Option<Ident> },
1029    Indexes { table: ObjectName },
1030    Cluster,
1031    Jobs,
1032    ProcessList,
1033    Cursor,
1034    SubscriptionCursor,
1035}
1036
1037#[derive(Debug, Clone, PartialEq, Eq, Hash)]
1038#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
1039pub struct JobIdents(pub Vec<u32>);
1040
1041impl fmt::Display for ShowObject {
1042    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1043        fn fmt_schema(schema: &Option<Ident>) -> String {
1044            if let Some(schema) = schema {
1045                format!(" FROM {}", schema.value)
1046            } else {
1047                "".to_owned()
1048            }
1049        }
1050
1051        match self {
1052            ShowObject::Database => f.write_str("DATABASES"),
1053            ShowObject::Schema => f.write_str("SCHEMAS"),
1054            ShowObject::Table { schema } => {
1055                write!(f, "TABLES{}", fmt_schema(schema))
1056            }
1057            ShowObject::InternalTable { schema } => {
1058                write!(f, "INTERNAL TABLES{}", fmt_schema(schema))
1059            }
1060            ShowObject::View { schema } => {
1061                write!(f, "VIEWS{}", fmt_schema(schema))
1062            }
1063            ShowObject::MaterializedView { schema } => {
1064                write!(f, "MATERIALIZED VIEWS{}", fmt_schema(schema))
1065            }
1066            ShowObject::Source { schema } => write!(f, "SOURCES{}", fmt_schema(schema)),
1067            ShowObject::Sink { schema } => write!(f, "SINKS{}", fmt_schema(schema)),
1068            ShowObject::Columns { table } => write!(f, "COLUMNS FROM {}", table),
1069            ShowObject::Connection { schema } => write!(f, "CONNECTIONS{}", fmt_schema(schema)),
1070            ShowObject::Function { schema } => write!(f, "FUNCTIONS{}", fmt_schema(schema)),
1071            ShowObject::Indexes { table } => write!(f, "INDEXES FROM {}", table),
1072            ShowObject::Cluster => {
1073                write!(f, "CLUSTER")
1074            }
1075            ShowObject::Jobs => write!(f, "JOBS"),
1076            ShowObject::ProcessList => write!(f, "PROCESSLIST"),
1077            ShowObject::Subscription { schema } => write!(f, "SUBSCRIPTIONS{}", fmt_schema(schema)),
1078            ShowObject::Secret { schema } => write!(f, "SECRETS{}", fmt_schema(schema)),
1079            ShowObject::Cursor => write!(f, "CURSORS"),
1080            ShowObject::SubscriptionCursor => write!(f, "SUBSCRIPTION CURSORS"),
1081        }
1082    }
1083}
1084
1085#[derive(Debug, Clone, PartialEq, Eq, Hash)]
1086#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
1087pub enum ShowCreateType {
1088    Table,
1089    MaterializedView,
1090    View,
1091    Index,
1092    Source,
1093    Sink,
1094    Function,
1095    Subscription,
1096}
1097
1098impl fmt::Display for ShowCreateType {
1099    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1100        match self {
1101            ShowCreateType::Table => f.write_str("TABLE"),
1102            ShowCreateType::MaterializedView => f.write_str("MATERIALIZED VIEW"),
1103            ShowCreateType::View => f.write_str("VIEW"),
1104            ShowCreateType::Index => f.write_str("INDEX"),
1105            ShowCreateType::Source => f.write_str("SOURCE"),
1106            ShowCreateType::Sink => f.write_str("SINK"),
1107            ShowCreateType::Function => f.write_str("FUNCTION"),
1108            ShowCreateType::Subscription => f.write_str("SUBSCRIPTION"),
1109        }
1110    }
1111}
1112
1113#[derive(Debug, Clone, PartialEq, Eq, Hash)]
1114#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
1115pub enum CommentObject {
1116    Column,
1117    Table,
1118}
1119
1120impl fmt::Display for CommentObject {
1121    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1122        match self {
1123            CommentObject::Column => f.write_str("COLUMN"),
1124            CommentObject::Table => f.write_str("TABLE"),
1125        }
1126    }
1127}
1128
1129#[derive(Debug, Clone, PartialEq, Eq, Hash)]
1130#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
1131pub enum ExplainType {
1132    Logical,
1133    Physical,
1134    DistSql,
1135}
1136
1137impl fmt::Display for ExplainType {
1138    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1139        match self {
1140            ExplainType::Logical => f.write_str("Logical"),
1141            ExplainType::Physical => f.write_str("Physical"),
1142            ExplainType::DistSql => f.write_str("DistSQL"),
1143        }
1144    }
1145}
1146
1147#[derive(Debug, Clone, PartialEq, Eq, Hash)]
1148#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
1149pub enum ExplainFormat {
1150    Text,
1151    Json,
1152    Xml,
1153    Yaml,
1154    Dot,
1155}
1156
1157impl fmt::Display for ExplainFormat {
1158    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1159        match self {
1160            ExplainFormat::Text => f.write_str("TEXT"),
1161            ExplainFormat::Json => f.write_str("JSON"),
1162            ExplainFormat::Xml => f.write_str("XML"),
1163            ExplainFormat::Yaml => f.write_str("YAML"),
1164            ExplainFormat::Dot => f.write_str("DOT"),
1165        }
1166    }
1167}
1168
1169#[derive(Debug, Clone, PartialEq, Eq, Hash)]
1170#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
1171pub struct ExplainOptions {
1172    /// Display additional information regarding the plan.
1173    pub verbose: bool,
1174    // Trace plan transformation of the optimizer step by step
1175    pub trace: bool,
1176    // explain's plan type
1177    pub explain_type: ExplainType,
1178    // explain's plan format
1179    pub explain_format: ExplainFormat,
1180}
1181
1182impl Default for ExplainOptions {
1183    fn default() -> Self {
1184        Self {
1185            verbose: false,
1186            trace: false,
1187            explain_type: ExplainType::Physical,
1188            explain_format: ExplainFormat::Text,
1189        }
1190    }
1191}
1192
1193impl fmt::Display for ExplainOptions {
1194    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1195        let default = Self::default();
1196        if *self == default {
1197            Ok(())
1198        } else {
1199            let mut option_strs = vec![];
1200            if self.verbose {
1201                option_strs.push("VERBOSE".to_owned());
1202            }
1203            if self.trace {
1204                option_strs.push("TRACE".to_owned());
1205            }
1206            if self.explain_type == default.explain_type {
1207                option_strs.push(self.explain_type.to_string());
1208            }
1209            if self.explain_format == default.explain_format {
1210                option_strs.push(self.explain_format.to_string());
1211            }
1212            write!(f, "{}", option_strs.iter().format(","))
1213        }
1214    }
1215}
1216
1217#[derive(Debug, Clone, PartialEq, Eq, Hash)]
1218#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
1219pub struct CdcTableInfo {
1220    pub source_name: ObjectName,
1221    pub external_table_name: String,
1222}
1223
1224/// A top-level statement (SELECT, INSERT, CREATE, etc.)
1225#[allow(clippy::large_enum_variant)]
1226#[derive(Debug, Clone, PartialEq, Eq, Hash)]
1227#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
1228pub enum Statement {
1229    /// Analyze (Hive)
1230    Analyze {
1231        table_name: ObjectName,
1232    },
1233    /// Truncate (Hive)
1234    Truncate {
1235        table_name: ObjectName,
1236    },
1237    /// SELECT
1238    Query(Box<Query>),
1239    /// INSERT
1240    Insert {
1241        /// TABLE
1242        table_name: ObjectName,
1243        /// COLUMNS
1244        columns: Vec<Ident>,
1245        /// A SQL query that specifies what to insert
1246        source: Box<Query>,
1247        /// Define output of this insert statement
1248        returning: Vec<SelectItem>,
1249    },
1250    Copy {
1251        /// TABLE
1252        table_name: ObjectName,
1253        /// COLUMNS
1254        columns: Vec<Ident>,
1255        /// VALUES a vector of values to be copied
1256        values: Vec<Option<String>>,
1257    },
1258    /// UPDATE
1259    Update {
1260        /// TABLE
1261        table_name: ObjectName,
1262        /// Column assignments
1263        assignments: Vec<Assignment>,
1264        /// WHERE
1265        selection: Option<Expr>,
1266        /// RETURNING
1267        returning: Vec<SelectItem>,
1268    },
1269    /// DELETE
1270    Delete {
1271        /// FROM
1272        table_name: ObjectName,
1273        /// WHERE
1274        selection: Option<Expr>,
1275        /// RETURNING
1276        returning: Vec<SelectItem>,
1277    },
1278    /// DISCARD
1279    Discard(DiscardType),
1280    /// CREATE VIEW
1281    CreateView {
1282        or_replace: bool,
1283        materialized: bool,
1284        if_not_exists: bool,
1285        /// View name
1286        name: ObjectName,
1287        columns: Vec<Ident>,
1288        query: Box<Query>,
1289        emit_mode: Option<EmitMode>,
1290        with_options: Vec<SqlOption>,
1291    },
1292    /// CREATE TABLE
1293    CreateTable {
1294        or_replace: bool,
1295        temporary: bool,
1296        if_not_exists: bool,
1297        /// Table name
1298        name: ObjectName,
1299        /// Optional schema
1300        columns: Vec<ColumnDef>,
1301        // The wildchar position in columns defined in sql. Only exist when using external schema.
1302        wildcard_idx: Option<usize>,
1303        constraints: Vec<TableConstraint>,
1304        with_options: Vec<SqlOption>,
1305        /// `FORMAT ... ENCODE ...` for table with connector
1306        format_encode: Option<CompatibleFormatEncode>,
1307        /// The watermark defined on source.
1308        source_watermarks: Vec<SourceWatermark>,
1309        /// Append only table.
1310        append_only: bool,
1311        /// On conflict behavior
1312        on_conflict: Option<OnConflict>,
1313        /// with_version_column behind on conflict
1314        with_version_column: Option<Ident>,
1315        /// `AS ( query )`
1316        query: Option<Box<Query>>,
1317        /// `FROM cdc_source TABLE database_name.table_name`
1318        cdc_table_info: Option<CdcTableInfo>,
1319        /// `INCLUDE a AS b INCLUDE c`
1320        include_column_options: IncludeOption,
1321        /// `VALIDATE SECRET secure_secret_name AS secure_compare ()`
1322        webhook_info: Option<WebhookSourceInfo>,
1323        /// `Engine = [hummock | iceberg]`
1324        engine: Engine,
1325    },
1326    /// CREATE INDEX
1327    CreateIndex {
1328        /// index name
1329        name: ObjectName,
1330        table_name: ObjectName,
1331        columns: Vec<OrderByExpr>,
1332        include: Vec<Ident>,
1333        distributed_by: Vec<Expr>,
1334        unique: bool,
1335        if_not_exists: bool,
1336    },
1337    /// CREATE SOURCE
1338    CreateSource {
1339        stmt: CreateSourceStatement,
1340    },
1341    /// CREATE SINK
1342    CreateSink {
1343        stmt: CreateSinkStatement,
1344    },
1345    /// CREATE SUBSCRIPTION
1346    CreateSubscription {
1347        stmt: CreateSubscriptionStatement,
1348    },
1349    /// CREATE CONNECTION
1350    CreateConnection {
1351        stmt: CreateConnectionStatement,
1352    },
1353    CreateSecret {
1354        stmt: CreateSecretStatement,
1355    },
1356    /// CREATE FUNCTION
1357    ///
1358    /// Postgres: <https://www.postgresql.org/docs/15/sql-createfunction.html>
1359    CreateFunction {
1360        or_replace: bool,
1361        temporary: bool,
1362        if_not_exists: bool,
1363        name: ObjectName,
1364        args: Option<Vec<OperateFunctionArg>>,
1365        returns: Option<CreateFunctionReturns>,
1366        /// Optional parameters.
1367        params: CreateFunctionBody,
1368        with_options: CreateFunctionWithOptions, // FIXME(eric): use Option<>
1369    },
1370    /// CREATE AGGREGATE
1371    ///
1372    /// Postgres: <https://www.postgresql.org/docs/15/sql-createaggregate.html>
1373    CreateAggregate {
1374        or_replace: bool,
1375        if_not_exists: bool,
1376        name: ObjectName,
1377        args: Vec<OperateFunctionArg>,
1378        returns: DataType,
1379        /// Optional parameters.
1380        append_only: bool,
1381        params: CreateFunctionBody,
1382    },
1383
1384    /// DECLARE CURSOR
1385    DeclareCursor {
1386        stmt: DeclareCursorStatement,
1387    },
1388
1389    // FETCH CURSOR
1390    FetchCursor {
1391        stmt: FetchCursorStatement,
1392    },
1393
1394    // CLOSE CURSOR
1395    CloseCursor {
1396        stmt: CloseCursorStatement,
1397    },
1398
1399    /// ALTER DATABASE
1400    AlterDatabase {
1401        name: ObjectName,
1402        operation: AlterDatabaseOperation,
1403    },
1404    /// ALTER SCHEMA
1405    AlterSchema {
1406        name: ObjectName,
1407        operation: AlterSchemaOperation,
1408    },
1409    /// ALTER TABLE
1410    AlterTable {
1411        /// Table name
1412        name: ObjectName,
1413        operation: AlterTableOperation,
1414    },
1415    /// ALTER INDEX
1416    AlterIndex {
1417        /// Index name
1418        name: ObjectName,
1419        operation: AlterIndexOperation,
1420    },
1421    /// ALTER VIEW
1422    AlterView {
1423        /// View name
1424        name: ObjectName,
1425        materialized: bool,
1426        operation: AlterViewOperation,
1427    },
1428    /// ALTER SINK
1429    AlterSink {
1430        /// Sink name
1431        name: ObjectName,
1432        operation: AlterSinkOperation,
1433    },
1434    AlterSubscription {
1435        name: ObjectName,
1436        operation: AlterSubscriptionOperation,
1437    },
1438    /// ALTER SOURCE
1439    AlterSource {
1440        /// Source name
1441        name: ObjectName,
1442        operation: AlterSourceOperation,
1443    },
1444    /// ALTER FUNCTION
1445    AlterFunction {
1446        /// Function name
1447        name: ObjectName,
1448        args: Option<Vec<OperateFunctionArg>>,
1449        operation: AlterFunctionOperation,
1450    },
1451    /// ALTER CONNECTION
1452    AlterConnection {
1453        /// Connection name
1454        name: ObjectName,
1455        operation: AlterConnectionOperation,
1456    },
1457    /// ALTER SECRET
1458    AlterSecret {
1459        /// Secret name
1460        name: ObjectName,
1461        with_options: Vec<SqlOption>,
1462        operation: AlterSecretOperation,
1463    },
1464    /// ALTER FRAGMENT
1465    AlterFragment {
1466        fragment_id: u32,
1467        operation: AlterFragmentOperation,
1468    },
1469    /// DESCRIBE relation
1470    Describe {
1471        /// relation name
1472        name: ObjectName,
1473        kind: DescribeKind,
1474    },
1475    /// SHOW OBJECT COMMAND
1476    ShowObjects {
1477        object: ShowObject,
1478        filter: Option<ShowStatementFilter>,
1479    },
1480    /// SHOW CREATE COMMAND
1481    ShowCreateObject {
1482        /// Show create object type
1483        create_type: ShowCreateType,
1484        /// Show create object name
1485        name: ObjectName,
1486    },
1487    ShowTransactionIsolationLevel,
1488    /// CANCEL JOBS COMMAND
1489    CancelJobs(JobIdents),
1490    /// KILL COMMAND
1491    /// Kill process in the show processlist.
1492    Kill(i32),
1493    /// DROP
1494    Drop(DropStatement),
1495    /// DROP FUNCTION
1496    DropFunction {
1497        if_exists: bool,
1498        /// One or more function to drop
1499        func_desc: Vec<FunctionDesc>,
1500        /// `CASCADE` or `RESTRICT`
1501        option: Option<ReferentialAction>,
1502    },
1503    /// DROP AGGREGATE
1504    DropAggregate {
1505        if_exists: bool,
1506        /// One or more function to drop
1507        func_desc: Vec<FunctionDesc>,
1508        /// `CASCADE` or `RESTRICT`
1509        option: Option<ReferentialAction>,
1510    },
1511    /// `SET <variable>`
1512    ///
1513    /// Note: this is not a standard SQL statement, but it is supported by at
1514    /// least MySQL and PostgreSQL. Not all MySQL-specific syntactic forms are
1515    /// supported yet.
1516    SetVariable {
1517        local: bool,
1518        variable: Ident,
1519        value: SetVariableValue,
1520    },
1521    /// `SHOW <variable>`
1522    ///
1523    /// Note: this is a PostgreSQL-specific statement.
1524    ShowVariable {
1525        variable: Vec<Ident>,
1526    },
1527    /// `START TRANSACTION ...`
1528    StartTransaction {
1529        modes: Vec<TransactionMode>,
1530    },
1531    /// `BEGIN [ TRANSACTION | WORK ]`
1532    Begin {
1533        modes: Vec<TransactionMode>,
1534    },
1535    /// ABORT
1536    Abort,
1537    /// `SET TRANSACTION ...`
1538    SetTransaction {
1539        modes: Vec<TransactionMode>,
1540        snapshot: Option<Value>,
1541        session: bool,
1542    },
1543    /// `SET [ SESSION | LOCAL ] TIME ZONE { value | 'value' | LOCAL | DEFAULT }`
1544    SetTimeZone {
1545        local: bool,
1546        value: SetTimeZoneValue,
1547    },
1548    /// `COMMENT ON ...`
1549    ///
1550    /// Note: this is a PostgreSQL-specific statement.
1551    Comment {
1552        object_type: CommentObject,
1553        object_name: ObjectName,
1554        comment: Option<String>,
1555    },
1556    /// `COMMIT [ TRANSACTION | WORK ] [ AND [ NO ] CHAIN ]`
1557    Commit {
1558        chain: bool,
1559    },
1560    /// `ROLLBACK [ TRANSACTION | WORK ] [ AND [ NO ] CHAIN ]`
1561    Rollback {
1562        chain: bool,
1563    },
1564    /// CREATE SCHEMA
1565    CreateSchema {
1566        schema_name: ObjectName,
1567        if_not_exists: bool,
1568        owner: Option<ObjectName>,
1569    },
1570    /// CREATE DATABASE
1571    CreateDatabase {
1572        db_name: ObjectName,
1573        if_not_exists: bool,
1574        owner: Option<ObjectName>,
1575        resource_group: Option<SetVariableValue>,
1576    },
1577    /// GRANT privileges ON objects TO grantees
1578    Grant {
1579        privileges: Privileges,
1580        objects: GrantObjects,
1581        grantees: Vec<Ident>,
1582        with_grant_option: bool,
1583        granted_by: Option<Ident>,
1584    },
1585    /// REVOKE privileges ON objects FROM grantees
1586    Revoke {
1587        privileges: Privileges,
1588        objects: GrantObjects,
1589        grantees: Vec<Ident>,
1590        granted_by: Option<Ident>,
1591        revoke_grant_option: bool,
1592        cascade: bool,
1593    },
1594    /// `DEALLOCATE [ PREPARE ] { name | ALL }`
1595    ///
1596    /// Note: this is a PostgreSQL-specific statement.
1597    Deallocate {
1598        name: Ident,
1599        prepare: bool,
1600    },
1601    /// `EXECUTE name [ ( parameter [, ...] ) ]`
1602    ///
1603    /// Note: this is a PostgreSQL-specific statement.
1604    Execute {
1605        name: Ident,
1606        parameters: Vec<Expr>,
1607    },
1608    /// `PREPARE name [ ( data_type [, ...] ) ] AS statement`
1609    ///
1610    /// Note: this is a PostgreSQL-specific statement.
1611    Prepare {
1612        name: Ident,
1613        data_types: Vec<DataType>,
1614        statement: Box<Statement>,
1615    },
1616    /// EXPLAIN / DESCRIBE for select_statement
1617    Explain {
1618        /// Carry out the command and show actual run times and other statistics.
1619        analyze: bool,
1620        /// A SQL query that specifies what to explain
1621        statement: Box<Statement>,
1622        /// options of the explain statement
1623        options: ExplainOptions,
1624    },
1625    /// EXPLAIN ANALYZE for stream job
1626    /// We introduce a new statement rather than reuse `EXPLAIN` because
1627    /// the body of the statement is not an SQL query.
1628    /// TODO(kwannoel): Make profiling duration configurable: EXPLAIN ANALYZE (DURATION 1s) ...
1629    ExplainAnalyzeStreamJob {
1630        target: AnalyzeTarget,
1631        duration_secs: Option<u64>,
1632    },
1633    /// CREATE USER
1634    CreateUser(CreateUserStatement),
1635    /// ALTER USER
1636    AlterUser(AlterUserStatement),
1637    /// ALTER SYSTEM SET configuration_parameter { TO | = } { value | 'value' | DEFAULT }
1638    AlterSystem {
1639        param: Ident,
1640        value: SetVariableValue,
1641    },
1642    /// FLUSH the current barrier.
1643    ///
1644    /// Note: RisingWave specific statement.
1645    Flush,
1646    /// WAIT for ALL running stream jobs to finish.
1647    /// It will block the current session the condition is met.
1648    Wait,
1649    /// Trigger stream job recover
1650    Recover,
1651    /// `USE <db_name>`
1652    ///
1653    /// Note: this is a RisingWave specific statement and used to switch the current database.
1654    Use {
1655        db_name: ObjectName,
1656    },
1657}
1658
1659#[derive(Debug, Clone, PartialEq, Eq, Hash)]
1660#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
1661pub enum DescribeKind {
1662    /// `DESCRIBE <name>`
1663    Plain,
1664
1665    /// `DESCRIBE FRAGMENTS <name>`
1666    Fragments,
1667}
1668
1669impl fmt::Display for Statement {
1670    /// Converts(unparses) the statement to a SQL string.
1671    ///
1672    /// If the resulting SQL is not valid, this function will panic. Use
1673    /// [`Statement::try_to_string`] to get a `Result` instead.
1674    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
1675        // Note: we ignore formatting options here.
1676        let sql = self
1677            .try_to_string()
1678            .expect("normalized SQL should be parsable");
1679        f.write_str(&sql)
1680    }
1681}
1682
1683impl Statement {
1684    /// Converts(unparses) the statement to a SQL string.
1685    ///
1686    /// If the resulting SQL is not valid, returns an error.
1687    pub fn try_to_string(&self) -> Result<String, ParserError> {
1688        let sql = self.to_string_unchecked();
1689
1690        // TODO(#20713): expand this check to all statements
1691        if matches!(
1692            self,
1693            Statement::CreateTable { .. } | Statement::CreateSource { .. }
1694        ) {
1695            let _ = Parser::parse_sql(&sql)?;
1696        }
1697        Ok(sql)
1698    }
1699
1700    /// Converts(unparses) the statement to a SQL string.
1701    ///
1702    /// The result may not be valid SQL if there's an implementation bug in the `Display`
1703    /// trait of any AST node. To avoid this, always prefer [`Statement::try_to_string`]
1704    /// to get a `Result`, or `to_string` which panics if the SQL is invalid.
1705    pub fn to_string_unchecked(&self) -> String {
1706        let mut buf = String::new();
1707        self.fmt_unchecked(&mut buf).unwrap();
1708        buf
1709    }
1710
1711    // NOTE: This function should not check the validity of the unparsed SQL (and panic).
1712    //       Thus, do not directly format a statement with `write!` or `format!`. Recursively
1713    //       call `fmt_unchecked` on the inner statements instead.
1714    //
1715    // Clippy thinks this function is too complicated, but it is painful to
1716    // split up without extracting structs for each `Statement` variant.
1717    #[allow(clippy::cognitive_complexity)]
1718    fn fmt_unchecked(&self, mut f: impl std::fmt::Write) -> fmt::Result {
1719        match self {
1720            Statement::Explain {
1721                analyze,
1722                statement,
1723                options,
1724            } => {
1725                write!(f, "EXPLAIN ")?;
1726
1727                if *analyze {
1728                    write!(f, "ANALYZE ")?;
1729                }
1730                write!(f, "{}", options)?;
1731
1732                statement.fmt_unchecked(f)
1733            }
1734            Statement::ExplainAnalyzeStreamJob {
1735                target,
1736                duration_secs,
1737            } => {
1738                write!(f, "EXPLAIN ANALYZE {}", target)?;
1739                if let Some(duration_secs) = duration_secs {
1740                    write!(f, " (DURATION_SECS {})", duration_secs)?;
1741                }
1742                Ok(())
1743            }
1744            Statement::Query(s) => write!(f, "{}", s),
1745            Statement::Truncate { table_name } => {
1746                write!(f, "TRUNCATE TABLE {}", table_name)?;
1747                Ok(())
1748            }
1749            Statement::Analyze { table_name } => {
1750                write!(f, "ANALYZE TABLE {}", table_name)?;
1751                Ok(())
1752            }
1753            Statement::Describe { name, kind } => {
1754                write!(f, "DESCRIBE {}", name)?;
1755                match kind {
1756                    DescribeKind::Plain => {}
1757
1758                    DescribeKind::Fragments => {
1759                        write!(f, " FRAGMENTS")?;
1760                    }
1761                }
1762                Ok(())
1763            }
1764            Statement::ShowObjects {
1765                object: show_object,
1766                filter,
1767            } => {
1768                write!(f, "SHOW {}", show_object)?;
1769                if let Some(filter) = filter {
1770                    write!(f, " {}", filter)?;
1771                }
1772                Ok(())
1773            }
1774            Statement::ShowCreateObject {
1775                create_type: show_type,
1776                name,
1777            } => {
1778                write!(f, "SHOW CREATE {} {}", show_type, name)?;
1779                Ok(())
1780            }
1781            Statement::ShowTransactionIsolationLevel => {
1782                write!(f, "SHOW TRANSACTION ISOLATION LEVEL")?;
1783                Ok(())
1784            }
1785            Statement::Insert {
1786                table_name,
1787                columns,
1788                source,
1789                returning,
1790            } => {
1791                write!(f, "INSERT INTO {table_name} ", table_name = table_name,)?;
1792                if !columns.is_empty() {
1793                    write!(f, "({}) ", display_comma_separated(columns))?;
1794                }
1795                write!(f, "{}", source)?;
1796                if !returning.is_empty() {
1797                    write!(f, " RETURNING ({})", display_comma_separated(returning))?;
1798                }
1799                Ok(())
1800            }
1801            Statement::Copy {
1802                table_name,
1803                columns,
1804                values,
1805            } => {
1806                write!(f, "COPY {}", table_name)?;
1807                if !columns.is_empty() {
1808                    write!(f, " ({})", display_comma_separated(columns))?;
1809                }
1810                write!(f, " FROM stdin; ")?;
1811                if !values.is_empty() {
1812                    writeln!(f)?;
1813                    let mut delim = "";
1814                    for v in values {
1815                        write!(f, "{}", delim)?;
1816                        delim = "\t";
1817                        if let Some(v) = v {
1818                            write!(f, "{}", v)?;
1819                        } else {
1820                            write!(f, "\\N")?;
1821                        }
1822                    }
1823                }
1824                write!(f, "\n\\.")
1825            }
1826            Statement::Update {
1827                table_name,
1828                assignments,
1829                selection,
1830                returning,
1831            } => {
1832                write!(f, "UPDATE {}", table_name)?;
1833                if !assignments.is_empty() {
1834                    write!(f, " SET {}", display_comma_separated(assignments))?;
1835                }
1836                if let Some(selection) = selection {
1837                    write!(f, " WHERE {}", selection)?;
1838                }
1839                if !returning.is_empty() {
1840                    write!(f, " RETURNING ({})", display_comma_separated(returning))?;
1841                }
1842                Ok(())
1843            }
1844            Statement::Delete {
1845                table_name,
1846                selection,
1847                returning,
1848            } => {
1849                write!(f, "DELETE FROM {}", table_name)?;
1850                if let Some(selection) = selection {
1851                    write!(f, " WHERE {}", selection)?;
1852                }
1853                if !returning.is_empty() {
1854                    write!(f, " RETURNING {}", display_comma_separated(returning))?;
1855                }
1856                Ok(())
1857            }
1858            Statement::CreateDatabase {
1859                db_name,
1860                if_not_exists,
1861                owner,
1862                resource_group,
1863            } => {
1864                write!(f, "CREATE DATABASE")?;
1865                if *if_not_exists {
1866                    write!(f, " IF NOT EXISTS")?;
1867                }
1868                write!(f, " {}", db_name)?;
1869                if let Some(owner) = owner {
1870                    write!(f, " WITH OWNER = {}", owner)?;
1871                }
1872                if let Some(resource_group) = resource_group {
1873                    write!(f, " RESOURCE_GROUP = {}", resource_group)?;
1874                }
1875
1876                Ok(())
1877            }
1878            Statement::CreateFunction {
1879                or_replace,
1880                temporary,
1881                if_not_exists,
1882                name,
1883                args,
1884                returns,
1885                params,
1886                with_options,
1887            } => {
1888                write!(
1889                    f,
1890                    "CREATE {or_replace}{temp}FUNCTION {if_not_exists}{name}",
1891                    temp = if *temporary { "TEMPORARY " } else { "" },
1892                    or_replace = if *or_replace { "OR REPLACE " } else { "" },
1893                    if_not_exists = if *if_not_exists { "IF NOT EXISTS " } else { "" },
1894                )?;
1895                if let Some(args) = args {
1896                    write!(f, "({})", display_comma_separated(args))?;
1897                }
1898                if let Some(return_type) = returns {
1899                    write!(f, " {}", return_type)?;
1900                }
1901                write!(f, "{params}")?;
1902                write!(f, "{with_options}")?;
1903                Ok(())
1904            }
1905            Statement::CreateAggregate {
1906                or_replace,
1907                if_not_exists,
1908                name,
1909                args,
1910                returns,
1911                append_only,
1912                params,
1913            } => {
1914                write!(
1915                    f,
1916                    "CREATE {or_replace}AGGREGATE {if_not_exists}{name}",
1917                    or_replace = if *or_replace { "OR REPLACE " } else { "" },
1918                    if_not_exists = if *if_not_exists { "IF NOT EXISTS " } else { "" },
1919                )?;
1920                write!(f, "({})", display_comma_separated(args))?;
1921                write!(f, " RETURNS {}", returns)?;
1922                if *append_only {
1923                    write!(f, " APPEND ONLY")?;
1924                }
1925                write!(f, "{params}")?;
1926                Ok(())
1927            }
1928            Statement::CreateView {
1929                name,
1930                or_replace,
1931                if_not_exists,
1932                columns,
1933                query,
1934                materialized,
1935                with_options,
1936                emit_mode,
1937            } => {
1938                write!(
1939                    f,
1940                    "CREATE {or_replace}{materialized}VIEW {if_not_exists}{name}",
1941                    or_replace = if *or_replace { "OR REPLACE " } else { "" },
1942                    materialized = if *materialized { "MATERIALIZED " } else { "" },
1943                    if_not_exists = if *if_not_exists { "IF NOT EXISTS " } else { "" },
1944                    name = name
1945                )?;
1946                if !with_options.is_empty() {
1947                    write!(f, " WITH ({})", display_comma_separated(with_options))?;
1948                }
1949                if !columns.is_empty() {
1950                    write!(f, " ({})", display_comma_separated(columns))?;
1951                }
1952                write!(f, " AS {}", query)?;
1953                if let Some(emit_mode) = emit_mode {
1954                    write!(f, " EMIT {}", emit_mode)?;
1955                }
1956                Ok(())
1957            }
1958            Statement::CreateTable {
1959                name,
1960                columns,
1961                wildcard_idx,
1962                constraints,
1963                with_options,
1964                or_replace,
1965                if_not_exists,
1966                temporary,
1967                format_encode,
1968                source_watermarks,
1969                append_only,
1970                on_conflict,
1971                with_version_column,
1972                query,
1973                cdc_table_info,
1974                include_column_options,
1975                webhook_info,
1976                engine,
1977            } => {
1978                // We want to allow the following options
1979                // Empty column list, allowed by PostgreSQL:
1980                //   `CREATE TABLE t ()`
1981                // No columns provided for CREATE TABLE AS:
1982                //   `CREATE TABLE t AS SELECT a from t2`
1983                // Columns provided for CREATE TABLE AS:
1984                //   `CREATE TABLE t (a INT) AS SELECT a from t2`
1985                write!(
1986                    f,
1987                    "CREATE {or_replace}{temporary}TABLE {if_not_exists}{name}",
1988                    or_replace = if *or_replace { "OR REPLACE " } else { "" },
1989                    if_not_exists = if *if_not_exists { "IF NOT EXISTS " } else { "" },
1990                    temporary = if *temporary { "TEMPORARY " } else { "" },
1991                    name = name,
1992                )?;
1993                if !columns.is_empty() || !constraints.is_empty() {
1994                    write!(
1995                        f,
1996                        " {}",
1997                        fmt_create_items(columns, constraints, source_watermarks, *wildcard_idx)?
1998                    )?;
1999                } else if query.is_none() {
2000                    // PostgreSQL allows `CREATE TABLE t ();`, but requires empty parens
2001                    write!(f, " ()")?;
2002                }
2003                if *append_only {
2004                    write!(f, " APPEND ONLY")?;
2005                }
2006
2007                if let Some(on_conflict_behavior) = on_conflict {
2008                    write!(f, " ON CONFLICT {}", on_conflict_behavior)?;
2009                }
2010                if let Some(version_column) = with_version_column {
2011                    write!(f, " WITH VERSION COLUMN({})", version_column)?;
2012                }
2013                if !include_column_options.is_empty() {
2014                    write!(f, " {}", display_separated(include_column_options, " "))?;
2015                }
2016                if !with_options.is_empty() {
2017                    write!(f, " WITH ({})", display_comma_separated(with_options))?;
2018                }
2019                if let Some(format_encode) = format_encode {
2020                    write!(f, " {}", format_encode)?;
2021                }
2022                if let Some(query) = query {
2023                    write!(f, " AS {}", query)?;
2024                }
2025                if let Some(info) = cdc_table_info {
2026                    write!(f, " FROM {}", info.source_name)?;
2027                    write!(f, " TABLE '{}'", info.external_table_name)?;
2028                }
2029                if let Some(info) = webhook_info {
2030                    if let Some(secret) = &info.secret_ref {
2031                        write!(f, " VALIDATE SECRET {}", secret.secret_name)?;
2032                    } else {
2033                        write!(f, " VALIDATE")?;
2034                    }
2035                    write!(f, " AS {}", info.signature_expr)?;
2036                }
2037                match engine {
2038                    Engine::Hummock => {}
2039                    Engine::Iceberg => {
2040                        write!(f, " ENGINE = {}", engine)?;
2041                    }
2042                }
2043                Ok(())
2044            }
2045            Statement::CreateIndex {
2046                name,
2047                table_name,
2048                columns,
2049                include,
2050                distributed_by,
2051                unique,
2052                if_not_exists,
2053            } => write!(
2054                f,
2055                "CREATE {unique}INDEX {if_not_exists}{name} ON {table_name}({columns}){include}{distributed_by}",
2056                unique = if *unique { "UNIQUE " } else { "" },
2057                if_not_exists = if *if_not_exists { "IF NOT EXISTS " } else { "" },
2058                name = name,
2059                table_name = table_name,
2060                columns = display_comma_separated(columns),
2061                include = if include.is_empty() {
2062                    "".to_owned()
2063                } else {
2064                    format!(" INCLUDE({})", display_separated(include, ","))
2065                },
2066                distributed_by = if distributed_by.is_empty() {
2067                    "".to_owned()
2068                } else {
2069                    format!(
2070                        " DISTRIBUTED BY({})",
2071                        display_separated(distributed_by, ",")
2072                    )
2073                }
2074            ),
2075            Statement::CreateSource { stmt } => write!(f, "CREATE SOURCE {}", stmt,),
2076            Statement::CreateSink { stmt } => write!(f, "CREATE SINK {}", stmt,),
2077            Statement::CreateSubscription { stmt } => write!(f, "CREATE SUBSCRIPTION {}", stmt,),
2078            Statement::CreateConnection { stmt } => write!(f, "CREATE CONNECTION {}", stmt,),
2079            Statement::DeclareCursor { stmt } => write!(f, "DECLARE {}", stmt,),
2080            Statement::FetchCursor { stmt } => write!(f, "FETCH {}", stmt),
2081            Statement::CloseCursor { stmt } => write!(f, "CLOSE {}", stmt),
2082            Statement::CreateSecret { stmt } => write!(f, "CREATE SECRET {}", stmt),
2083            Statement::AlterDatabase { name, operation } => {
2084                write!(f, "ALTER DATABASE {} {}", name, operation)
2085            }
2086            Statement::AlterSchema { name, operation } => {
2087                write!(f, "ALTER SCHEMA {} {}", name, operation)
2088            }
2089            Statement::AlterTable { name, operation } => {
2090                write!(f, "ALTER TABLE {} {}", name, operation)
2091            }
2092            Statement::AlterIndex { name, operation } => {
2093                write!(f, "ALTER INDEX {} {}", name, operation)
2094            }
2095            Statement::AlterView {
2096                materialized,
2097                name,
2098                operation,
2099            } => {
2100                write!(
2101                    f,
2102                    "ALTER {}VIEW {} {}",
2103                    if *materialized { "MATERIALIZED " } else { "" },
2104                    name,
2105                    operation
2106                )
2107            }
2108            Statement::AlterSink { name, operation } => {
2109                write!(f, "ALTER SINK {} {}", name, operation)
2110            }
2111            Statement::AlterSubscription { name, operation } => {
2112                write!(f, "ALTER SUBSCRIPTION {} {}", name, operation)
2113            }
2114            Statement::AlterSource { name, operation } => {
2115                write!(f, "ALTER SOURCE {} {}", name, operation)
2116            }
2117            Statement::AlterFunction {
2118                name,
2119                args,
2120                operation,
2121            } => {
2122                write!(f, "ALTER FUNCTION {}", name)?;
2123                if let Some(args) = args {
2124                    write!(f, "({})", display_comma_separated(args))?;
2125                }
2126                write!(f, " {}", operation)
2127            }
2128            Statement::AlterConnection { name, operation } => {
2129                write!(f, "ALTER CONNECTION {} {}", name, operation)
2130            }
2131            Statement::AlterSecret {
2132                name,
2133                with_options,
2134                operation,
2135            } => {
2136                write!(f, "ALTER SECRET {}", name)?;
2137                if !with_options.is_empty() {
2138                    write!(f, " WITH ({})", display_comma_separated(with_options))?;
2139                }
2140                write!(f, " {}", operation)
2141            }
2142            Statement::Discard(t) => write!(f, "DISCARD {}", t),
2143            Statement::Drop(stmt) => write!(f, "DROP {}", stmt),
2144            Statement::DropFunction {
2145                if_exists,
2146                func_desc,
2147                option,
2148            } => {
2149                write!(
2150                    f,
2151                    "DROP FUNCTION{} {}",
2152                    if *if_exists { " IF EXISTS" } else { "" },
2153                    display_comma_separated(func_desc),
2154                )?;
2155                if let Some(op) = option {
2156                    write!(f, " {}", op)?;
2157                }
2158                Ok(())
2159            }
2160            Statement::DropAggregate {
2161                if_exists,
2162                func_desc,
2163                option,
2164            } => {
2165                write!(
2166                    f,
2167                    "DROP AGGREGATE{} {}",
2168                    if *if_exists { " IF EXISTS" } else { "" },
2169                    display_comma_separated(func_desc),
2170                )?;
2171                if let Some(op) = option {
2172                    write!(f, " {}", op)?;
2173                }
2174                Ok(())
2175            }
2176            Statement::SetVariable {
2177                local,
2178                variable,
2179                value,
2180            } => {
2181                f.write_str("SET ")?;
2182                if *local {
2183                    f.write_str("LOCAL ")?;
2184                }
2185                write!(f, "{name} = {value}", name = variable,)
2186            }
2187            Statement::ShowVariable { variable } => {
2188                write!(f, "SHOW")?;
2189                if !variable.is_empty() {
2190                    write!(f, " {}", display_separated(variable, " "))?;
2191                }
2192                Ok(())
2193            }
2194            Statement::StartTransaction { modes } => {
2195                write!(f, "START TRANSACTION")?;
2196                if !modes.is_empty() {
2197                    write!(f, " {}", display_comma_separated(modes))?;
2198                }
2199                Ok(())
2200            }
2201            Statement::Abort => {
2202                write!(f, "ABORT")?;
2203                Ok(())
2204            }
2205            Statement::SetTransaction {
2206                modes,
2207                snapshot,
2208                session,
2209            } => {
2210                if *session {
2211                    write!(f, "SET SESSION CHARACTERISTICS AS TRANSACTION")?;
2212                } else {
2213                    write!(f, "SET TRANSACTION")?;
2214                }
2215                if !modes.is_empty() {
2216                    write!(f, " {}", display_comma_separated(modes))?;
2217                }
2218                if let Some(snapshot_id) = snapshot {
2219                    write!(f, " SNAPSHOT {}", snapshot_id)?;
2220                }
2221                Ok(())
2222            }
2223            Statement::SetTimeZone { local, value } => {
2224                write!(f, "SET")?;
2225                if *local {
2226                    write!(f, " LOCAL")?;
2227                }
2228                write!(f, " TIME ZONE {}", value)?;
2229                Ok(())
2230            }
2231            Statement::Commit { chain } => {
2232                write!(f, "COMMIT{}", if *chain { " AND CHAIN" } else { "" },)
2233            }
2234            Statement::Rollback { chain } => {
2235                write!(f, "ROLLBACK{}", if *chain { " AND CHAIN" } else { "" },)
2236            }
2237            Statement::CreateSchema {
2238                schema_name,
2239                if_not_exists,
2240                owner,
2241            } => {
2242                write!(
2243                    f,
2244                    "CREATE SCHEMA {if_not_exists}{name}",
2245                    if_not_exists = if *if_not_exists { "IF NOT EXISTS " } else { "" },
2246                    name = schema_name
2247                )?;
2248                if let Some(user) = owner {
2249                    write!(f, " AUTHORIZATION {}", user)?;
2250                }
2251                Ok(())
2252            }
2253            Statement::Grant {
2254                privileges,
2255                objects,
2256                grantees,
2257                with_grant_option,
2258                granted_by,
2259            } => {
2260                write!(f, "GRANT {} ", privileges)?;
2261                write!(f, "ON {} ", objects)?;
2262                write!(f, "TO {}", display_comma_separated(grantees))?;
2263                if *with_grant_option {
2264                    write!(f, " WITH GRANT OPTION")?;
2265                }
2266                if let Some(grantor) = granted_by {
2267                    write!(f, " GRANTED BY {}", grantor)?;
2268                }
2269                Ok(())
2270            }
2271            Statement::Revoke {
2272                privileges,
2273                objects,
2274                grantees,
2275                granted_by,
2276                revoke_grant_option,
2277                cascade,
2278            } => {
2279                write!(
2280                    f,
2281                    "REVOKE {}{} ",
2282                    if *revoke_grant_option {
2283                        "GRANT OPTION FOR "
2284                    } else {
2285                        ""
2286                    },
2287                    privileges
2288                )?;
2289                write!(f, "ON {} ", objects)?;
2290                write!(f, "FROM {}", display_comma_separated(grantees))?;
2291                if let Some(grantor) = granted_by {
2292                    write!(f, " GRANTED BY {}", grantor)?;
2293                }
2294                write!(f, " {}", if *cascade { "CASCADE" } else { "RESTRICT" })?;
2295                Ok(())
2296            }
2297            Statement::Deallocate { name, prepare } => write!(
2298                f,
2299                "DEALLOCATE {prepare}{name}",
2300                prepare = if *prepare { "PREPARE " } else { "" },
2301                name = name,
2302            ),
2303            Statement::Execute { name, parameters } => {
2304                write!(f, "EXECUTE {}", name)?;
2305                if !parameters.is_empty() {
2306                    write!(f, "({})", display_comma_separated(parameters))?;
2307                }
2308                Ok(())
2309            }
2310            Statement::Prepare {
2311                name,
2312                data_types,
2313                statement,
2314            } => {
2315                write!(f, "PREPARE {} ", name)?;
2316                if !data_types.is_empty() {
2317                    write!(f, "({}) ", display_comma_separated(data_types))?;
2318                }
2319                write!(f, "AS ")?;
2320                statement.fmt_unchecked(f)
2321            }
2322            Statement::Comment {
2323                object_type,
2324                object_name,
2325                comment,
2326            } => {
2327                write!(f, "COMMENT ON {} {} IS ", object_type, object_name)?;
2328                if let Some(c) = comment {
2329                    write!(f, "'{}'", c)
2330                } else {
2331                    write!(f, "NULL")
2332                }
2333            }
2334            Statement::CreateUser(statement) => {
2335                write!(f, "CREATE USER {}", statement)
2336            }
2337            Statement::AlterUser(statement) => {
2338                write!(f, "ALTER USER {}", statement)
2339            }
2340            Statement::AlterSystem { param, value } => {
2341                f.write_str("ALTER SYSTEM SET ")?;
2342                write!(f, "{param} = {value}",)
2343            }
2344            Statement::Flush => {
2345                write!(f, "FLUSH")
2346            }
2347            Statement::Wait => {
2348                write!(f, "WAIT")
2349            }
2350            Statement::Begin { modes } => {
2351                write!(f, "BEGIN")?;
2352                if !modes.is_empty() {
2353                    write!(f, " {}", display_comma_separated(modes))?;
2354                }
2355                Ok(())
2356            }
2357            Statement::CancelJobs(jobs) => {
2358                write!(f, "CANCEL JOBS {}", display_comma_separated(&jobs.0))?;
2359                Ok(())
2360            }
2361            Statement::Kill(process_id) => {
2362                write!(f, "KILL {}", process_id)?;
2363                Ok(())
2364            }
2365            Statement::Recover => {
2366                write!(f, "RECOVER")?;
2367                Ok(())
2368            }
2369            Statement::Use { db_name } => {
2370                write!(f, "USE {}", db_name)?;
2371                Ok(())
2372            }
2373            Statement::AlterFragment {
2374                fragment_id,
2375                operation,
2376            } => {
2377                write!(f, "ALTER FRAGMENT {} {}", fragment_id, operation)
2378            }
2379        }
2380    }
2381
2382    pub fn is_create(&self) -> bool {
2383        matches!(
2384            self,
2385            Statement::CreateTable { .. }
2386                | Statement::CreateView { .. }
2387                | Statement::CreateSource { .. }
2388                | Statement::CreateSink { .. }
2389                | Statement::CreateSubscription { .. }
2390                | Statement::CreateConnection { .. }
2391                | Statement::CreateSecret { .. }
2392                | Statement::CreateUser { .. }
2393                | Statement::CreateDatabase { .. }
2394                | Statement::CreateFunction { .. }
2395                | Statement::CreateAggregate { .. }
2396                | Statement::CreateIndex { .. }
2397                | Statement::CreateSchema { .. }
2398        )
2399    }
2400}
2401
2402impl Display for IncludeOptionItem {
2403    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
2404        let Self {
2405            column_type,
2406            inner_field,
2407            header_inner_expect_type,
2408            column_alias,
2409        } = self;
2410        write!(f, "INCLUDE {}", column_type)?;
2411        if let Some(inner_field) = inner_field {
2412            write!(f, " '{}'", value::escape_single_quote_string(inner_field))?;
2413            if let Some(expected_type) = header_inner_expect_type {
2414                write!(f, " {}", expected_type)?;
2415            }
2416        }
2417        if let Some(alias) = column_alias {
2418            write!(f, " AS {}", alias)?;
2419        }
2420        Ok(())
2421    }
2422}
2423
2424#[derive(Debug, Clone, PartialEq, Eq, Hash)]
2425#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
2426#[non_exhaustive]
2427pub enum OnInsert {
2428    /// ON DUPLICATE KEY UPDATE (MySQL when the key already exists, then execute an update instead)
2429    DuplicateKeyUpdate(Vec<Assignment>),
2430}
2431
2432impl fmt::Display for OnInsert {
2433    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
2434        match self {
2435            Self::DuplicateKeyUpdate(expr) => write!(
2436                f,
2437                " ON DUPLICATE KEY UPDATE {}",
2438                display_comma_separated(expr)
2439            ),
2440        }
2441    }
2442}
2443
2444/// Privileges granted in a GRANT statement or revoked in a REVOKE statement.
2445#[derive(Debug, Clone, PartialEq, Eq, Hash)]
2446#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
2447pub enum Privileges {
2448    /// All privileges applicable to the object type
2449    All {
2450        /// Optional keyword from the spec, ignored in practice
2451        with_privileges_keyword: bool,
2452    },
2453    /// Specific privileges (e.g. `SELECT`, `INSERT`)
2454    Actions(Vec<Action>),
2455}
2456
2457impl fmt::Display for Privileges {
2458    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
2459        match self {
2460            Privileges::All {
2461                with_privileges_keyword,
2462            } => {
2463                write!(
2464                    f,
2465                    "ALL{}",
2466                    if *with_privileges_keyword {
2467                        " PRIVILEGES"
2468                    } else {
2469                        ""
2470                    }
2471                )
2472            }
2473            Privileges::Actions(actions) => {
2474                write!(f, "{}", display_comma_separated(actions))
2475            }
2476        }
2477    }
2478}
2479
2480/// A privilege on a database object (table, sequence, etc.).
2481#[derive(Debug, Clone, PartialEq, Eq, Hash)]
2482#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
2483pub enum Action {
2484    Connect,
2485    Create,
2486    Delete,
2487    Execute,
2488    Insert { columns: Option<Vec<Ident>> },
2489    References { columns: Option<Vec<Ident>> },
2490    Select { columns: Option<Vec<Ident>> },
2491    Temporary,
2492    Trigger,
2493    Truncate,
2494    Update { columns: Option<Vec<Ident>> },
2495    Usage,
2496}
2497
2498impl fmt::Display for Action {
2499    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
2500        match self {
2501            Action::Connect => f.write_str("CONNECT")?,
2502            Action::Create => f.write_str("CREATE")?,
2503            Action::Delete => f.write_str("DELETE")?,
2504            Action::Execute => f.write_str("EXECUTE")?,
2505            Action::Insert { .. } => f.write_str("INSERT")?,
2506            Action::References { .. } => f.write_str("REFERENCES")?,
2507            Action::Select { .. } => f.write_str("SELECT")?,
2508            Action::Temporary => f.write_str("TEMPORARY")?,
2509            Action::Trigger => f.write_str("TRIGGER")?,
2510            Action::Truncate => f.write_str("TRUNCATE")?,
2511            Action::Update { .. } => f.write_str("UPDATE")?,
2512            Action::Usage => f.write_str("USAGE")?,
2513        };
2514        match self {
2515            Action::Insert { columns }
2516            | Action::References { columns }
2517            | Action::Select { columns }
2518            | Action::Update { columns } => {
2519                if let Some(columns) = columns {
2520                    write!(f, " ({})", display_comma_separated(columns))?;
2521                }
2522            }
2523            _ => (),
2524        };
2525        Ok(())
2526    }
2527}
2528
2529/// Objects on which privileges are granted in a GRANT statement.
2530#[derive(Debug, Clone, PartialEq, Eq, Hash)]
2531#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
2532pub enum GrantObjects {
2533    /// Grant privileges on `ALL SEQUENCES IN SCHEMA <schema_name> [, ...]`
2534    AllSequencesInSchema { schemas: Vec<ObjectName> },
2535    /// Grant privileges on `ALL TABLES IN SCHEMA <schema_name> [, ...]`
2536    AllTablesInSchema { schemas: Vec<ObjectName> },
2537    /// Grant privileges on `ALL SOURCES IN SCHEMA <schema_name> [, ...]`
2538    AllSourcesInSchema { schemas: Vec<ObjectName> },
2539    /// Grant privileges on `ALL SINKS IN SCHEMA <schema_name> [, ...]`
2540    AllSinksInSchema { schemas: Vec<ObjectName> },
2541    /// Grant privileges on `ALL MATERIALIZED VIEWS IN SCHEMA <schema_name> [, ...]`
2542    AllMviewsInSchema { schemas: Vec<ObjectName> },
2543    /// Grant privileges on `ALL VIEWS IN SCHEMA <schema_name> [, ...]`
2544    AllViewsInSchema { schemas: Vec<ObjectName> },
2545    /// Grant privileges on `ALL FUNCTIONS IN SCHEMA <schema_name> [, ...]`
2546    AllFunctionsInSchema { schemas: Vec<ObjectName> },
2547    /// Grant privileges on `ALL SECRETS IN SCHEMA <schema_name> [, ...]`
2548    AllSecretsInSchema { schemas: Vec<ObjectName> },
2549    /// Grant privileges on `ALL SUBSCRIPTIONS IN SCHEMA <schema_name> [, ...]`
2550    AllSubscriptionsInSchema { schemas: Vec<ObjectName> },
2551    /// Grant privileges on `ALL CONNECTIONS IN SCHEMA <schema_name> [, ...]`
2552    AllConnectionsInSchema { schemas: Vec<ObjectName> },
2553    /// Grant privileges on specific databases
2554    Databases(Vec<ObjectName>),
2555    /// Grant privileges on specific schemas
2556    Schemas(Vec<ObjectName>),
2557    /// Grant privileges on specific sources
2558    Sources(Vec<ObjectName>),
2559    /// Grant privileges on specific materialized views
2560    Mviews(Vec<ObjectName>),
2561    /// Grant privileges on specific sequences
2562    Sequences(Vec<ObjectName>),
2563    /// Grant privileges on specific tables
2564    Tables(Vec<ObjectName>),
2565    /// Grant privileges on specific sinks
2566    Sinks(Vec<ObjectName>),
2567    /// Grant privileges on specific views
2568    Views(Vec<ObjectName>),
2569    /// Grant privileges on specific connections
2570    Connections(Vec<ObjectName>),
2571    /// Grant privileges on specific subscriptions
2572    Subscriptions(Vec<ObjectName>),
2573    /// Grant privileges on specific functions
2574    Functions(Vec<FunctionDesc>),
2575    /// Grant privileges on specific secrets
2576    Secrets(Vec<ObjectName>),
2577}
2578
2579impl fmt::Display for GrantObjects {
2580    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
2581        match self {
2582            GrantObjects::Sequences(sequences) => {
2583                write!(f, "SEQUENCE {}", display_comma_separated(sequences))
2584            }
2585            GrantObjects::Schemas(schemas) => {
2586                write!(f, "SCHEMA {}", display_comma_separated(schemas))
2587            }
2588            GrantObjects::Tables(tables) => {
2589                write!(f, "{}", display_comma_separated(tables))
2590            }
2591            GrantObjects::AllSequencesInSchema { schemas } => {
2592                write!(
2593                    f,
2594                    "ALL SEQUENCES IN SCHEMA {}",
2595                    display_comma_separated(schemas)
2596                )
2597            }
2598            GrantObjects::AllTablesInSchema { schemas } => {
2599                write!(
2600                    f,
2601                    "ALL TABLES IN SCHEMA {}",
2602                    display_comma_separated(schemas)
2603                )
2604            }
2605            GrantObjects::AllSourcesInSchema { schemas } => {
2606                write!(
2607                    f,
2608                    "ALL SOURCES IN SCHEMA {}",
2609                    display_comma_separated(schemas)
2610                )
2611            }
2612            GrantObjects::AllMviewsInSchema { schemas } => {
2613                write!(
2614                    f,
2615                    "ALL MATERIALIZED VIEWS IN SCHEMA {}",
2616                    display_comma_separated(schemas)
2617                )
2618            }
2619            GrantObjects::AllSinksInSchema { schemas } => {
2620                write!(
2621                    f,
2622                    "ALL SINKS IN SCHEMA {}",
2623                    display_comma_separated(schemas)
2624                )
2625            }
2626            GrantObjects::AllViewsInSchema { schemas } => {
2627                write!(
2628                    f,
2629                    "ALL VIEWS IN SCHEMA {}",
2630                    display_comma_separated(schemas)
2631                )
2632            }
2633            GrantObjects::AllFunctionsInSchema { schemas } => {
2634                write!(
2635                    f,
2636                    "ALL FUNCTIONS IN SCHEMA {}",
2637                    display_comma_separated(schemas)
2638                )
2639            }
2640            GrantObjects::AllSecretsInSchema { schemas } => {
2641                write!(
2642                    f,
2643                    "ALL SECRETS IN SCHEMA {}",
2644                    display_comma_separated(schemas)
2645                )
2646            }
2647            GrantObjects::AllSubscriptionsInSchema { schemas } => {
2648                write!(
2649                    f,
2650                    "ALL SUBSCRIPTIONS IN SCHEMA {}",
2651                    display_comma_separated(schemas)
2652                )
2653            }
2654            GrantObjects::AllConnectionsInSchema { schemas } => {
2655                write!(
2656                    f,
2657                    "ALL CONNECTIONS IN SCHEMA {}",
2658                    display_comma_separated(schemas)
2659                )
2660            }
2661            GrantObjects::Databases(databases) => {
2662                write!(f, "DATABASE {}", display_comma_separated(databases))
2663            }
2664            GrantObjects::Sources(sources) => {
2665                write!(f, "SOURCE {}", display_comma_separated(sources))
2666            }
2667            GrantObjects::Mviews(mviews) => {
2668                write!(f, "MATERIALIZED VIEW {}", display_comma_separated(mviews))
2669            }
2670            GrantObjects::Sinks(sinks) => {
2671                write!(f, "SINK {}", display_comma_separated(sinks))
2672            }
2673            GrantObjects::Views(views) => {
2674                write!(f, "VIEW {}", display_comma_separated(views))
2675            }
2676            GrantObjects::Connections(connections) => {
2677                write!(f, "CONNECTION {}", display_comma_separated(connections))
2678            }
2679            GrantObjects::Subscriptions(subscriptions) => {
2680                write!(f, "SUBSCRIPTION {}", display_comma_separated(subscriptions))
2681            }
2682            GrantObjects::Functions(func_descs) => {
2683                write!(f, "FUNCTION {}", display_comma_separated(func_descs))
2684            }
2685            GrantObjects::Secrets(secrets) => {
2686                write!(f, "SECRET {}", display_comma_separated(secrets))
2687            }
2688        }
2689    }
2690}
2691
2692#[derive(Debug, Clone, PartialEq, Eq, Hash)]
2693#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
2694pub enum AssignmentValue {
2695    /// An expression, e.g. `foo = 1`
2696    Expr(Expr),
2697    /// The `DEFAULT` keyword, e.g. `foo = DEFAULT`
2698    Default,
2699}
2700
2701impl fmt::Display for AssignmentValue {
2702    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
2703        match self {
2704            AssignmentValue::Expr(expr) => write!(f, "{}", expr),
2705            AssignmentValue::Default => f.write_str("DEFAULT"),
2706        }
2707    }
2708}
2709
2710/// SQL assignment `foo = { expr | DEFAULT }` as used in SQLUpdate
2711#[derive(Debug, Clone, PartialEq, Eq, Hash)]
2712#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
2713pub struct Assignment {
2714    pub id: Vec<Ident>,
2715    pub value: AssignmentValue,
2716}
2717
2718impl fmt::Display for Assignment {
2719    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
2720        write!(f, "{} = {}", display_separated(&self.id, "."), self.value)
2721    }
2722}
2723
2724#[derive(Debug, Clone, PartialEq, Eq, Hash)]
2725#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
2726pub enum FunctionArgExpr {
2727    Expr(Expr),
2728    /// Expr is an arbitrary expression, returning either a table or a column.
2729    /// Idents are the prefix of `*`, which are consecutive field accesses.
2730    /// e.g. `(table.v1).*` or `(table).v1.*`
2731    ExprQualifiedWildcard(Expr, Vec<Ident>),
2732    /// Qualified wildcard, e.g. `alias.*` or `schema.table.*`, followed by optional
2733    /// except syntax
2734    QualifiedWildcard(ObjectName, Option<Vec<Expr>>),
2735    /// An unqualified `*` or `* except (columns)`
2736    Wildcard(Option<Vec<Expr>>),
2737}
2738
2739impl fmt::Display for FunctionArgExpr {
2740    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
2741        match self {
2742            FunctionArgExpr::Expr(expr) => write!(f, "{}", expr),
2743            FunctionArgExpr::ExprQualifiedWildcard(expr, prefix) => {
2744                write!(
2745                    f,
2746                    "({}){}.*",
2747                    expr,
2748                    prefix
2749                        .iter()
2750                        .format_with("", |i, f| f(&format_args!(".{i}")))
2751                )
2752            }
2753            FunctionArgExpr::QualifiedWildcard(prefix, except) => match except {
2754                Some(exprs) => write!(
2755                    f,
2756                    "{}.* EXCEPT ({})",
2757                    prefix,
2758                    exprs
2759                        .iter()
2760                        .map(|v| v.to_string())
2761                        .collect::<Vec<String>>()
2762                        .as_slice()
2763                        .join(", ")
2764                ),
2765                None => write!(f, "{}.*", prefix),
2766            },
2767
2768            FunctionArgExpr::Wildcard(except) => match except {
2769                Some(exprs) => write!(
2770                    f,
2771                    "* EXCEPT ({})",
2772                    exprs
2773                        .iter()
2774                        .map(|v| v.to_string())
2775                        .collect::<Vec<String>>()
2776                        .as_slice()
2777                        .join(", ")
2778                ),
2779                None => f.write_str("*"),
2780            },
2781        }
2782    }
2783}
2784
2785#[derive(Debug, Clone, PartialEq, Eq, Hash)]
2786#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
2787pub enum FunctionArg {
2788    Named { name: Ident, arg: FunctionArgExpr },
2789    Unnamed(FunctionArgExpr),
2790}
2791
2792impl FunctionArg {
2793    pub fn get_expr(&self) -> FunctionArgExpr {
2794        match self {
2795            FunctionArg::Named { name: _, arg } => arg.clone(),
2796            FunctionArg::Unnamed(arg) => arg.clone(),
2797        }
2798    }
2799}
2800
2801impl fmt::Display for FunctionArg {
2802    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
2803        match self {
2804            FunctionArg::Named { name, arg } => write!(f, "{} => {}", name, arg),
2805            FunctionArg::Unnamed(unnamed_arg) => write!(f, "{}", unnamed_arg),
2806        }
2807    }
2808}
2809
2810/// A list of function arguments, including additional modifiers like `DISTINCT` or `ORDER BY`.
2811/// This basically holds all the information between the `(` and `)` in a function call.
2812#[derive(Debug, Clone, PartialEq, Eq, Hash)]
2813#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
2814pub struct FunctionArgList {
2815    /// Aggregate function calls may have a `DISTINCT`, e.g. `count(DISTINCT x)`.
2816    pub distinct: bool,
2817    pub args: Vec<FunctionArg>,
2818    /// Whether the last argument is variadic, e.g. `foo(a, b, VARIADIC c)`.
2819    pub variadic: bool,
2820    /// Aggregate function calls may have an `ORDER BY`, e.g. `array_agg(x ORDER BY y)`.
2821    pub order_by: Vec<OrderByExpr>,
2822    /// Window function calls may have an `IGNORE NULLS`, e.g. `first_value(x IGNORE NULLS)`.
2823    pub ignore_nulls: bool,
2824}
2825
2826impl fmt::Display for FunctionArgList {
2827    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
2828        write!(f, "(")?;
2829        if self.distinct {
2830            write!(f, "DISTINCT ")?;
2831        }
2832        if self.variadic {
2833            for arg in &self.args[0..self.args.len() - 1] {
2834                write!(f, "{}, ", arg)?;
2835            }
2836            write!(f, "VARIADIC {}", self.args.last().unwrap())?;
2837        } else {
2838            write!(f, "{}", display_comma_separated(&self.args))?;
2839        }
2840        if !self.order_by.is_empty() {
2841            write!(f, " ORDER BY {}", display_comma_separated(&self.order_by))?;
2842        }
2843        if self.ignore_nulls {
2844            write!(f, " IGNORE NULLS")?;
2845        }
2846        write!(f, ")")?;
2847        Ok(())
2848    }
2849}
2850
2851impl FunctionArgList {
2852    pub fn empty() -> Self {
2853        Self {
2854            distinct: false,
2855            args: vec![],
2856            variadic: false,
2857            order_by: vec![],
2858            ignore_nulls: false,
2859        }
2860    }
2861
2862    pub fn args_only(args: Vec<FunctionArg>) -> Self {
2863        Self {
2864            distinct: false,
2865            args,
2866            variadic: false,
2867            order_by: vec![],
2868            ignore_nulls: false,
2869        }
2870    }
2871
2872    pub fn is_args_only(&self) -> bool {
2873        !self.distinct && !self.variadic && self.order_by.is_empty() && !self.ignore_nulls
2874    }
2875
2876    pub fn for_agg(distinct: bool, args: Vec<FunctionArg>, order_by: Vec<OrderByExpr>) -> Self {
2877        Self {
2878            distinct,
2879            args,
2880            variadic: false,
2881            order_by,
2882            ignore_nulls: false,
2883        }
2884    }
2885}
2886
2887/// A function call
2888#[derive(Debug, Clone, PartialEq, Eq, Hash)]
2889#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
2890pub struct Function {
2891    /// Whether the function is prefixed with `AGGREGATE:`
2892    pub scalar_as_agg: bool,
2893    /// Function name.
2894    pub name: ObjectName,
2895    /// Argument list of the function call, i.e. things in `()`.
2896    pub arg_list: FunctionArgList,
2897    /// `WITHIN GROUP` clause of the function call, for ordered-set aggregate functions.
2898    /// FIXME(rc): why we only support one expression here?
2899    pub within_group: Option<Box<OrderByExpr>>,
2900    /// `FILTER` clause of the function call, for aggregate and window (not supported yet) functions.
2901    pub filter: Option<Box<Expr>>,
2902    /// `OVER` clause of the function call, for window functions.
2903    pub over: Option<WindowSpec>,
2904}
2905
2906impl Function {
2907    pub fn no_arg(name: ObjectName) -> Self {
2908        Self {
2909            scalar_as_agg: false,
2910            name,
2911            arg_list: FunctionArgList::empty(),
2912            within_group: None,
2913            filter: None,
2914            over: None,
2915        }
2916    }
2917}
2918
2919impl fmt::Display for Function {
2920    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
2921        if self.scalar_as_agg {
2922            write!(f, "AGGREGATE:")?;
2923        }
2924        write!(f, "{}{}", self.name, self.arg_list)?;
2925        if let Some(within_group) = &self.within_group {
2926            write!(f, " WITHIN GROUP (ORDER BY {})", within_group)?;
2927        }
2928        if let Some(filter) = &self.filter {
2929            write!(f, " FILTER (WHERE {})", filter)?;
2930        }
2931        if let Some(o) = &self.over {
2932            write!(f, " OVER ({})", o)?;
2933        }
2934        Ok(())
2935    }
2936}
2937
2938#[derive(Debug, Clone, PartialEq, Eq, Hash)]
2939#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
2940pub enum ObjectType {
2941    Table,
2942    View,
2943    MaterializedView,
2944    Index,
2945    Schema,
2946    Source,
2947    Sink,
2948    Database,
2949    User,
2950    Connection,
2951    Secret,
2952    Subscription,
2953}
2954
2955impl fmt::Display for ObjectType {
2956    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
2957        f.write_str(match self {
2958            ObjectType::Table => "TABLE",
2959            ObjectType::View => "VIEW",
2960            ObjectType::MaterializedView => "MATERIALIZED VIEW",
2961            ObjectType::Index => "INDEX",
2962            ObjectType::Schema => "SCHEMA",
2963            ObjectType::Source => "SOURCE",
2964            ObjectType::Sink => "SINK",
2965            ObjectType::Database => "DATABASE",
2966            ObjectType::User => "USER",
2967            ObjectType::Secret => "SECRET",
2968            ObjectType::Connection => "CONNECTION",
2969            ObjectType::Subscription => "SUBSCRIPTION",
2970        })
2971    }
2972}
2973
2974impl ParseTo for ObjectType {
2975    fn parse_to(parser: &mut Parser<'_>) -> ModalResult<Self> {
2976        let object_type = if parser.parse_keyword(Keyword::TABLE) {
2977            ObjectType::Table
2978        } else if parser.parse_keyword(Keyword::VIEW) {
2979            ObjectType::View
2980        } else if parser.parse_keywords(&[Keyword::MATERIALIZED, Keyword::VIEW]) {
2981            ObjectType::MaterializedView
2982        } else if parser.parse_keyword(Keyword::SOURCE) {
2983            ObjectType::Source
2984        } else if parser.parse_keyword(Keyword::SINK) {
2985            ObjectType::Sink
2986        } else if parser.parse_keyword(Keyword::INDEX) {
2987            ObjectType::Index
2988        } else if parser.parse_keyword(Keyword::SCHEMA) {
2989            ObjectType::Schema
2990        } else if parser.parse_keyword(Keyword::DATABASE) {
2991            ObjectType::Database
2992        } else if parser.parse_keyword(Keyword::USER) {
2993            ObjectType::User
2994        } else if parser.parse_keyword(Keyword::CONNECTION) {
2995            ObjectType::Connection
2996        } else if parser.parse_keyword(Keyword::SECRET) {
2997            ObjectType::Secret
2998        } else if parser.parse_keyword(Keyword::SUBSCRIPTION) {
2999            ObjectType::Subscription
3000        } else {
3001            return parser.expected(
3002                "TABLE, VIEW, INDEX, MATERIALIZED VIEW, SOURCE, SINK, SUBSCRIPTION, SCHEMA, DATABASE, USER, SECRET or CONNECTION after DROP",
3003            );
3004        };
3005        Ok(object_type)
3006    }
3007}
3008
3009#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3010#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3011pub struct SqlOption {
3012    pub name: ObjectName,
3013    pub value: SqlOptionValue,
3014}
3015
3016impl fmt::Display for SqlOption {
3017    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3018        let should_redact = REDACT_SQL_OPTION_KEYWORDS
3019            .try_with(|keywords| {
3020                let sql_option_name = self.name.real_value().to_lowercase();
3021                keywords.iter().any(|k| sql_option_name.contains(k))
3022            })
3023            .unwrap_or(false);
3024        if should_redact {
3025            write!(f, "{} = '[REDACTED]'", self.name)
3026        } else {
3027            write!(f, "{} = {}", self.name, self.value)
3028        }
3029    }
3030}
3031
3032impl TryFrom<(&String, &String)> for SqlOption {
3033    type Error = ParserError;
3034
3035    fn try_from((name, value): (&String, &String)) -> Result<Self, Self::Error> {
3036        let query = format!("{} = {}", name, value);
3037        let mut tokenizer = Tokenizer::new(query.as_str());
3038        let tokens = tokenizer.tokenize_with_location()?;
3039        let mut parser = Parser(&tokens);
3040        parser
3041            .parse_sql_option()
3042            .map_err(|e| ParserError::ParserError(e.to_string()))
3043    }
3044}
3045
3046#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3047#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3048pub enum SqlOptionValue {
3049    Value(Value),
3050    SecretRef(SecretRefValue),
3051    ConnectionRef(ConnectionRefValue),
3052}
3053
3054impl fmt::Display for SqlOptionValue {
3055    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3056        match self {
3057            SqlOptionValue::Value(value) => write!(f, "{}", value),
3058            SqlOptionValue::SecretRef(secret_ref) => write!(f, "secret {}", secret_ref),
3059            SqlOptionValue::ConnectionRef(connection_ref) => {
3060                write!(f, "{}", connection_ref)
3061            }
3062        }
3063    }
3064}
3065
3066impl From<Value> for SqlOptionValue {
3067    fn from(value: Value) -> Self {
3068        SqlOptionValue::Value(value)
3069    }
3070}
3071
3072#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3073#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3074pub enum EmitMode {
3075    Immediately,
3076    OnWindowClose,
3077}
3078
3079impl fmt::Display for EmitMode {
3080    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3081        f.write_str(match self {
3082            EmitMode::Immediately => "IMMEDIATELY",
3083            EmitMode::OnWindowClose => "ON WINDOW CLOSE",
3084        })
3085    }
3086}
3087
3088#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)]
3089#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3090pub enum OnConflict {
3091    UpdateFull,
3092    Nothing,
3093    UpdateIfNotNull,
3094}
3095
3096impl fmt::Display for OnConflict {
3097    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3098        f.write_str(match self {
3099            OnConflict::UpdateFull => "DO UPDATE FULL",
3100            OnConflict::Nothing => "DO NOTHING",
3101            OnConflict::UpdateIfNotNull => "DO UPDATE IF NOT NULL",
3102        })
3103    }
3104}
3105
3106#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3107#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3108pub enum Engine {
3109    Hummock,
3110    Iceberg,
3111}
3112
3113impl fmt::Display for crate::ast::Engine {
3114    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3115        f.write_str(match self {
3116            crate::ast::Engine::Hummock => "HUMMOCK",
3117            crate::ast::Engine::Iceberg => "ICEBERG",
3118        })
3119    }
3120}
3121
3122#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3123#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3124pub enum SetTimeZoneValue {
3125    Ident(Ident),
3126    Literal(Value),
3127    Local,
3128    Default,
3129}
3130
3131impl fmt::Display for SetTimeZoneValue {
3132    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3133        match self {
3134            SetTimeZoneValue::Ident(ident) => write!(f, "{}", ident),
3135            SetTimeZoneValue::Literal(value) => write!(f, "{}", value),
3136            SetTimeZoneValue::Local => f.write_str("LOCAL"),
3137            SetTimeZoneValue::Default => f.write_str("DEFAULT"),
3138        }
3139    }
3140}
3141
3142#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3143#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3144pub enum TransactionMode {
3145    AccessMode(TransactionAccessMode),
3146    IsolationLevel(TransactionIsolationLevel),
3147}
3148
3149impl fmt::Display for TransactionMode {
3150    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3151        use TransactionMode::*;
3152        match self {
3153            AccessMode(access_mode) => write!(f, "{}", access_mode),
3154            IsolationLevel(iso_level) => write!(f, "ISOLATION LEVEL {}", iso_level),
3155        }
3156    }
3157}
3158
3159#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3160#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3161pub enum TransactionAccessMode {
3162    ReadOnly,
3163    ReadWrite,
3164}
3165
3166impl fmt::Display for TransactionAccessMode {
3167    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3168        use TransactionAccessMode::*;
3169        f.write_str(match self {
3170            ReadOnly => "READ ONLY",
3171            ReadWrite => "READ WRITE",
3172        })
3173    }
3174}
3175
3176#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3177#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3178pub enum TransactionIsolationLevel {
3179    ReadUncommitted,
3180    ReadCommitted,
3181    RepeatableRead,
3182    Serializable,
3183}
3184
3185impl fmt::Display for TransactionIsolationLevel {
3186    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3187        use TransactionIsolationLevel::*;
3188        f.write_str(match self {
3189            ReadUncommitted => "READ UNCOMMITTED",
3190            ReadCommitted => "READ COMMITTED",
3191            RepeatableRead => "REPEATABLE READ",
3192            Serializable => "SERIALIZABLE",
3193        })
3194    }
3195}
3196
3197#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3198#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3199pub enum ShowStatementFilter {
3200    Like(String),
3201    ILike(String),
3202    Where(Expr),
3203}
3204
3205impl fmt::Display for ShowStatementFilter {
3206    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3207        use ShowStatementFilter::*;
3208        match self {
3209            Like(pattern) => write!(f, "LIKE '{}'", value::escape_single_quote_string(pattern)),
3210            ILike(pattern) => write!(f, "ILIKE {}", value::escape_single_quote_string(pattern)),
3211            Where(expr) => write!(f, "WHERE {}", expr),
3212        }
3213    }
3214}
3215
3216/// Function describe in DROP FUNCTION.
3217#[derive(Debug, Clone, PartialEq, PartialOrd, Eq, Ord, Hash)]
3218#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3219pub enum DropFunctionOption {
3220    Restrict,
3221    Cascade,
3222}
3223
3224impl fmt::Display for DropFunctionOption {
3225    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3226        match self {
3227            DropFunctionOption::Restrict => write!(f, "RESTRICT "),
3228            DropFunctionOption::Cascade => write!(f, "CASCADE  "),
3229        }
3230    }
3231}
3232
3233/// Function describe in DROP FUNCTION.
3234#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3235#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3236pub struct FunctionDesc {
3237    pub name: ObjectName,
3238    pub args: Option<Vec<OperateFunctionArg>>,
3239}
3240
3241impl fmt::Display for FunctionDesc {
3242    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3243        write!(f, "{}", self.name)?;
3244        if let Some(args) = &self.args {
3245            write!(f, "({})", display_comma_separated(args))?;
3246        }
3247        Ok(())
3248    }
3249}
3250
3251/// Function argument in CREATE FUNCTION.
3252#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3253#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3254pub struct OperateFunctionArg {
3255    pub mode: Option<ArgMode>,
3256    pub name: Option<Ident>,
3257    pub data_type: DataType,
3258    pub default_expr: Option<Expr>,
3259}
3260
3261impl OperateFunctionArg {
3262    /// Returns an unnamed argument.
3263    pub fn unnamed(data_type: DataType) -> Self {
3264        Self {
3265            mode: None,
3266            name: None,
3267            data_type,
3268            default_expr: None,
3269        }
3270    }
3271
3272    /// Returns an argument with name.
3273    pub fn with_name(name: &str, data_type: DataType) -> Self {
3274        Self {
3275            mode: None,
3276            name: Some(name.into()),
3277            data_type,
3278            default_expr: None,
3279        }
3280    }
3281}
3282
3283impl fmt::Display for OperateFunctionArg {
3284    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3285        if let Some(mode) = &self.mode {
3286            write!(f, "{} ", mode)?;
3287        }
3288        if let Some(name) = &self.name {
3289            write!(f, "{} ", name)?;
3290        }
3291        write!(f, "{}", self.data_type)?;
3292        if let Some(default_expr) = &self.default_expr {
3293            write!(f, " = {}", default_expr)?;
3294        }
3295        Ok(())
3296    }
3297}
3298
3299/// The mode of an argument in CREATE FUNCTION.
3300#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3301#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3302pub enum ArgMode {
3303    In,
3304    Out,
3305    InOut,
3306}
3307
3308impl fmt::Display for ArgMode {
3309    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3310        match self {
3311            ArgMode::In => write!(f, "IN"),
3312            ArgMode::Out => write!(f, "OUT"),
3313            ArgMode::InOut => write!(f, "INOUT"),
3314        }
3315    }
3316}
3317
3318/// These attributes inform the query optimizer about the behavior of the function.
3319#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3320#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3321pub enum FunctionBehavior {
3322    Immutable,
3323    Stable,
3324    Volatile,
3325}
3326
3327impl fmt::Display for FunctionBehavior {
3328    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3329        match self {
3330            FunctionBehavior::Immutable => write!(f, "IMMUTABLE"),
3331            FunctionBehavior::Stable => write!(f, "STABLE"),
3332            FunctionBehavior::Volatile => write!(f, "VOLATILE"),
3333        }
3334    }
3335}
3336
3337#[derive(Debug, Clone, PartialEq, PartialOrd, Eq, Ord, Hash)]
3338#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3339pub enum FunctionDefinition {
3340    Identifier(String),
3341    SingleQuotedDef(String),
3342    DoubleDollarDef(String),
3343}
3344
3345impl fmt::Display for FunctionDefinition {
3346    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3347        match self {
3348            FunctionDefinition::Identifier(s) => write!(f, "{s}")?,
3349            FunctionDefinition::SingleQuotedDef(s) => write!(f, "'{s}'")?,
3350            FunctionDefinition::DoubleDollarDef(s) => write!(f, "$${s}$$")?,
3351        }
3352        Ok(())
3353    }
3354}
3355
3356impl FunctionDefinition {
3357    /// Returns the function definition as a string slice.
3358    pub fn as_str(&self) -> &str {
3359        match self {
3360            FunctionDefinition::Identifier(s) => s,
3361            FunctionDefinition::SingleQuotedDef(s) => s,
3362            FunctionDefinition::DoubleDollarDef(s) => s,
3363        }
3364    }
3365
3366    /// Returns the function definition as a string.
3367    pub fn into_string(self) -> String {
3368        match self {
3369            FunctionDefinition::Identifier(s) => s,
3370            FunctionDefinition::SingleQuotedDef(s) => s,
3371            FunctionDefinition::DoubleDollarDef(s) => s,
3372        }
3373    }
3374}
3375
3376/// Return types of a function.
3377#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3378#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3379pub enum CreateFunctionReturns {
3380    /// RETURNS rettype
3381    Value(DataType),
3382    /// RETURNS TABLE ( column_name column_type [, ...] )
3383    Table(Vec<TableColumnDef>),
3384}
3385
3386impl fmt::Display for CreateFunctionReturns {
3387    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3388        match self {
3389            Self::Value(data_type) => write!(f, "RETURNS {}", data_type),
3390            Self::Table(columns) => {
3391                write!(f, "RETURNS TABLE ({})", display_comma_separated(columns))
3392            }
3393        }
3394    }
3395}
3396
3397/// Table column definition
3398#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3399#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3400pub struct TableColumnDef {
3401    pub name: Ident,
3402    pub data_type: DataType,
3403}
3404
3405impl fmt::Display for TableColumnDef {
3406    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3407        write!(f, "{} {}", self.name, self.data_type)
3408    }
3409}
3410
3411/// Postgres specific feature.
3412///
3413/// See [Postgresdocs](https://www.postgresql.org/docs/15/sql-createfunction.html)
3414/// for more details
3415#[derive(Debug, Default, Clone, PartialEq, Eq, Hash)]
3416#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3417pub struct CreateFunctionBody {
3418    /// LANGUAGE lang_name
3419    pub language: Option<Ident>,
3420    /// RUNTIME runtime_name
3421    pub runtime: Option<Ident>,
3422
3423    /// IMMUTABLE | STABLE | VOLATILE
3424    pub behavior: Option<FunctionBehavior>,
3425    /// AS 'definition'
3426    ///
3427    /// Note that Hive's `AS class_name` is also parsed here.
3428    pub as_: Option<FunctionDefinition>,
3429    /// RETURN expression
3430    pub return_: Option<Expr>,
3431    /// USING ...
3432    pub using: Option<CreateFunctionUsing>,
3433}
3434
3435impl fmt::Display for CreateFunctionBody {
3436    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3437        if let Some(language) = &self.language {
3438            write!(f, " LANGUAGE {language}")?;
3439        }
3440        if let Some(runtime) = &self.runtime {
3441            write!(f, " RUNTIME {runtime}")?;
3442        }
3443        if let Some(behavior) = &self.behavior {
3444            write!(f, " {behavior}")?;
3445        }
3446        if let Some(definition) = &self.as_ {
3447            write!(f, " AS {definition}")?;
3448        }
3449        if let Some(expr) = &self.return_ {
3450            write!(f, " RETURN {expr}")?;
3451        }
3452        if let Some(using) = &self.using {
3453            write!(f, " {using}")?;
3454        }
3455        Ok(())
3456    }
3457}
3458
3459#[derive(Debug, Default, Clone, PartialEq, Eq, Hash)]
3460#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3461pub struct CreateFunctionWithOptions {
3462    /// Always retry on network errors.
3463    pub always_retry_on_network_error: Option<bool>,
3464    /// Use async functions (only available for JS UDF)
3465    pub r#async: Option<bool>,
3466    /// Call in batch mode (only available for JS UDF)
3467    pub batch: Option<bool>,
3468}
3469
3470/// TODO(kwannoel): Generate from the struct definition instead.
3471impl TryFrom<Vec<SqlOption>> for CreateFunctionWithOptions {
3472    type Error = StrError;
3473
3474    fn try_from(with_options: Vec<SqlOption>) -> Result<Self, Self::Error> {
3475        let mut options = Self::default();
3476        for option in with_options {
3477            match option.name.to_string().to_lowercase().as_str() {
3478                "always_retry_on_network_error" => {
3479                    options.always_retry_on_network_error = Some(matches!(
3480                        option.value,
3481                        SqlOptionValue::Value(Value::Boolean(true))
3482                    ));
3483                }
3484                "async" => {
3485                    options.r#async = Some(matches!(
3486                        option.value,
3487                        SqlOptionValue::Value(Value::Boolean(true))
3488                    ))
3489                }
3490                "batch" => {
3491                    options.batch = Some(matches!(
3492                        option.value,
3493                        SqlOptionValue::Value(Value::Boolean(true))
3494                    ))
3495                }
3496                _ => {
3497                    return Err(StrError(format!("unknown option: {}", option.name)));
3498                }
3499            }
3500        }
3501        Ok(options)
3502    }
3503}
3504
3505impl Display for CreateFunctionWithOptions {
3506    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3507        if self == &Self::default() {
3508            return Ok(());
3509        }
3510        let mut options = vec![];
3511        if let Some(v) = self.always_retry_on_network_error {
3512            options.push(format!("always_retry_on_network_error = {}", v));
3513        }
3514        if let Some(v) = self.r#async {
3515            options.push(format!("async = {}", v));
3516        }
3517        if let Some(v) = self.batch {
3518            options.push(format!("batch = {}", v));
3519        }
3520        write!(f, " WITH ( {} )", display_comma_separated(&options))
3521    }
3522}
3523
3524#[derive(Debug, Clone, PartialEq, PartialOrd, Eq, Ord, Hash)]
3525#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3526pub enum CreateFunctionUsing {
3527    Link(String),
3528    Base64(String),
3529}
3530
3531impl fmt::Display for CreateFunctionUsing {
3532    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3533        write!(f, "USING ")?;
3534        match self {
3535            CreateFunctionUsing::Link(uri) => write!(f, "LINK '{uri}'"),
3536            CreateFunctionUsing::Base64(s) => {
3537                write!(f, "BASE64 '{s}'")
3538            }
3539        }
3540    }
3541}
3542
3543#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3544#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3545pub enum SetVariableValue {
3546    Single(SetVariableValueSingle),
3547    List(Vec<SetVariableValueSingle>),
3548    Default,
3549}
3550
3551impl From<SetVariableValueSingle> for SetVariableValue {
3552    fn from(value: SetVariableValueSingle) -> Self {
3553        SetVariableValue::Single(value)
3554    }
3555}
3556
3557impl fmt::Display for SetVariableValue {
3558    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3559        use SetVariableValue::*;
3560        match self {
3561            Single(val) => write!(f, "{}", val),
3562            List(list) => write!(f, "{}", display_comma_separated(list),),
3563            Default => write!(f, "DEFAULT"),
3564        }
3565    }
3566}
3567
3568#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3569#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3570pub enum SetVariableValueSingle {
3571    Ident(Ident),
3572    Literal(Value),
3573}
3574
3575impl SetVariableValueSingle {
3576    pub fn to_string_unquoted(&self) -> String {
3577        match self {
3578            Self::Literal(Value::SingleQuotedString(s))
3579            | Self::Literal(Value::DoubleQuotedString(s)) => s.clone(),
3580            _ => self.to_string(),
3581        }
3582    }
3583}
3584
3585impl fmt::Display for SetVariableValueSingle {
3586    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3587        use SetVariableValueSingle::*;
3588        match self {
3589            Ident(ident) => write!(f, "{}", ident),
3590            Literal(literal) => write!(f, "{}", literal),
3591        }
3592    }
3593}
3594
3595#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3596#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3597pub enum AsOf {
3598    ProcessTime,
3599    // used by time travel
3600    ProcessTimeWithInterval((String, DateTimeField)),
3601    // the number of seconds that have elapsed since the Unix epoch, which is January 1, 1970 at 00:00:00 Coordinated Universal Time (UTC).
3602    TimestampNum(i64),
3603    TimestampString(String),
3604    VersionNum(i64),
3605    VersionString(String),
3606}
3607
3608impl fmt::Display for AsOf {
3609    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3610        use AsOf::*;
3611        match self {
3612            ProcessTime => write!(f, " FOR SYSTEM_TIME AS OF PROCTIME()"),
3613            ProcessTimeWithInterval((value, leading_field)) => write!(
3614                f,
3615                " FOR SYSTEM_TIME AS OF NOW() - {} {}",
3616                value, leading_field
3617            ),
3618            TimestampNum(ts) => write!(f, " FOR SYSTEM_TIME AS OF {}", ts),
3619            TimestampString(ts) => write!(f, " FOR SYSTEM_TIME AS OF '{}'", ts),
3620            VersionNum(v) => write!(f, " FOR SYSTEM_VERSION AS OF {}", v),
3621            VersionString(v) => write!(f, " FOR SYSTEM_VERSION AS OF '{}'", v),
3622        }
3623    }
3624}
3625
3626#[derive(Debug, Clone, PartialEq, Eq, Hash)]
3627#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
3628pub enum DiscardType {
3629    All,
3630}
3631
3632impl fmt::Display for DiscardType {
3633    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
3634        use DiscardType::*;
3635        match self {
3636            All => write!(f, "ALL"),
3637        }
3638    }
3639}
3640
3641impl Statement {
3642    pub fn to_redacted_string(&self, keywords: RedactSqlOptionKeywordsRef) -> String {
3643        REDACT_SQL_OPTION_KEYWORDS.sync_scope(keywords, || self.to_string())
3644    }
3645
3646    /// Create a new `CREATE TABLE` statement with the given `name` and empty fields.
3647    pub fn default_create_table(name: ObjectName) -> Self {
3648        Self::CreateTable {
3649            name,
3650            or_replace: false,
3651            temporary: false,
3652            if_not_exists: false,
3653            columns: Vec::new(),
3654            wildcard_idx: None,
3655            constraints: Vec::new(),
3656            with_options: Vec::new(),
3657            format_encode: None,
3658            source_watermarks: Vec::new(),
3659            append_only: false,
3660            on_conflict: None,
3661            with_version_column: None,
3662            query: None,
3663            cdc_table_info: None,
3664            include_column_options: Vec::new(),
3665            webhook_info: None,
3666            engine: Engine::Hummock,
3667        }
3668    }
3669}
3670
3671#[cfg(test)]
3672mod tests {
3673    use super::*;
3674
3675    #[test]
3676    fn test_grouping_sets_display() {
3677        // a and b in different group
3678        let grouping_sets = Expr::GroupingSets(vec![
3679            vec![Expr::Identifier(Ident::new_unchecked("a"))],
3680            vec![Expr::Identifier(Ident::new_unchecked("b"))],
3681        ]);
3682        assert_eq!("GROUPING SETS ((a), (b))", format!("{}", grouping_sets));
3683
3684        // a and b in the same group
3685        let grouping_sets = Expr::GroupingSets(vec![vec![
3686            Expr::Identifier(Ident::new_unchecked("a")),
3687            Expr::Identifier(Ident::new_unchecked("b")),
3688        ]]);
3689        assert_eq!("GROUPING SETS ((a, b))", format!("{}", grouping_sets));
3690
3691        // (a, b) and (c, d) in different group
3692        let grouping_sets = Expr::GroupingSets(vec![
3693            vec![
3694                Expr::Identifier(Ident::new_unchecked("a")),
3695                Expr::Identifier(Ident::new_unchecked("b")),
3696            ],
3697            vec![
3698                Expr::Identifier(Ident::new_unchecked("c")),
3699                Expr::Identifier(Ident::new_unchecked("d")),
3700            ],
3701        ]);
3702        assert_eq!(
3703            "GROUPING SETS ((a, b), (c, d))",
3704            format!("{}", grouping_sets)
3705        );
3706    }
3707
3708    #[test]
3709    fn test_rollup_display() {
3710        let rollup = Expr::Rollup(vec![vec![Expr::Identifier(Ident::new_unchecked("a"))]]);
3711        assert_eq!("ROLLUP (a)", format!("{}", rollup));
3712
3713        let rollup = Expr::Rollup(vec![vec![
3714            Expr::Identifier(Ident::new_unchecked("a")),
3715            Expr::Identifier(Ident::new_unchecked("b")),
3716        ]]);
3717        assert_eq!("ROLLUP ((a, b))", format!("{}", rollup));
3718
3719        let rollup = Expr::Rollup(vec![
3720            vec![Expr::Identifier(Ident::new_unchecked("a"))],
3721            vec![Expr::Identifier(Ident::new_unchecked("b"))],
3722        ]);
3723        assert_eq!("ROLLUP (a, b)", format!("{}", rollup));
3724
3725        let rollup = Expr::Rollup(vec![
3726            vec![Expr::Identifier(Ident::new_unchecked("a"))],
3727            vec![
3728                Expr::Identifier(Ident::new_unchecked("b")),
3729                Expr::Identifier(Ident::new_unchecked("c")),
3730            ],
3731            vec![Expr::Identifier(Ident::new_unchecked("d"))],
3732        ]);
3733        assert_eq!("ROLLUP (a, (b, c), d)", format!("{}", rollup));
3734    }
3735
3736    #[test]
3737    fn test_cube_display() {
3738        let cube = Expr::Cube(vec![vec![Expr::Identifier(Ident::new_unchecked("a"))]]);
3739        assert_eq!("CUBE (a)", format!("{}", cube));
3740
3741        let cube = Expr::Cube(vec![vec![
3742            Expr::Identifier(Ident::new_unchecked("a")),
3743            Expr::Identifier(Ident::new_unchecked("b")),
3744        ]]);
3745        assert_eq!("CUBE ((a, b))", format!("{}", cube));
3746
3747        let cube = Expr::Cube(vec![
3748            vec![Expr::Identifier(Ident::new_unchecked("a"))],
3749            vec![Expr::Identifier(Ident::new_unchecked("b"))],
3750        ]);
3751        assert_eq!("CUBE (a, b)", format!("{}", cube));
3752
3753        let cube = Expr::Cube(vec![
3754            vec![Expr::Identifier(Ident::new_unchecked("a"))],
3755            vec![
3756                Expr::Identifier(Ident::new_unchecked("b")),
3757                Expr::Identifier(Ident::new_unchecked("c")),
3758            ],
3759            vec![Expr::Identifier(Ident::new_unchecked("d"))],
3760        ]);
3761        assert_eq!("CUBE (a, (b, c), d)", format!("{}", cube));
3762    }
3763
3764    #[test]
3765    fn test_array_index_display() {
3766        let array_index = Expr::Index {
3767            obj: Box::new(Expr::Identifier(Ident::new_unchecked("v1"))),
3768            index: Box::new(Expr::Value(Value::Number("1".into()))),
3769        };
3770        assert_eq!("v1[1]", format!("{}", array_index));
3771
3772        let array_index2 = Expr::Index {
3773            obj: Box::new(array_index),
3774            index: Box::new(Expr::Value(Value::Number("1".into()))),
3775        };
3776        assert_eq!("v1[1][1]", format!("{}", array_index2));
3777    }
3778
3779    #[test]
3780    /// issue: https://github.com/risingwavelabs/risingwave/issues/7635
3781    fn test_nested_op_display() {
3782        let binary_op = Expr::BinaryOp {
3783            left: Box::new(Expr::Value(Value::Boolean(true))),
3784            op: BinaryOperator::Or,
3785            right: Box::new(Expr::IsNotFalse(Box::new(Expr::Value(Value::Boolean(
3786                true,
3787            ))))),
3788        };
3789        assert_eq!("true OR true IS NOT FALSE", format!("{}", binary_op));
3790
3791        let unary_op = Expr::UnaryOp {
3792            op: UnaryOperator::Not,
3793            expr: Box::new(Expr::IsNotFalse(Box::new(Expr::Value(Value::Boolean(
3794                true,
3795            ))))),
3796        };
3797        assert_eq!("NOT true IS NOT FALSE", format!("{}", unary_op));
3798    }
3799
3800    #[test]
3801    fn test_create_function_display() {
3802        let create_function = Statement::CreateFunction {
3803            or_replace: false,
3804            temporary: false,
3805            if_not_exists: false,
3806            name: ObjectName(vec![Ident::new_unchecked("foo")]),
3807            args: Some(vec![OperateFunctionArg::unnamed(DataType::Int)]),
3808            returns: Some(CreateFunctionReturns::Value(DataType::Int)),
3809            params: CreateFunctionBody {
3810                language: Some(Ident::new_unchecked("python")),
3811                runtime: None,
3812                behavior: Some(FunctionBehavior::Immutable),
3813                as_: Some(FunctionDefinition::SingleQuotedDef("SELECT 1".to_owned())),
3814                return_: None,
3815                using: None,
3816            },
3817            with_options: CreateFunctionWithOptions {
3818                always_retry_on_network_error: None,
3819                r#async: None,
3820                batch: None,
3821            },
3822        };
3823        assert_eq!(
3824            "CREATE FUNCTION foo(INT) RETURNS INT LANGUAGE python IMMUTABLE AS 'SELECT 1'",
3825            format!("{}", create_function)
3826        );
3827        let create_function = Statement::CreateFunction {
3828            or_replace: false,
3829            temporary: false,
3830            if_not_exists: false,
3831            name: ObjectName(vec![Ident::new_unchecked("foo")]),
3832            args: Some(vec![OperateFunctionArg::unnamed(DataType::Int)]),
3833            returns: Some(CreateFunctionReturns::Value(DataType::Int)),
3834            params: CreateFunctionBody {
3835                language: Some(Ident::new_unchecked("python")),
3836                runtime: None,
3837                behavior: Some(FunctionBehavior::Immutable),
3838                as_: Some(FunctionDefinition::SingleQuotedDef("SELECT 1".to_owned())),
3839                return_: None,
3840                using: None,
3841            },
3842            with_options: CreateFunctionWithOptions {
3843                always_retry_on_network_error: Some(true),
3844                r#async: None,
3845                batch: None,
3846            },
3847        };
3848        assert_eq!(
3849            "CREATE FUNCTION foo(INT) RETURNS INT LANGUAGE python IMMUTABLE AS 'SELECT 1' WITH ( always_retry_on_network_error = true )",
3850            format!("{}", create_function)
3851        );
3852    }
3853}