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