mz_sql/plan/
typeconv.rs

1// Copyright Materialize, Inc. and contributors. All rights reserved.
2//
3// Use of this software is governed by the Business Source License
4// included in the LICENSE file.
5//
6// As of the Change Date specified in that file, in accordance with
7// the Business Source License, use of this software will be governed
8// by the Apache License, Version 2.0.
9
10//! Maintains a catalog of valid casts between [`mz_repr::SqlScalarType`]s, as well as
11//! other cast-related functions.
12
13use std::cell::RefCell;
14use std::collections::BTreeMap;
15use std::sync::LazyLock;
16
17use dynfmt::{Format, SimpleCurlyFormat};
18use itertools::Itertools;
19use mz_expr::func::{CastArrayToJsonb, CastListToJsonb};
20use mz_expr::{VariadicFunc, func};
21use mz_repr::{
22    ColumnName, Datum, SqlColumnType, SqlRelationType, SqlScalarBaseType, SqlScalarType,
23};
24
25use crate::catalog::TypeCategory;
26use crate::plan::error::PlanError;
27use crate::plan::hir::{
28    AbstractColumnType, CoercibleScalarExpr, CoercibleScalarType, HirScalarExpr, UnaryFunc,
29};
30use crate::plan::query::{ExprContext, QueryContext};
31use crate::plan::scope::Scope;
32
33/// Like func::sql_impl_func, but for casts.
34fn sql_impl_cast(expr: &'static str) -> CastTemplate {
35    let invoke = crate::func::sql_impl(expr);
36    CastTemplate::new(move |ecx, _ccx, from_type, _to_type| {
37        // Oddly, this needs to be able to gracefully fail so we can detect unmet dependencies.
38        let mut out = invoke(ecx.qcx, vec![from_type.clone()]).ok()?;
39        Some(move |e| {
40            out.splice_parameters(&[e], 0);
41            out
42        })
43    })
44}
45
46fn sql_impl_cast_per_context(casts: &[(CastContext, &'static str)]) -> CastTemplate {
47    let casts: BTreeMap<CastContext, _> = casts
48        .iter()
49        .map(|(ccx, expr)| (ccx.clone(), crate::func::sql_impl(expr)))
50        .collect();
51    CastTemplate::new(move |ecx, ccx, from_type, _to_type| {
52        let invoke = &casts[&ccx];
53        let r = invoke(ecx.qcx, vec![from_type.clone()]);
54        let mut out = r.ok()?;
55        Some(move |e| {
56            out.splice_parameters(&[e], 0);
57            out
58        })
59    })
60}
61
62/// A cast is a function that takes a `ScalarExpr` to another `ScalarExpr`.
63type Cast = Box<dyn FnOnce(HirScalarExpr) -> HirScalarExpr>;
64
65/// A cast template is a function that produces a `Cast` given a concrete input
66/// and output type. A template can return `None` to indicate that it is
67/// incapable of producing a cast for the specified types.
68///
69/// Cast templates are used to share code for similar casts, where the input or
70/// output type is of one "category" of type. For example, a single cast
71/// template handles converting from strings to any list type. Without cast
72/// templates, we'd have to enumerate every possible list -> list conversion,
73/// which is impractical.
74struct CastTemplate(
75    Box<
76        dyn Fn(&ExprContext, CastContext, &SqlScalarType, &SqlScalarType) -> Option<Cast>
77            + Send
78            + Sync,
79    >,
80);
81
82impl CastTemplate {
83    fn new<T, C>(t: T) -> CastTemplate
84    where
85        T: Fn(&ExprContext, CastContext, &SqlScalarType, &SqlScalarType) -> Option<C>
86            + Send
87            + Sync
88            + 'static,
89        C: FnOnce(HirScalarExpr) -> HirScalarExpr + 'static,
90    {
91        CastTemplate(Box::new(move |ecx, ccx, from_ty, to_ty| {
92            Some(Box::new(t(ecx, ccx, from_ty, to_ty)?))
93        }))
94    }
95}
96
97impl From<UnaryFunc> for CastTemplate {
98    fn from(u: UnaryFunc) -> CastTemplate {
99        CastTemplate::new(move |_ecx, _ccx, _from, _to| {
100            let u = u.clone();
101            Some(move |expr: HirScalarExpr| expr.call_unary(u))
102        })
103    }
104}
105
106impl<const N: usize> From<[UnaryFunc; N]> for CastTemplate {
107    fn from(funcs: [UnaryFunc; N]) -> CastTemplate {
108        CastTemplate::new(move |_ecx, _ccx, _from, _to| {
109            let funcs = funcs.clone();
110            Some(move |mut expr: HirScalarExpr| {
111                for func in funcs {
112                    expr = expr.call_unary(func.clone());
113                }
114                expr
115            })
116        })
117    }
118}
119
120/// STRING to REG*
121///
122/// A reg* type represents a specific type of object by oid.
123///
124/// Casting from a string to a reg*:
125/// - Accepts a string that looks like an OID and converts the value to the
126///   specified reg* type. This is available in all cases except explicitly
127///   casting text values to regclass (e.g. `SELECT '2'::text::regclass`)
128/// - Resolves non-OID-appearing strings to objects. If this string resolves to
129///   more than one OID (e.g. functions), it errors.
130///
131/// The below code provides a template to accomplish this for various reg*
132/// types. Arguments in order are:
133/// - 0: type catalog name this is casting to
134/// - 1: the category of this reg for the error message
135/// - 2: Whether or not to permit passing through numeric values as OIDs
136const STRING_REG_CAST_TEMPLATE: &str = "
137(SELECT
138CASE
139    WHEN $1 IS NULL THEN NULL
140-- Handle OID-like input, if available via {2}
141    WHEN {2} AND pg_catalog.substring($1, 1, 1) BETWEEN '0' AND '9' THEN
142        $1::pg_catalog.oid::pg_catalog.{0}
143    ELSE (
144    -- String case; look up that the item exists
145        SELECT o.oid
146        FROM mz_unsafe.mz_error_if_null(
147            (
148                -- We need to ensure a distinct here in the case of e.g. functions,
149                -- where multiple items share a GlobalId.
150                SELECT DISTINCT id AS name_id
151                FROM mz_internal.mz_resolve_object_name('{0}', $1)
152            ),
153            -- TODO: Support the correct error code for does not exist (42883).
154            '{1} \"' || $1 || '\" does not exist'
155        ) AS i (name_id),
156        -- Lateral lets us error separately from DNE case
157        LATERAL (
158            SELECT
159                CASE
160            -- Handle too many OIDs
161                WHEN mz_catalog.list_length(mz_catalog.list_agg(oid)) > 1 THEN
162                    mz_unsafe.mz_error_if_null(
163                        NULL::pg_catalog.{0},
164                        'more than one {1} named \"' || $1 || '\"'
165                    )
166            -- Resolve object name's OID if we know there is only one
167                ELSE
168                    CAST(mz_catalog.list_agg(oid)[1] AS pg_catalog.{0})
169                END
170            FROM mz_catalog.mz_objects
171            WHERE id = name_id
172            GROUP BY id
173        ) AS o (oid)
174    )
175END)";
176
177static STRING_TO_REGCLASS_EXPLICIT: LazyLock<String> = LazyLock::new(|| {
178    SimpleCurlyFormat
179        .format(STRING_REG_CAST_TEMPLATE, ["regclass", "relation", "false"])
180        .unwrap()
181        .to_string()
182});
183
184static STRING_TO_REGCLASS_COERCED: LazyLock<String> = LazyLock::new(|| {
185    SimpleCurlyFormat
186        .format(STRING_REG_CAST_TEMPLATE, ["regclass", "relation", "true"])
187        .unwrap()
188        .to_string()
189});
190
191static STRING_TO_REGPROC: LazyLock<String> = LazyLock::new(|| {
192    SimpleCurlyFormat
193        .format(STRING_REG_CAST_TEMPLATE, ["regproc", "function", "true"])
194        .unwrap()
195        .to_string()
196});
197
198static STRING_TO_REGTYPE: LazyLock<String> = LazyLock::new(|| {
199    SimpleCurlyFormat
200        .format(STRING_REG_CAST_TEMPLATE, ["regtype", "type", "true"])
201        .unwrap()
202        .to_string()
203});
204
205const REG_STRING_CAST_TEMPLATE: &str = "(
206SELECT
207    COALESCE(mz_internal.mz_global_id_to_name(o.id), CAST($1 AS pg_catalog.oid)::pg_catalog.text)
208    AS text
209FROM
210  (
211        SELECT
212          (
213            SELECT DISTINCT id
214            FROM
215              mz_catalog.mz_objects AS o
216                JOIN
217                  mz_internal.mz_object_oid_alias AS a
218                  ON o.type = a.object_type
219            WHERE
220              oid = CAST($1 AS pg_catalog.oid)
221                AND
222              a.oid_alias = '{0}'
223          )
224      )
225      AS o
226)";
227
228static REGCLASS_TO_STRING: LazyLock<String> = LazyLock::new(|| {
229    SimpleCurlyFormat
230        .format(REG_STRING_CAST_TEMPLATE, ["regclass"])
231        .unwrap()
232        .to_string()
233});
234
235static REGPROC_TO_STRING: LazyLock<String> = LazyLock::new(|| {
236    SimpleCurlyFormat
237        .format(REG_STRING_CAST_TEMPLATE, ["regproc"])
238        .unwrap()
239        .to_string()
240});
241
242static REGTYPE_TO_STRING: LazyLock<String> = LazyLock::new(|| {
243    SimpleCurlyFormat
244        .format(REG_STRING_CAST_TEMPLATE, ["regtype"])
245        .unwrap()
246        .to_string()
247});
248
249/// Describes the context of a cast.
250///
251/// n.b. this type derived `Ord, PartialOrd` and the ordering of these values
252/// has semantics meaning; casts are only permitted when the caller's cast
253/// context is geq the ccx we store, which is the minimum required context.
254#[derive(Debug, Clone, Copy, Eq, PartialEq, Ord, PartialOrd)]
255pub enum CastContext {
256    /// Implicit casts are "no-brainer" casts that apply automatically in
257    /// expressions. They are typically lossless, such as `SqlScalarType::Int32` to
258    /// `SqlScalarType::Int64`.
259    Implicit,
260    /// Assignment casts are "reasonable" casts that make sense to apply
261    /// automatically in `INSERT` statements, but are surprising enough that
262    /// they don't apply implicitly in expressions.
263    Assignment,
264    /// Explicit casts are casts that are possible but may be surprising, like
265    /// casting `SqlScalarType::Json` to `SqlScalarType::Int32`, and therefore they do
266    /// not happen unless explicitly requested by the user with a cast operator.
267    Explicit,
268    /// Coerced casts permit different behavior when a type is coerced from a
269    /// string literal vs. a value of type `pg_catalog::text`.
270    ///
271    /// The only call site that should pass this value in to this module is
272    /// string coercion.
273    Coerced,
274}
275
276/// The implementation of a cast.
277struct CastImpl {
278    template: CastTemplate,
279    context: CastContext,
280}
281
282macro_rules! casts(
283    {
284        $(
285            $from_to:expr => $cast_context:ident: $cast_template:expr
286        ),+
287    } => {{
288        let mut m = BTreeMap::new();
289        $(
290            m.insert($from_to, CastImpl {
291                template: $cast_template.into(),
292                context: CastContext::$cast_context,
293            });
294        )+
295        m
296    }};
297);
298
299static VALID_CASTS: LazyLock<BTreeMap<(SqlScalarBaseType, SqlScalarBaseType), CastImpl>> =
300    LazyLock::new(|| {
301        use SqlScalarBaseType::*;
302        use UnaryFunc::*;
303
304        casts! {
305            // BOOL
306            (Bool, Int32) => Explicit: CastBoolToInt32(func::CastBoolToInt32),
307            (Bool, Int64) => Explicit: CastBoolToInt64(func::CastBoolToInt64),
308            (Bool, String) => Assignment: CastBoolToString(func::CastBoolToString),
309
310            //INT16
311            (Int16, Int32) => Implicit: CastInt16ToInt32(func::CastInt16ToInt32),
312            (Int16, Int64) => Implicit: CastInt16ToInt64(func::CastInt16ToInt64),
313            (Int16, UInt16) => Assignment: CastInt16ToUint16(func::CastInt16ToUint16),
314            (Int16, UInt32) => Assignment: CastInt16ToUint32(func::CastInt16ToUint32),
315            (Int16, UInt64) => Assignment: CastInt16ToUint64(func::CastInt16ToUint64),
316            (Int16, Float32) => Implicit: CastInt16ToFloat32(func::CastInt16ToFloat32),
317            (Int16, Float64) => Implicit: CastInt16ToFloat64(func::CastInt16ToFloat64),
318            (Int16, Numeric) => Implicit: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
319                let s = to_type.unwrap_numeric_max_scale();
320                Some(move |e: HirScalarExpr| e.call_unary(CastInt16ToNumeric(func::CastInt16ToNumeric(s))))
321            }),
322            (Int16, Oid) => Implicit: [
323                CastInt16ToInt32(func::CastInt16ToInt32),
324                CastInt32ToOid(func::CastInt32ToOid),
325            ],
326            (Int16, RegClass) => Implicit: [
327                CastInt16ToInt32(func::CastInt16ToInt32),
328                CastInt32ToOid(func::CastInt32ToOid),
329                CastOidToRegClass(func::CastOidToRegClass),
330            ],
331            (Int16, RegProc) => Implicit: [
332                CastInt16ToInt32(func::CastInt16ToInt32),
333                CastInt32ToOid(func::CastInt32ToOid),
334                CastOidToRegProc(func::CastOidToRegProc),
335            ],
336            (Int16, RegType) => Implicit: [
337                CastInt16ToInt32(func::CastInt16ToInt32),
338                CastInt32ToOid(func::CastInt32ToOid),
339                CastOidToRegType(func::CastOidToRegType),
340            ],
341            (Int16, String) => Assignment: CastInt16ToString(func::CastInt16ToString),
342
343            //INT32
344            (Int32, Bool) => Explicit: CastInt32ToBool(func::CastInt32ToBool),
345            (Int32, Oid) => Implicit: CastInt32ToOid(func::CastInt32ToOid),
346            (Int32, RegClass) => Implicit: [
347                CastInt32ToOid(func::CastInt32ToOid),
348                CastOidToRegClass(func::CastOidToRegClass),
349            ],
350            (Int32, RegProc) => Implicit: [
351                CastInt32ToOid(func::CastInt32ToOid),
352                CastOidToRegProc(func::CastOidToRegProc),
353            ],
354            (Int32, RegType) => Implicit: [
355                CastInt32ToOid(func::CastInt32ToOid),
356                CastOidToRegType(func::CastOidToRegType),
357            ],
358            (Int32, PgLegacyChar) => Explicit: CastInt32ToPgLegacyChar(func::CastInt32ToPgLegacyChar),
359            (Int32, Int16) => Assignment: CastInt32ToInt16(func::CastInt32ToInt16),
360            (Int32, Int64) => Implicit: CastInt32ToInt64(func::CastInt32ToInt64),
361            (Int32, UInt16) => Assignment: CastInt32ToUint16(func::CastInt32ToUint16),
362            (Int32, UInt32) => Assignment: CastInt32ToUint32(func::CastInt32ToUint32),
363            (Int32, UInt64) => Assignment: CastInt32ToUint64(func::CastInt32ToUint64),
364            (Int32, Float32) => Implicit: CastInt32ToFloat32(func::CastInt32ToFloat32),
365            (Int32, Float64) => Implicit: CastInt32ToFloat64(func::CastInt32ToFloat64),
366            (Int32, Numeric) => Implicit: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
367                let s = to_type.unwrap_numeric_max_scale();
368                Some(move |e: HirScalarExpr| e.call_unary(CastInt32ToNumeric(func::CastInt32ToNumeric(s))))
369            }),
370            (Int32, String) => Assignment: CastInt32ToString(func::CastInt32ToString),
371
372            // INT64
373            (Int64, Bool) => Explicit: CastInt64ToBool(func::CastInt64ToBool),
374            (Int64, Int16) => Assignment: CastInt64ToInt16(func::CastInt64ToInt16),
375            (Int64, Int32) => Assignment: CastInt64ToInt32(func::CastInt64ToInt32),
376            (Int64, UInt16) => Assignment: CastInt64ToUint16(func::CastInt64ToUint16),
377            (Int64, UInt32) => Assignment: CastInt64ToUint32(func::CastInt64ToUint32),
378            (Int64, UInt64) => Assignment: CastInt64ToUint64(func::CastInt64ToUint64),
379            (Int64, Numeric) => Implicit: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
380                let s = to_type.unwrap_numeric_max_scale();
381                Some(move |e: HirScalarExpr| e.call_unary(CastInt64ToNumeric(func::CastInt64ToNumeric(s))))
382            }),
383            (Int64, Float32) => Implicit: CastInt64ToFloat32(func::CastInt64ToFloat32),
384            (Int64, Float64) => Implicit: CastInt64ToFloat64(func::CastInt64ToFloat64),
385            (Int64, Oid) => Implicit: CastInt64ToOid(func::CastInt64ToOid),
386            (Int64, RegClass) => Implicit: [
387                CastInt64ToOid(func::CastInt64ToOid),
388                CastOidToRegClass(func::CastOidToRegClass),
389            ],
390            (Int64, RegProc) => Implicit: [
391                CastInt64ToOid(func::CastInt64ToOid),
392                CastOidToRegProc(func::CastOidToRegProc),
393            ],
394            (Int64, RegType) => Implicit: [
395                CastInt64ToOid(func::CastInt64ToOid),
396                CastOidToRegType(func::CastOidToRegType),
397            ],
398            (Int64, String) => Assignment: CastInt64ToString(func::CastInt64ToString),
399
400            // UINT16
401            (UInt16, UInt32) => Implicit: CastUint16ToUint32(func::CastUint16ToUint32),
402            (UInt16, UInt64) => Implicit: CastUint16ToUint64(func::CastUint16ToUint64),
403            (UInt16, Int16) => Assignment: CastUint16ToInt16(func::CastUint16ToInt16),
404            (UInt16, Int32) => Implicit: CastUint16ToInt32(func::CastUint16ToInt32),
405            (UInt16, Int64) => Implicit: CastUint16ToInt64(func::CastUint16ToInt64),
406            (UInt16, Numeric) => Implicit: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
407                let s = to_type.unwrap_numeric_max_scale();
408                Some(move |e: HirScalarExpr| e.call_unary(CastUint16ToNumeric(func::CastUint16ToNumeric(s))))
409            }),
410            (UInt16, Float32) => Implicit: CastUint16ToFloat32(func::CastUint16ToFloat32),
411            (UInt16, Float64) => Implicit: CastUint16ToFloat64(func::CastUint16ToFloat64),
412            (UInt16, String) => Assignment: CastUint16ToString(func::CastUint16ToString),
413
414            // UINT32
415            (UInt32, UInt16) => Assignment: CastUint32ToUint16(func::CastUint32ToUint16),
416            (UInt32, UInt64) => Implicit: CastUint32ToUint64(func::CastUint32ToUint64),
417            (UInt32, Int16) => Assignment: CastUint32ToInt16(func::CastUint32ToInt16),
418            (UInt32, Int32) => Assignment: CastUint32ToInt32(func::CastUint32ToInt32),
419            (UInt32, Int64) => Implicit: CastUint32ToInt64(func::CastUint32ToInt64),
420            (UInt32, Numeric) => Implicit: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
421                let s = to_type.unwrap_numeric_max_scale();
422                Some(move |e: HirScalarExpr| e.call_unary(CastUint32ToNumeric(func::CastUint32ToNumeric(s))))
423            }),
424            (UInt32, Float32) => Implicit: CastUint32ToFloat32(func::CastUint32ToFloat32),
425            (UInt32, Float64) => Implicit: CastUint32ToFloat64(func::CastUint32ToFloat64),
426            (UInt32, String) => Assignment: CastUint32ToString(func::CastUint32ToString),
427
428            // UINT64
429            (UInt64, UInt16) => Assignment: CastUint64ToUint16(func::CastUint64ToUint16),
430            (UInt64, UInt32) => Assignment: CastUint64ToUint32(func::CastUint64ToUint32),
431            (UInt64, Int16) => Assignment: CastUint64ToInt16(func::CastUint64ToInt16),
432            (UInt64, Int32) => Assignment: CastUint64ToInt32(func::CastUint64ToInt32),
433            (UInt64, Int64) => Assignment: CastUint64ToInt64(func::CastUint64ToInt64),
434            (UInt64, Numeric) => Implicit: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
435                let s = to_type.unwrap_numeric_max_scale();
436                Some(move |e: HirScalarExpr| e.call_unary(CastUint64ToNumeric(func::CastUint64ToNumeric(s))))
437            }),
438            (UInt64, Float32) => Implicit: CastUint64ToFloat32(func::CastUint64ToFloat32),
439            (UInt64, Float64) => Implicit: CastUint64ToFloat64(func::CastUint64ToFloat64),
440            (UInt64, String) => Assignment: CastUint64ToString(func::CastUint64ToString),
441
442            // MZ_TIMESTAMP
443            (MzTimestamp, String) => Assignment: CastMzTimestampToString(func::CastMzTimestampToString),
444            (MzTimestamp, Timestamp) => Assignment: CastMzTimestampToTimestamp(func::CastMzTimestampToTimestamp),
445            (MzTimestamp, TimestampTz) => Assignment: CastMzTimestampToTimestampTz(func::CastMzTimestampToTimestampTz),
446            (String, MzTimestamp) => Assignment: CastStringToMzTimestamp(func::CastStringToMzTimestamp),
447            (UInt64, MzTimestamp) => Implicit: CastUint64ToMzTimestamp(func::CastUint64ToMzTimestamp),
448            (UInt32, MzTimestamp) => Implicit: CastUint32ToMzTimestamp(func::CastUint32ToMzTimestamp),
449            (Int64, MzTimestamp) => Implicit: CastInt64ToMzTimestamp(func::CastInt64ToMzTimestamp),
450            (Int32, MzTimestamp) => Implicit: CastInt32ToMzTimestamp(func::CastInt32ToMzTimestamp),
451            (Numeric, MzTimestamp) => Implicit: CastNumericToMzTimestamp(func::CastNumericToMzTimestamp),
452            (Timestamp, MzTimestamp) => Implicit: CastTimestampToMzTimestamp(func::CastTimestampToMzTimestamp),
453            (TimestampTz, MzTimestamp) => Implicit: CastTimestampTzToMzTimestamp(func::CastTimestampTzToMzTimestamp),
454            (Date, MzTimestamp) => Implicit: CastDateToMzTimestamp(func::CastDateToMzTimestamp),
455
456            // OID
457            (Oid, Int32) => Assignment: CastOidToInt32(func::CastOidToInt32),
458            (Oid, Int64) => Assignment: CastOidToInt32(func::CastOidToInt32),
459            (Oid, String) => Explicit: CastOidToString(func::CastOidToString),
460            (Oid, RegClass) => Implicit: CastOidToRegClass(func::CastOidToRegClass),
461            (Oid, RegProc) => Implicit: CastOidToRegProc(func::CastOidToRegProc),
462            (Oid, RegType) => Implicit: CastOidToRegType(func::CastOidToRegType),
463
464            // REGCLASS
465            (RegClass, Oid) => Implicit: CastRegClassToOid(func::CastRegClassToOid),
466            (RegClass, String) => Explicit: sql_impl_cast(&REGCLASS_TO_STRING),
467
468            // REGPROC
469            (RegProc, Oid) => Implicit: CastRegProcToOid(func::CastRegProcToOid),
470            (RegProc, String) => Explicit: sql_impl_cast(&REGPROC_TO_STRING),
471
472            // REGTYPE
473            (RegType, Oid) => Implicit: CastRegTypeToOid(func::CastRegTypeToOid),
474            (RegType, String) => Explicit: sql_impl_cast(&REGTYPE_TO_STRING),
475
476            // FLOAT32
477            (Float32, Int16) => Assignment: CastFloat32ToInt16(func::CastFloat32ToInt16),
478            (Float32, Int32) => Assignment: CastFloat32ToInt32(func::CastFloat32ToInt32),
479            (Float32, Int64) => Assignment: CastFloat32ToInt64(func::CastFloat32ToInt64),
480            (Float32, UInt16) => Assignment: CastFloat32ToUint16(func::CastFloat32ToUint16),
481            (Float32, UInt32) => Assignment: CastFloat32ToUint32(func::CastFloat32ToUint32),
482            (Float32, UInt64) => Assignment: CastFloat32ToUint64(func::CastFloat32ToUint64),
483            (Float32, Float64) => Implicit: CastFloat32ToFloat64(func::CastFloat32ToFloat64),
484            (Float32, Numeric) => Assignment: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
485                let s = to_type.unwrap_numeric_max_scale();
486                Some(move |e: HirScalarExpr| e.call_unary(CastFloat32ToNumeric(func::CastFloat32ToNumeric(s))))
487            }),
488            (Float32, String) => Assignment: CastFloat32ToString(func::CastFloat32ToString),
489
490            // FLOAT64
491            (Float64, Int16) => Assignment: CastFloat64ToInt16(func::CastFloat64ToInt16),
492            (Float64, Int32) => Assignment: CastFloat64ToInt32(func::CastFloat64ToInt32),
493            (Float64, Int64) => Assignment: CastFloat64ToInt64(func::CastFloat64ToInt64),
494            (Float64, UInt16) => Assignment: CastFloat64ToUint16(func::CastFloat64ToUint16),
495            (Float64, UInt32) => Assignment: CastFloat64ToUint32(func::CastFloat64ToUint32),
496            (Float64, UInt64) => Assignment: CastFloat64ToUint64(func::CastFloat64ToUint64),
497            (Float64, Float32) => Assignment: CastFloat64ToFloat32(func::CastFloat64ToFloat32),
498            (Float64, Numeric) => Assignment: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
499                let s = to_type.unwrap_numeric_max_scale();
500                Some(move |e: HirScalarExpr| e.call_unary(CastFloat64ToNumeric(func::CastFloat64ToNumeric(s))))
501            }),
502            (Float64, String) => Assignment: CastFloat64ToString(func::CastFloat64ToString),
503
504            // DATE
505            (Date, Timestamp) => Implicit: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
506                let p = to_type.unwrap_timestamp_precision();
507                Some(move |e: HirScalarExpr| e.call_unary(CastDateToTimestamp(func::CastDateToTimestamp(p))))
508            }),
509            (Date, TimestampTz) => Implicit: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
510                let p = to_type.unwrap_timestamp_precision();
511                Some(move |e: HirScalarExpr| e.call_unary(CastDateToTimestampTz(func::CastDateToTimestampTz(p))))
512            }),
513            (Date, String) => Assignment: CastDateToString(func::CastDateToString),
514
515            // TIME
516            (Time, Interval) => Implicit: CastTimeToInterval(func::CastTimeToInterval),
517            (Time, String) => Assignment: CastTimeToString(func::CastTimeToString),
518
519            // TIMESTAMP
520            (Timestamp, Date) => Assignment: CastTimestampToDate(func::CastTimestampToDate),
521            (Timestamp, TimestampTz) => Implicit: CastTemplate::new(|_ecx, _ccx, from_type, to_type| {
522                let from = from_type.unwrap_timestamp_precision();
523                let to = to_type.unwrap_timestamp_precision();
524                Some(move |e: HirScalarExpr| e.call_unary(CastTimestampToTimestampTz(func::CastTimestampToTimestampTz{from, to})))
525            }),
526            (Timestamp, Timestamp) => Assignment: CastTemplate::new(|_ecx, _ccx, from_type, to_type| {
527                let from = from_type.unwrap_timestamp_precision();
528                let to = to_type.unwrap_timestamp_precision();
529                Some(move |e: HirScalarExpr| e.call_unary(AdjustTimestampPrecision(func::AdjustTimestampPrecision{from, to})))
530            }),
531            (Timestamp, Time) => Assignment: CastTimestampToTime(func::CastTimestampToTime),
532            (Timestamp, String) => Assignment: CastTimestampToString(func::CastTimestampToString),
533
534            // TIMESTAMPTZ
535            (TimestampTz, Date) => Assignment: CastTimestampTzToDate(func::CastTimestampTzToDate),
536            (TimestampTz, Timestamp) => Assignment: CastTemplate::new(|_ecx, _ccx, from_type, to_type| {
537                let from = from_type.unwrap_timestamp_precision();
538                let to = to_type.unwrap_timestamp_precision();
539                Some(move |e: HirScalarExpr| e.call_unary(CastTimestampTzToTimestamp(func::CastTimestampTzToTimestamp{from, to})))
540            }),
541            (TimestampTz, TimestampTz) => Assignment: CastTemplate::new(|_ecx, _ccx, from_type, to_type| {
542                let from = from_type.unwrap_timestamp_precision();
543                let to = to_type.unwrap_timestamp_precision();
544                Some(move |e: HirScalarExpr| e.call_unary(AdjustTimestampTzPrecision(func::AdjustTimestampTzPrecision{from, to})))
545            }),
546            (TimestampTz, Time) => Assignment: CastTimestampTzToTime(func::CastTimestampTzToTime),
547            (TimestampTz, String) => Assignment: CastTimestampTzToString(func::CastTimestampTzToString),
548
549            // INTERVAL
550            (Interval, Time) => Assignment: CastIntervalToTime(func::CastIntervalToTime),
551            (Interval, String) => Assignment: CastIntervalToString(func::CastIntervalToString),
552
553            // BYTES
554            (Bytes, String) => Assignment: CastBytesToString(func::CastBytesToString),
555
556            // STRING
557            (String, Bool) => Explicit: CastStringToBool(func::CastStringToBool),
558            (String, Int16) => Explicit: CastStringToInt16(func::CastStringToInt16),
559            (String, Int32) => Explicit: CastStringToInt32(func::CastStringToInt32),
560            (String, Int64) => Explicit: CastStringToInt64(func::CastStringToInt64),
561            (String, UInt16) => Explicit: CastStringToUint16(func::CastStringToUint16),
562            (String, UInt32) => Explicit: CastStringToUint32(func::CastStringToUint32),
563            (String, UInt64) => Explicit: CastStringToUint64(func::CastStringToUint64),
564            (String, Oid) => Explicit: CastStringToOid(func::CastStringToOid),
565
566            // STRING to REG*
567            // A reg* type represents a specific type of object by oid.
568            // Converting from string to reg* does a lookup of the object name
569            // in the corresponding mz_catalog table and expects exactly one object to match it.
570            // You can also specify (in postgres) a string that's a valid
571            // int4 and it'll happily cast it (without verifying that the int4 matches
572            // an object oid).
573            // TODO: Support the correct error code for does not exist (42883).
574            (String, RegClass) => Explicit: sql_impl_cast_per_context(
575                &[
576                    (CastContext::Explicit, &STRING_TO_REGCLASS_EXPLICIT),
577                    (CastContext::Coerced, &STRING_TO_REGCLASS_COERCED)
578                ]
579            ),
580            (String, RegProc) => Explicit: sql_impl_cast(&STRING_TO_REGPROC),
581            (String, RegType) => Explicit: sql_impl_cast(&STRING_TO_REGTYPE),
582
583            (String, Float32) => Explicit: CastStringToFloat32(func::CastStringToFloat32),
584            (String, Float64) => Explicit: CastStringToFloat64(func::CastStringToFloat64),
585            (String, Numeric) => Explicit: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
586                let s = to_type.unwrap_numeric_max_scale();
587                Some(move |e: HirScalarExpr| e.call_unary(CastStringToNumeric(func::CastStringToNumeric(s))))
588            }),
589            (String, Date) => Explicit: CastStringToDate(func::CastStringToDate),
590            (String, Time) => Explicit: CastStringToTime(func::CastStringToTime),
591            (String, Timestamp) => Explicit: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
592                let p = to_type.unwrap_timestamp_precision();
593                Some(move |e: HirScalarExpr| e.call_unary(CastStringToTimestamp(func::CastStringToTimestamp(p))))
594            }),
595            (String, TimestampTz) => Explicit: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
596                let p = to_type.unwrap_timestamp_precision();
597                Some(move |e: HirScalarExpr| e.call_unary(CastStringToTimestampTz(func::CastStringToTimestampTz(p))))
598            }),
599            (String, Interval) => Explicit: CastStringToInterval(func::CastStringToInterval),
600            (String, Bytes) => Explicit: CastStringToBytes(func::CastStringToBytes),
601            (String, Jsonb) => Explicit: CastStringToJsonb(func::CastStringToJsonb),
602            (String, Uuid) => Explicit: CastStringToUuid(func::CastStringToUuid),
603            (String, Array) => Explicit: CastTemplate::new(|ecx, ccx, from_type, to_type| {
604                let return_ty = to_type.clone();
605                let to_el_type = to_type.unwrap_array_element_type();
606                let cast_expr = plan_hypothetical_cast(ecx, ccx, from_type, to_el_type)?;
607                Some(|e: HirScalarExpr| e.call_unary(UnaryFunc::CastStringToArray(func::CastStringToArray {
608                    return_ty,
609                    cast_expr: Box::new(cast_expr),
610                })))
611            }),
612            (String, List) => Explicit: CastTemplate::new(|ecx, ccx, from_type, to_type| {
613                let return_ty = to_type.clone();
614                let to_el_type = to_type.unwrap_list_element_type();
615                let cast_expr = plan_hypothetical_cast(ecx, ccx, from_type, to_el_type)?;
616                Some(|e: HirScalarExpr| e.call_unary(UnaryFunc::CastStringToList(func::CastStringToList {
617                    return_ty,
618                    cast_expr: Box::new(cast_expr),
619                })))
620            }),
621            (String, Map) => Explicit: CastTemplate::new(|ecx, ccx, from_type, to_type| {
622                let return_ty = to_type.clone();
623                let to_val_type = to_type.unwrap_map_value_type();
624                let cast_expr = plan_hypothetical_cast(ecx, ccx, from_type, to_val_type)?;
625                Some(|e: HirScalarExpr| e.call_unary(UnaryFunc::CastStringToMap(func::CastStringToMap {
626                    return_ty,
627                    cast_expr: Box::new(cast_expr),
628                })))
629            }),
630            (String, Range) => Explicit: CastTemplate::new(|ecx, ccx, from_type, to_type| {
631                let return_ty = to_type.clone();
632                let to_el_type = to_type.unwrap_range_element_type();
633                let cast_expr = plan_hypothetical_cast(ecx, ccx, from_type, to_el_type)?;
634                Some(|e: HirScalarExpr| e.call_unary(UnaryFunc::CastStringToRange(func::CastStringToRange {
635                    return_ty,
636                    cast_expr: Box::new(cast_expr),
637                })))
638            }),
639            (String, Int2Vector) => Explicit: CastStringToInt2Vector(func::CastStringToInt2Vector),
640            (String, Char) => Implicit: CastTemplate::new(|_ecx, ccx, _from_type, to_type| {
641                let length = to_type.unwrap_char_length();
642                Some(move |e: HirScalarExpr| e.call_unary(CastStringToChar(func::CastStringToChar {length, fail_on_len: ccx != CastContext::Explicit})))
643            }),
644            (String, VarChar) => Implicit: CastTemplate::new(|_ecx, ccx, _from_type, to_type| {
645                let length = to_type.unwrap_varchar_max_length();
646                Some(move |e: HirScalarExpr| e.call_unary(CastStringToVarChar(func::CastStringToVarChar {length, fail_on_len: ccx != CastContext::Explicit})))
647            }),
648            (String, PgLegacyChar) => Assignment: CastStringToPgLegacyChar(func::CastStringToPgLegacyChar),
649            // CHAR
650            (Char, String) => Implicit: CastCharToString(func::CastCharToString),
651            (Char, Char) => Implicit: CastTemplate::new(|_ecx, ccx, _from_type, to_type| {
652                let length = to_type.unwrap_char_length();
653                Some(move |e: HirScalarExpr| e.call_unary(CastStringToChar(func::CastStringToChar {length, fail_on_len: ccx != CastContext::Explicit})))
654            }),
655            (Char, VarChar) => Implicit: CastTemplate::new(|_ecx, ccx, _from_type, to_type| {
656                let length = to_type.unwrap_varchar_max_length();
657                Some(move |e: HirScalarExpr| e.call_unary(CastStringToVarChar(func::CastStringToVarChar {length, fail_on_len: ccx != CastContext::Explicit})))
658            }),
659            (Char, PgLegacyChar) => Assignment: CastStringToPgLegacyChar(func::CastStringToPgLegacyChar),
660
661            // VARCHAR
662            (VarChar, String) => Implicit: CastVarCharToString(func::CastVarCharToString),
663            (VarChar, Char) => Implicit: CastTemplate::new(|_ecx, ccx, _from_type, to_type| {
664                let length = to_type.unwrap_char_length();
665                Some(move |e: HirScalarExpr| e.call_unary(CastStringToChar(func::CastStringToChar {length, fail_on_len: ccx != CastContext::Explicit})))
666            }),
667            (VarChar, VarChar) => Implicit: CastTemplate::new(|_ecx, ccx, _from_type, to_type| {
668                let length = to_type.unwrap_varchar_max_length();
669                Some(move |e: HirScalarExpr| e.call_unary(CastStringToVarChar(func::CastStringToVarChar {length, fail_on_len: ccx != CastContext::Explicit})))
670            }),
671            (VarChar, PgLegacyChar) => Assignment: CastStringToPgLegacyChar(func::CastStringToPgLegacyChar),
672
673            // PG LEGACY CHAR
674            (PgLegacyChar, String) => Implicit: CastPgLegacyCharToString(func::CastPgLegacyCharToString),
675            (PgLegacyChar, Char) => Assignment: CastPgLegacyCharToChar(func::CastPgLegacyCharToChar),
676            (PgLegacyChar, VarChar) => Assignment: CastPgLegacyCharToVarChar(func::CastPgLegacyCharToVarChar),
677            (PgLegacyChar, Int32) => Explicit: CastPgLegacyCharToInt32(func::CastPgLegacyCharToInt32),
678
679            // PG LEGACY NAME
680            // Under the hood VarChars and Name's are just Strings, so we can re-use existing methods
681            // on Strings and VarChars instead of defining new ones.
682            (PgLegacyName, String) => Implicit: CastVarCharToString(func::CastVarCharToString),
683            (PgLegacyName, Char) => Assignment: CastTemplate::new(|_ecx, ccx, _from_type, to_type| {
684                let length = to_type.unwrap_char_length();
685                Some(move |e: HirScalarExpr| e.call_unary(CastStringToChar(func::CastStringToChar {length, fail_on_len: ccx != CastContext::Explicit})))
686            }),
687            (PgLegacyName, VarChar) => Assignment: CastTemplate::new(|_ecx, ccx, _from_type, to_type| {
688                let length = to_type.unwrap_varchar_max_length();
689                Some(move |e: HirScalarExpr| e.call_unary(CastStringToVarChar(func::CastStringToVarChar {length, fail_on_len: ccx != CastContext::Explicit})))
690            }),
691            (String, PgLegacyName) => Implicit: CastStringToPgLegacyName(func::CastStringToPgLegacyName),
692            (Char, PgLegacyName) => Implicit: CastStringToPgLegacyName(func::CastStringToPgLegacyName),
693            (VarChar, PgLegacyName) => Implicit: CastStringToPgLegacyName(func::CastStringToPgLegacyName),
694
695            // RECORD
696            (Record, String) => Assignment: CastTemplate::new(|_ecx, _ccx, from_type, _to_type| {
697                let ty = from_type.clone();
698                Some(|e: HirScalarExpr| e.call_unary(CastRecordToString(func::CastRecordToString { ty })))
699            }),
700            (Record, Record) => Implicit: CastTemplate::new(|ecx, ccx, from_type, to_type| {
701                if from_type.unwrap_record_element_type().len() != to_type.unwrap_record_element_type().len() {
702                    return None;
703                }
704
705                if let (l @ SqlScalarType::Record {custom_id: Some(..), ..}, r) = (from_type, to_type) {
706                    // Changing `from`'s custom_id requires at least Assignment context
707                    if ccx == CastContext::Implicit && l != r {
708                        return None;
709                    }
710                }
711
712                let cast_exprs = from_type.unwrap_record_element_type()
713                    .iter()
714                    .zip_eq(to_type.unwrap_record_element_type())
715                    .map(|(f, t)| plan_hypothetical_cast(ecx, ccx, f, t))
716                    .collect::<Option<Box<_>>>()?;
717                let to = to_type.clone();
718                Some(|e: HirScalarExpr| e.call_unary(CastRecord1ToRecord2(func::CastRecord1ToRecord2 { return_ty: to, cast_exprs })))
719            }),
720
721            // ARRAY
722            (Array, String) => Assignment: CastTemplate::new(|_ecx, _ccx, from_type, _to_type| {
723                let ty = from_type.clone();
724                Some(|e: HirScalarExpr| e.call_unary(CastArrayToString(func::CastArrayToString { ty })))
725            }),
726            (Array, List) => Explicit: CastArrayToListOneDim(func::CastArrayToListOneDim),
727            (Array, Array) => Explicit: CastTemplate::new(|ecx, ccx, from_type, to_type| {
728                let inner_from_type = from_type.unwrap_array_element_type();
729                let inner_to_type = to_type.unwrap_array_element_type();
730                let cast_expr = plan_hypothetical_cast(ecx, ccx, inner_from_type, inner_to_type)?;
731                let return_ty = to_type.clone();
732
733                Some(move |e: HirScalarExpr| e.call_unary(CastArrayToArray(func::CastArrayToArray { return_ty, cast_expr: Box::new(cast_expr) })))
734            }),
735
736            // INT2VECTOR
737            (Int2Vector, Array) => Implicit: CastTemplate::new(|_ecx, _ccx, _from_type, _to_type| {
738                Some(|e: HirScalarExpr| e.call_unary(UnaryFunc::CastInt2VectorToArray(func::CastInt2VectorToArray)))
739            }),
740            (Int2Vector, String) => Explicit: CastInt2VectorToString(func::CastInt2VectorToString),
741
742            // LIST
743            (List, String) => Assignment: CastTemplate::new(|_ecx, _ccx, from_type, _to_type| {
744                let ty = from_type.clone();
745                Some(|e: HirScalarExpr| e.call_unary(CastListToString(func::CastListToString { ty })))
746            }),
747            (List, List) => Implicit: CastTemplate::new(|ecx, ccx, from_type, to_type| {
748
749                if let (l @ SqlScalarType::List {custom_id: Some(..), ..}, r) = (from_type, to_type) {
750                    // Changing `from`'s custom_id requires at least Assignment context
751                    if ccx == CastContext::Implicit && !l.base_eq(r) {
752                        return None;
753                    }
754                }
755
756                let return_ty = to_type.clone();
757                let from_el_type = from_type.unwrap_list_element_type();
758                let to_el_type = to_type.unwrap_list_element_type();
759                let cast_expr = plan_hypothetical_cast(ecx, ccx, from_el_type, to_el_type)?;
760                Some(|e: HirScalarExpr| e.call_unary(UnaryFunc::CastList1ToList2(func::CastList1ToList2 {
761                    return_ty,
762                    cast_expr: Box::new(cast_expr),
763                })))
764            }),
765
766            // MAP
767            (Map, String) => Assignment: CastTemplate::new(|_ecx, _ccx, from_type, _to_type| {
768                let ty = from_type.clone();
769                Some(|e: HirScalarExpr| e.call_unary(CastMapToString(func::CastMapToString { ty })))
770            }),
771
772            // JSONB
773            (Jsonb, Bool) => Explicit: CastJsonbToBool(func::CastJsonbToBool),
774            (Jsonb, Int16) => Explicit: CastJsonbToInt16(func::CastJsonbToInt16),
775            (Jsonb, Int32) => Explicit: CastJsonbToInt32(func::CastJsonbToInt32),
776            (Jsonb, Int64) => Explicit: CastJsonbToInt64(func::CastJsonbToInt64),
777            (Jsonb, Float32) => Explicit: CastJsonbToFloat32(func::CastJsonbToFloat32),
778            (Jsonb, Float64) => Explicit: CastJsonbToFloat64(func::CastJsonbToFloat64),
779            (Jsonb, Numeric) => Explicit: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
780                let s = to_type.unwrap_numeric_max_scale();
781                Some(move |e: HirScalarExpr| e.call_unary(CastJsonbToNumeric(func::CastJsonbToNumeric(s))))
782            }),
783            (Jsonb, String) => Assignment: CastJsonbToString(func::CastJsonbToString),
784
785            // UUID
786            (Uuid, String) => Assignment: CastUuidToString(func::CastUuidToString),
787
788            // Numeric
789            (Numeric, Numeric) => Assignment: CastTemplate::new(|_ecx, _ccx, _from_type, to_type| {
790                let scale = to_type.unwrap_numeric_max_scale();
791                Some(move |e: HirScalarExpr| match scale {
792                    None => e,
793                    Some(scale) => e.call_unary(UnaryFunc::AdjustNumericScale(func::AdjustNumericScale(scale))),
794                })
795            }),
796            (Numeric, Float32) => Implicit: CastNumericToFloat32(func::CastNumericToFloat32),
797            (Numeric, Float64) => Implicit: CastNumericToFloat64(func::CastNumericToFloat64),
798            (Numeric, Int16) => Assignment: CastNumericToInt16(func::CastNumericToInt16),
799            (Numeric, Int32) => Assignment: CastNumericToInt32(func::CastNumericToInt32),
800            (Numeric, Int64) => Assignment: CastNumericToInt64(func::CastNumericToInt64),
801            (Numeric, UInt16) => Assignment: CastNumericToUint16(func::CastNumericToUint16),
802            (Numeric, UInt32) => Assignment: CastNumericToUint32(func::CastNumericToUint32),
803            (Numeric, UInt64) => Assignment: CastNumericToUint64(func::CastNumericToUint64),
804            (Numeric, String) => Assignment: CastNumericToString(func::CastNumericToString),
805
806            // Range
807            (Range, String) => Assignment: CastTemplate::new(|_ecx, _ccx, from_type, _to_type| {
808                let ty = from_type.clone();
809                Some(|e: HirScalarExpr| e.call_unary(CastRangeToString(func::CastRangeToString { ty })))
810            }),
811
812            // MzAclItem
813            (MzAclItem, String) => Explicit: sql_impl_cast("(
814                SELECT
815                    (CASE
816                        WHEN grantee_role_id = 'p' THEN ''
817                        ELSE COALESCE(grantee_role.name, grantee_role_id)
818                    END)
819                    || '='
820                    || mz_internal.mz_aclitem_privileges($1)
821                    || '/'
822                    || COALESCE(grantor_role.name, grantor_role_id)
823                FROM
824                    (SELECT mz_internal.mz_aclitem_grantee($1) AS grantee_role_id),
825                    (SELECT mz_internal.mz_aclitem_grantor($1) AS grantor_role_id)
826                LEFT JOIN mz_catalog.mz_roles AS grantee_role ON grantee_role_id = grantee_role.id
827                LEFT JOIN mz_catalog.mz_roles AS grantor_role ON grantor_role_id = grantor_role.id
828            )"),
829            (MzAclItem, AclItem) => Explicit: sql_impl_cast("(
830                SELECT makeaclitem(
831                    (CASE mz_internal.mz_aclitem_grantee($1)
832                        WHEN 'p' THEN 0
833                        ELSE (SELECT oid FROM mz_catalog.mz_roles WHERE id = mz_internal.mz_aclitem_grantee($1))
834                    END),
835                    (SELECT oid FROM mz_catalog.mz_roles WHERE id = mz_internal.mz_aclitem_grantor($1)),
836                    (SELECT array_to_string(mz_internal.mz_format_privileges(mz_internal.mz_aclitem_privileges($1)), ',')),
837                    -- GRANT OPTION isn't implemented so we hardcode false.
838                    false
839                )
840            )"),
841
842            // AclItem
843            (AclItem, String) => Explicit: sql_impl_cast("(
844                SELECT
845                    (CASE grantee_oid
846                        WHEN 0 THEN ''
847                        ELSE COALESCE(grantee_role.name, grantee_oid::text)
848                    END)
849                    || '='
850                    || mz_internal.aclitem_privileges($1)
851                    || '/'
852                    || COALESCE(grantor_role.name, grantor_oid::text)
853                FROM
854                    (SELECT mz_internal.aclitem_grantee($1) AS grantee_oid),
855                    (SELECT mz_internal.aclitem_grantor($1) AS grantor_oid)
856                LEFT JOIN mz_catalog.mz_roles AS grantee_role ON grantee_oid = grantee_role.oid
857                LEFT JOIN mz_catalog.mz_roles AS grantor_role ON grantor_oid = grantor_role.oid
858            )"),
859            (AclItem, MzAclItem) => Explicit: sql_impl_cast("(
860                SELECT mz_internal.make_mz_aclitem(
861                    (CASE mz_internal.aclitem_grantee($1)
862                        WHEN 0 THEN 'p'
863                        ELSE (SELECT id FROM mz_catalog.mz_roles WHERE oid = mz_internal.aclitem_grantee($1))
864                    END),
865                    (SELECT id FROM mz_catalog.mz_roles WHERE oid = mz_internal.aclitem_grantor($1)),
866                    (SELECT array_to_string(mz_internal.mz_format_privileges(mz_internal.aclitem_privileges($1)), ','))
867                )
868            )")
869        }
870    });
871
872/// Get casts directly between two [`SqlScalarType`]s, with control over the
873/// allowed [`CastContext`].
874fn get_cast(
875    ecx: &ExprContext,
876    ccx: CastContext,
877    from: &SqlScalarType,
878    to: &SqlScalarType,
879) -> Option<Cast> {
880    use CastContext::*;
881
882    if from == to || (ccx == Implicit && from.base_eq(to)) {
883        return Some(Box::new(|expr| expr));
884    }
885
886    let imp = VALID_CASTS.get(&(from.into(), to.into()))?;
887    let template = if ccx >= imp.context {
888        Some(&imp.template)
889    } else {
890        None
891    };
892    template.and_then(|template| (template.0)(ecx, ccx, from, to))
893}
894
895/// Converts an expression to `SqlScalarType::String`.
896///
897/// All types are convertible to string, so this never fails.
898pub fn to_string(ecx: &ExprContext, expr: HirScalarExpr) -> HirScalarExpr {
899    plan_cast(ecx, CastContext::Explicit, expr, &SqlScalarType::String)
900        .expect("cast known to exist")
901}
902
903/// Converts an expression to `SqlScalarType::Jsonb`.
904///
905/// The rules are as follows:
906///   * `SqlScalarType::Boolean`s become JSON booleans.
907///   * All numeric types are converted to `Float64`s, then become JSON numbers.
908///   * Records are converted to a JSON object where the record's field names
909///     are the keys of the object, and the record's fields are recursively
910///     converted to JSON by `to_jsonb`.
911///   * Other types are converted to strings by their usual cast function an
912//      become JSON strings.
913pub fn to_jsonb(ecx: &ExprContext, expr: HirScalarExpr) -> HirScalarExpr {
914    use SqlScalarType::*;
915
916    match ecx.scalar_type(&expr) {
917        Bool | Jsonb | Numeric { .. } => {
918            expr.call_unary(UnaryFunc::CastJsonbableToJsonb(func::CastJsonbableToJsonb))
919        }
920        Int16 | Int32 | Int64 | UInt16 | UInt32 | UInt64 | Float32 | Float64 => plan_cast(
921            ecx,
922            CastContext::Explicit,
923            expr,
924            &Numeric { max_scale: None },
925        )
926        .expect("cast known to exist")
927        .call_unary(UnaryFunc::CastJsonbableToJsonb(func::CastJsonbableToJsonb)),
928        Record { fields, .. } => {
929            let mut exprs = vec![];
930            for (i, (name, _ty)) in fields.iter().enumerate() {
931                exprs.push(HirScalarExpr::literal(
932                    Datum::String(name),
933                    SqlScalarType::String,
934                ));
935                exprs.push(to_jsonb(
936                    ecx,
937                    expr.clone()
938                        .call_unary(UnaryFunc::RecordGet(func::RecordGet(i))),
939                ));
940            }
941            HirScalarExpr::call_variadic(VariadicFunc::JsonbBuildObject, exprs)
942        }
943        ref ty @ List {
944            ref element_type, ..
945        }
946        | ref ty @ Array(ref element_type) => {
947            // Construct a new expression context with one column whose type
948            // is the container's element type.
949            let qcx = QueryContext::root(ecx.qcx.scx, ecx.qcx.lifetime);
950            let ecx = ExprContext {
951                qcx: &qcx,
952                name: "to_jsonb",
953                scope: &Scope::empty(),
954                relation_type: &SqlRelationType::new(vec![element_type.clone().nullable(true)]),
955                allow_aggregates: false,
956                allow_subqueries: false,
957                allow_parameters: false,
958                allow_windows: false,
959            };
960
961            // Create an element-casting expression by calling `to_jsonb` on
962            // an expression that references the first column in a row.
963            let cast_element = to_jsonb(&ecx, HirScalarExpr::column(0));
964            let cast_element = cast_element
965                .lower_uncorrelated()
966                .expect("to_jsonb does not produce correlated expressions on uncorrelated input");
967
968            // The `Cast{Array|List}ToJsonb` functions take the element-casting
969            // expression as an argument and evaluate the expression against
970            // each element of the container at runtime.
971            let func = match ty {
972                List { .. } => UnaryFunc::CastListToJsonb(CastListToJsonb {
973                    cast_element: Box::new(cast_element),
974                }),
975                Array { .. } => UnaryFunc::CastArrayToJsonb(CastArrayToJsonb {
976                    cast_element: Box::new(cast_element),
977                }),
978                _ => unreachable!("validated above"),
979            };
980
981            expr.call_unary(func)
982        }
983        Date
984        | Time
985        | Timestamp { .. }
986        | TimestampTz { .. }
987        | Interval
988        | PgLegacyChar
989        | PgLegacyName
990        | Bytes
991        | String
992        | Char { .. }
993        | VarChar { .. }
994        | Uuid
995        | Oid
996        | Map { .. }
997        | RegProc
998        | RegType
999        | RegClass
1000        | Int2Vector
1001        | MzTimestamp
1002        | Range { .. }
1003        | MzAclItem
1004        | AclItem => to_string(ecx, expr)
1005            .call_unary(UnaryFunc::CastJsonbableToJsonb(func::CastJsonbableToJsonb)),
1006    }
1007}
1008
1009/// Guesses the most-common type among a set of [`SqlScalarType`]s that all members
1010/// can be cast to. Returns `None` if a common type cannot be deduced.
1011///
1012/// Note that this function implements the type-determination components of
1013/// Postgres' ["`UNION`, `CASE`, and Related Constructs"][union-type-conv] type
1014/// conversion.
1015///
1016/// [union-type-conv]: https://www.postgresql.org/docs/12/typeconv-union-case.html
1017pub fn guess_best_common_type(
1018    ecx: &ExprContext,
1019    types: &[CoercibleScalarType],
1020) -> Result<SqlScalarType, PlanError> {
1021    // This function is a translation of `select_common_type` in PostgreSQL with
1022    // the addition of our near match logic, which supports Materialize
1023    // non-linear type promotions.
1024    // https://github.com/postgres/postgres/blob/d1b307eef/src/backend/parser/parse_coerce.c#L1288-L1308
1025
1026    // If every type is a literal record with the same number of fields, the
1027    // best common type is a record with that number of fields. We recursively
1028    // guess the best type for each field.
1029    if let Some(CoercibleScalarType::Record(field_tys)) = types.first() {
1030        if types
1031            .iter()
1032            .all(|t| matches!(t, CoercibleScalarType::Record(fts) if field_tys.len() == fts.len()))
1033        {
1034            let mut fields = vec![];
1035            for i in 0..field_tys.len() {
1036                let name = ColumnName::from(format!("f{}", fields.len() + 1));
1037                let mut guesses = vec![];
1038                let mut nullable = false;
1039                for ty in types {
1040                    let field_ty = match ty {
1041                        CoercibleScalarType::Record(fts) => fts[i].clone(),
1042                        _ => unreachable!(),
1043                    };
1044                    if field_ty.nullable() {
1045                        nullable = true;
1046                    }
1047                    guesses.push(field_ty.scalar_type());
1048                }
1049                let guess = guess_best_common_type(ecx, &guesses)?;
1050                fields.push((name, guess.nullable(nullable)));
1051            }
1052            return Ok(SqlScalarType::Record {
1053                fields: fields.into(),
1054                custom_id: None,
1055            });
1056        }
1057    }
1058
1059    // Remove unknown types, and collect them.
1060    let mut types: Vec<_> = types.into_iter().filter_map(|v| v.as_coerced()).collect();
1061
1062    // In the case of mixed ints and uints, replace uints with their near match
1063    let contains_int = types.iter().any(|t| {
1064        matches!(
1065            t,
1066            SqlScalarType::Int16 | SqlScalarType::Int32 | SqlScalarType::Int64
1067        )
1068    });
1069
1070    for t in types.iter_mut() {
1071        if contains_int
1072            && matches!(
1073                t,
1074                SqlScalarType::UInt16 | SqlScalarType::UInt32 | SqlScalarType::UInt64
1075            )
1076        {
1077            *t = t.near_match().expect("unsigned ints have near matches")
1078        }
1079    }
1080
1081    let mut types = types.iter();
1082
1083    let mut candidate = match types.next() {
1084        // If no known types, fall back to `String`.
1085        None => return Ok(SqlScalarType::String),
1086        // Start by guessing the first type.
1087        Some(t) => t,
1088    };
1089
1090    let preferred_type = TypeCategory::from_type(candidate).preferred_type();
1091
1092    for typ in types {
1093        if TypeCategory::from_type(candidate) != TypeCategory::from_type(typ) {
1094            // The next type is in a different category; give up.
1095            sql_bail!(
1096                "{} types {} and {} cannot be matched",
1097                ecx.name,
1098                ecx.humanize_scalar_type(candidate, false),
1099                ecx.humanize_scalar_type(typ, false),
1100            );
1101        };
1102
1103        // If this type is the preferred type, make it the candidate.
1104        if preferred_type.as_ref() != Some(candidate)
1105            && can_cast(ecx, CastContext::Implicit, candidate, typ)
1106            && !can_cast(ecx, CastContext::Implicit, typ, candidate)
1107        {
1108            // The current candidate is not the preferred type for its category
1109            // and the next type is implicitly convertible to the current
1110            // candidate, but not vice-versa, so take the next type as the new
1111            // candidate.
1112            candidate = typ;
1113        }
1114    }
1115    Ok(candidate.without_modifiers())
1116}
1117
1118pub fn plan_coerce<'a>(
1119    ecx: &'a ExprContext,
1120    e: CoercibleScalarExpr,
1121    coerce_to: &SqlScalarType,
1122) -> Result<HirScalarExpr, PlanError> {
1123    use CoercibleScalarExpr::*;
1124
1125    Ok(match e {
1126        Coerced(e) => e,
1127
1128        LiteralNull => HirScalarExpr::literal_null(coerce_to.clone()),
1129
1130        LiteralString(s) => {
1131            let lit = HirScalarExpr::literal(Datum::String(&s), SqlScalarType::String);
1132            // Per PostgreSQL, string literal explicitly casts to the base type.
1133            // The caller is responsible for applying any desired modifiers
1134            // (with either implicit or explicit semantics) via a separate call
1135            // to `plan_cast`.
1136            let coerce_to_base = &coerce_to.without_modifiers();
1137            plan_cast(ecx, CastContext::Coerced, lit, coerce_to_base)?
1138        }
1139
1140        LiteralRecord(exprs) => {
1141            let arity = exprs.len();
1142            let coercions = match coerce_to {
1143                SqlScalarType::Record { fields, .. } if fields.len() == arity => fields
1144                    .iter()
1145                    .map(|(_name, ty)| &ty.scalar_type)
1146                    .cloned()
1147                    .collect(),
1148                _ => vec![SqlScalarType::String; exprs.len()],
1149            };
1150            let mut out = vec![];
1151            for (e, coerce_to) in exprs.into_iter().zip(coercions) {
1152                out.push(plan_coerce(ecx, e, &coerce_to)?);
1153            }
1154            HirScalarExpr::call_variadic(
1155                VariadicFunc::RecordCreate {
1156                    field_names: (0..arity)
1157                        .map(|i| ColumnName::from(format!("f{}", i + 1)))
1158                        .collect(),
1159                },
1160                out,
1161            )
1162        }
1163
1164        Parameter(n) => {
1165            let prev = ecx.param_types().borrow_mut().insert(n, coerce_to.clone());
1166            if let Some(prev) = prev {
1167                if prev != *coerce_to {
1168                    sql_bail!(
1169                        "there are contradicting constraints for the type of parameter ${}: should be both {} and {}",
1170                        n,
1171                        ecx.humanize_scalar_type(&prev, false),
1172                        ecx.humanize_scalar_type(coerce_to, false),
1173                    );
1174                }
1175            }
1176            HirScalarExpr::parameter(n)
1177        }
1178    })
1179}
1180
1181/// Similar to `plan_cast`, but for situations where you only know the type of
1182/// the input expression (`from`) and not the expression itself. The returned
1183/// expression refers to the first column of some imaginary row, where the first
1184/// column is assumed to have type `from`.
1185///
1186/// If casting from `from` to `to` is not possible, returns `None`.
1187pub fn plan_hypothetical_cast(
1188    ecx: &ExprContext,
1189    ccx: CastContext,
1190    from: &SqlScalarType,
1191    to: &SqlScalarType,
1192) -> Option<mz_expr::MirScalarExpr> {
1193    // Reconstruct an expression context where the expression is evaluated on
1194    // the "first column" of some imaginary row.
1195    let mut scx = ecx.qcx.scx.clone();
1196    scx.param_types = RefCell::new(BTreeMap::new());
1197    let qcx = QueryContext::root(&scx, ecx.qcx.lifetime);
1198    let relation_type = SqlRelationType {
1199        column_types: vec![SqlColumnType {
1200            nullable: true,
1201            scalar_type: from.clone(),
1202        }],
1203        keys: vec![vec![0]],
1204    };
1205    let ecx = ExprContext {
1206        qcx: &qcx,
1207        name: "plan_hypothetical_cast",
1208        scope: &Scope::empty(),
1209        relation_type: &relation_type,
1210        allow_aggregates: false,
1211        allow_subqueries: true,
1212        allow_parameters: true,
1213        allow_windows: false,
1214    };
1215
1216    let col_expr = HirScalarExpr::column(0);
1217
1218    // Determine the `ScalarExpr` required to cast our column to the target
1219    // component type.
1220    plan_cast(&ecx, ccx, col_expr, to)
1221        .ok()?
1222        // TODO(jkosh44) Support casts that have correlated implementations.
1223        .lower_uncorrelated()
1224        .ok()
1225}
1226
1227/// Plans a cast between [`SqlScalarType`]s, specifying which types of casts are
1228/// permitted using [`CastContext`].
1229///
1230/// # Errors
1231///
1232/// If a cast between the `ScalarExpr`'s base type and the specified type is:
1233/// - Not possible, e.g. `Bytes` to `Interval`
1234/// - Not permitted, e.g. implicitly casting from `Float64` to `Float32`.
1235/// - Not implemented yet
1236pub fn plan_cast(
1237    ecx: &ExprContext,
1238    ccx: CastContext,
1239    expr: HirScalarExpr,
1240    to: &SqlScalarType,
1241) -> Result<HirScalarExpr, PlanError> {
1242    let from = ecx.scalar_type(&expr);
1243
1244    // Close over `ccx`, `from`, and `to` to simplify error messages in the
1245    // face of intermediate expressions.
1246    let cast_inner = |from, to, expr| match get_cast(ecx, ccx, from, to) {
1247        Some(cast) => Ok(cast(expr)),
1248        None => Err(PlanError::InvalidCast {
1249            name: ecx.name.into(),
1250            ccx,
1251            from: ecx.humanize_scalar_type(from, false),
1252            to: ecx.humanize_scalar_type(to, false),
1253        }),
1254    };
1255
1256    // Get cast which might include parameter rewrites + generating intermediate
1257    // expressions.
1258    //
1259    // String-like types get special handling to match PostgreSQL.
1260    // See: https://github.com/postgres/postgres/blob/6b04abdfc/src/backend/parser/parse_coerce.c#L3205-L3223
1261    let from_category = TypeCategory::from_type(&from);
1262    let to_category = TypeCategory::from_type(to);
1263    if from_category == TypeCategory::String && to_category != TypeCategory::String {
1264        // Converting from stringlike to something non-stringlike. Handle as if
1265        // `from` were a `SqlScalarType::String.
1266        cast_inner(&SqlScalarType::String, to, expr)
1267    } else if from_category != TypeCategory::String && to_category == TypeCategory::String {
1268        // Converting from non-stringlike to something stringlike. Convert to a
1269        // `SqlScalarType::String` and then to the desired type.
1270        let expr = cast_inner(&from, &SqlScalarType::String, expr)?;
1271        cast_inner(&SqlScalarType::String, to, expr)
1272    } else {
1273        // Standard cast.
1274        cast_inner(&from, to, expr)
1275    }
1276}
1277
1278/// Reports whether it is possible to perform a cast from the specified types.
1279pub fn can_cast(
1280    ecx: &ExprContext,
1281    ccx: CastContext,
1282    cast_from: &SqlScalarType,
1283    cast_to: &SqlScalarType,
1284) -> bool {
1285    get_cast(ecx, ccx, cast_from, cast_to).is_some()
1286}