Edit on GitHub

sqlglot.optimizer.annotate_types

  1from __future__ import annotations
  2
  3import functools
  4import logging
  5import typing as t
  6
  7from sqlglot import exp
  8from sqlglot.helper import (
  9    ensure_list,
 10    is_date_unit,
 11    is_iso_date,
 12    is_iso_datetime,
 13    seq_get,
 14)
 15from sqlglot.optimizer.scope import Scope, traverse_scope
 16from sqlglot.schema import MappingSchema, Schema, ensure_schema
 17from sqlglot.dialects.dialect import Dialect
 18
 19if t.TYPE_CHECKING:
 20    from sqlglot._typing import B, E
 21
 22    BinaryCoercionFunc = t.Callable[[exp.Expression, exp.Expression], exp.DataType.Type]
 23    BinaryCoercions = t.Dict[
 24        t.Tuple[exp.DataType.Type, exp.DataType.Type],
 25        BinaryCoercionFunc,
 26    ]
 27
 28    from sqlglot.dialects.dialect import DialectType, AnnotatorsType
 29
 30logger = logging.getLogger("sqlglot")
 31
 32
 33def annotate_types(
 34    expression: E,
 35    schema: t.Optional[t.Dict | Schema] = None,
 36    annotators: t.Optional[AnnotatorsType] = None,
 37    coerces_to: t.Optional[t.Dict[exp.DataType.Type, t.Set[exp.DataType.Type]]] = None,
 38    dialect: DialectType = None,
 39) -> E:
 40    """
 41    Infers the types of an expression, annotating its AST accordingly.
 42
 43    Example:
 44        >>> import sqlglot
 45        >>> schema = {"y": {"cola": "SMALLINT"}}
 46        >>> sql = "SELECT x.cola + 2.5 AS cola FROM (SELECT y.cola AS cola FROM y AS y) AS x"
 47        >>> annotated_expr = annotate_types(sqlglot.parse_one(sql), schema=schema)
 48        >>> annotated_expr.expressions[0].type.this  # Get the type of "x.cola + 2.5 AS cola"
 49        <Type.DOUBLE: 'DOUBLE'>
 50
 51    Args:
 52        expression: Expression to annotate.
 53        schema: Database schema.
 54        annotators: Maps expression type to corresponding annotation function.
 55        coerces_to: Maps expression type to set of types that it can be coerced into.
 56
 57    Returns:
 58        The expression annotated with types.
 59    """
 60
 61    schema = ensure_schema(schema, dialect=dialect)
 62
 63    return TypeAnnotator(schema, annotators, coerces_to).annotate(expression)
 64
 65
 66def _coerce_date_literal(l: exp.Expression, unit: t.Optional[exp.Expression]) -> exp.DataType.Type:
 67    date_text = l.name
 68    is_iso_date_ = is_iso_date(date_text)
 69
 70    if is_iso_date_ and is_date_unit(unit):
 71        return exp.DataType.Type.DATE
 72
 73    # An ISO date is also an ISO datetime, but not vice versa
 74    if is_iso_date_ or is_iso_datetime(date_text):
 75        return exp.DataType.Type.DATETIME
 76
 77    return exp.DataType.Type.UNKNOWN
 78
 79
 80def _coerce_date(l: exp.Expression, unit: t.Optional[exp.Expression]) -> exp.DataType.Type:
 81    if not is_date_unit(unit):
 82        return exp.DataType.Type.DATETIME
 83    return l.type.this if l.type else exp.DataType.Type.UNKNOWN
 84
 85
 86def swap_args(func: BinaryCoercionFunc) -> BinaryCoercionFunc:
 87    @functools.wraps(func)
 88    def _swapped(l: exp.Expression, r: exp.Expression) -> exp.DataType.Type:
 89        return func(r, l)
 90
 91    return _swapped
 92
 93
 94def swap_all(coercions: BinaryCoercions) -> BinaryCoercions:
 95    return {**coercions, **{(b, a): swap_args(func) for (a, b), func in coercions.items()}}
 96
 97
 98class _TypeAnnotator(type):
 99    def __new__(cls, clsname, bases, attrs):
100        klass = super().__new__(cls, clsname, bases, attrs)
101
102        # Highest-to-lowest type precedence, as specified in Spark's docs (ANSI):
103        # https://spark.apache.org/docs/3.2.0/sql-ref-ansi-compliance.html
104        text_precedence = (
105            exp.DataType.Type.TEXT,
106            exp.DataType.Type.NVARCHAR,
107            exp.DataType.Type.VARCHAR,
108            exp.DataType.Type.NCHAR,
109            exp.DataType.Type.CHAR,
110        )
111        numeric_precedence = (
112            exp.DataType.Type.DOUBLE,
113            exp.DataType.Type.FLOAT,
114            exp.DataType.Type.DECIMAL,
115            exp.DataType.Type.BIGINT,
116            exp.DataType.Type.INT,
117            exp.DataType.Type.SMALLINT,
118            exp.DataType.Type.TINYINT,
119        )
120        timelike_precedence = (
121            exp.DataType.Type.TIMESTAMPLTZ,
122            exp.DataType.Type.TIMESTAMPTZ,
123            exp.DataType.Type.TIMESTAMP,
124            exp.DataType.Type.DATETIME,
125            exp.DataType.Type.DATE,
126        )
127
128        for type_precedence in (text_precedence, numeric_precedence, timelike_precedence):
129            coerces_to = set()
130            for data_type in type_precedence:
131                klass.COERCES_TO[data_type] = coerces_to.copy()
132                coerces_to |= {data_type}
133
134        # NULL can be coerced to any type, so e.g. NULL + 1 will have type INT
135        klass.COERCES_TO[exp.DataType.Type.NULL] = {
136            *text_precedence,
137            *numeric_precedence,
138            *timelike_precedence,
139        }
140
141        return klass
142
143
144class TypeAnnotator(metaclass=_TypeAnnotator):
145    NESTED_TYPES = {
146        exp.DataType.Type.ARRAY,
147    }
148
149    # Specifies what types a given type can be coerced into (autofilled)
150    COERCES_TO: t.Dict[exp.DataType.Type, t.Set[exp.DataType.Type]] = {}
151
152    # Coercion functions for binary operations.
153    # Map of type pairs to a callable that takes both sides of the binary operation and returns the resulting type.
154    BINARY_COERCIONS: BinaryCoercions = {
155        **swap_all(
156            {
157                (t, exp.DataType.Type.INTERVAL): lambda l, r: _coerce_date_literal(
158                    l, r.args.get("unit")
159                )
160                for t in exp.DataType.TEXT_TYPES
161            }
162        ),
163        **swap_all(
164            {
165                # text + numeric will yield the numeric type to match most dialects' semantics
166                (text, numeric): lambda l, r: t.cast(
167                    exp.DataType.Type, l.type if l.type in exp.DataType.NUMERIC_TYPES else r.type
168                )
169                for text in exp.DataType.TEXT_TYPES
170                for numeric in exp.DataType.NUMERIC_TYPES
171            }
172        ),
173        **swap_all(
174            {
175                (exp.DataType.Type.DATE, exp.DataType.Type.INTERVAL): lambda l, r: _coerce_date(
176                    l, r.args.get("unit")
177                ),
178            }
179        ),
180    }
181
182    def __init__(
183        self,
184        schema: Schema,
185        annotators: t.Optional[AnnotatorsType] = None,
186        coerces_to: t.Optional[t.Dict[exp.DataType.Type, t.Set[exp.DataType.Type]]] = None,
187        binary_coercions: t.Optional[BinaryCoercions] = None,
188    ) -> None:
189        self.schema = schema
190        self.annotators = annotators or Dialect.get_or_raise(schema.dialect).ANNOTATORS
191        self.coerces_to = (
192            coerces_to or Dialect.get_or_raise(schema.dialect).COERCES_TO or self.COERCES_TO
193        )
194        self.binary_coercions = binary_coercions or self.BINARY_COERCIONS
195
196        # Caches the ids of annotated sub-Expressions, to ensure we only visit them once
197        self._visited: t.Set[int] = set()
198
199        # Caches NULL-annotated expressions to set them to UNKNOWN after type inference is completed
200        self._null_expressions: t.Dict[int, exp.Expression] = {}
201
202        # Databricks and Spark ≥v3 actually support NULL (i.e., VOID) as a type
203        self._supports_null_type = schema.dialect in ("databricks", "spark")
204
205        # Maps an exp.SetOperation's id (e.g. UNION) to its projection types. This is computed if the
206        # exp.SetOperation is the expression of a scope source, as selecting from it multiple times
207        # would reprocess the entire subtree to coerce the types of its operands' projections
208        self._setop_column_types: t.Dict[int, t.Dict[str, exp.DataType | exp.DataType.Type]] = {}
209
210    def _set_type(
211        self, expression: exp.Expression, target_type: t.Optional[exp.DataType | exp.DataType.Type]
212    ) -> None:
213        prev_type = expression.type
214        expression_id = id(expression)
215
216        expression.type = target_type or exp.DataType.Type.UNKNOWN  # type: ignore
217        self._visited.add(expression_id)
218
219        if (
220            not self._supports_null_type
221            and t.cast(exp.DataType, expression.type).this == exp.DataType.Type.NULL
222        ):
223            self._null_expressions[expression_id] = expression
224        elif prev_type and t.cast(exp.DataType, prev_type).this == exp.DataType.Type.NULL:
225            self._null_expressions.pop(expression_id, None)
226
227    def annotate(self, expression: E) -> E:
228        for scope in traverse_scope(expression):
229            self.annotate_scope(scope)
230
231        # This takes care of non-traversable expressions
232        expression = self._maybe_annotate(expression)
233
234        # Replace NULL type with UNKNOWN, since the former is not an actual type;
235        # it is mostly used to aid type coercion, e.g. in query set operations.
236        for expr in self._null_expressions.values():
237            expr.type = exp.DataType.Type.UNKNOWN
238
239        return expression
240
241    def annotate_scope(self, scope: Scope) -> None:
242        selects = {}
243        for name, source in scope.sources.items():
244            if not isinstance(source, Scope):
245                continue
246
247            expression = source.expression
248            if isinstance(expression, exp.UDTF):
249                values = []
250
251                if isinstance(expression, exp.Lateral):
252                    if isinstance(expression.this, exp.Explode):
253                        values = [expression.this.this]
254                elif isinstance(expression, exp.Unnest):
255                    values = [expression]
256                elif not isinstance(expression, exp.TableFromRows):
257                    values = expression.expressions[0].expressions
258
259                if not values:
260                    continue
261
262                selects[name] = {
263                    alias: column.type
264                    for alias, column in zip(expression.alias_column_names, values)
265                }
266            elif isinstance(expression, exp.SetOperation) and len(expression.left.selects) == len(
267                expression.right.selects
268            ):
269                selects[name] = col_types = self._setop_column_types.setdefault(id(expression), {})
270
271                if not col_types:
272                    # Process a chain / sub-tree of set operations
273                    for set_op in expression.walk(
274                        prune=lambda n: not isinstance(n, (exp.SetOperation, exp.Subquery))
275                    ):
276                        if not isinstance(set_op, exp.SetOperation):
277                            continue
278
279                        if set_op.args.get("by_name"):
280                            r_type_by_select = {
281                                s.alias_or_name: s.type for s in set_op.right.selects
282                            }
283                            setop_cols = {
284                                s.alias_or_name: self._maybe_coerce(
285                                    t.cast(exp.DataType, s.type),
286                                    r_type_by_select.get(s.alias_or_name)
287                                    or exp.DataType.Type.UNKNOWN,
288                                )
289                                for s in set_op.left.selects
290                            }
291                        else:
292                            setop_cols = {
293                                ls.alias_or_name: self._maybe_coerce(
294                                    t.cast(exp.DataType, ls.type), t.cast(exp.DataType, rs.type)
295                                )
296                                for ls, rs in zip(set_op.left.selects, set_op.right.selects)
297                            }
298
299                        # Coerce intermediate results with the previously registered types, if they exist
300                        for col_name, col_type in setop_cols.items():
301                            col_types[col_name] = self._maybe_coerce(
302                                col_type, col_types.get(col_name, exp.DataType.Type.NULL)
303                            )
304
305            else:
306                selects[name] = {s.alias_or_name: s.type for s in expression.selects}
307
308        # First annotate the current scope's column references
309        for col in scope.columns:
310            if not col.table:
311                continue
312
313            source = scope.sources.get(col.table)
314            if isinstance(source, exp.Table):
315                self._set_type(col, self.schema.get_column_type(source, col))
316            elif source:
317                if col.table in selects and col.name in selects[col.table]:
318                    self._set_type(col, selects[col.table][col.name])
319                elif isinstance(source.expression, exp.Unnest):
320                    self._set_type(col, source.expression.type)
321
322        if isinstance(self.schema, MappingSchema):
323            for table_column in scope.table_columns:
324                source = scope.sources.get(table_column.name)
325
326                if isinstance(source, exp.Table):
327                    schema = self.schema.find(
328                        source, raise_on_missing=False, ensure_data_types=True
329                    )
330                    if not isinstance(schema, dict):
331                        continue
332
333                    struct_type = exp.DataType(
334                        this=exp.DataType.Type.STRUCT,
335                        expressions=[
336                            exp.ColumnDef(this=exp.to_identifier(c), kind=kind)
337                            for c, kind in schema.items()
338                        ],
339                        nested=True,
340                    )
341                    self._set_type(table_column, struct_type)
342                elif (
343                    isinstance(source, Scope)
344                    and isinstance(source.expression, exp.Query)
345                    and (
346                        source.expression.meta.get("query_type") or exp.DataType.build("UNKNOWN")
347                    ).is_type(exp.DataType.Type.STRUCT)
348                ):
349                    self._set_type(table_column, source.expression.meta["query_type"])
350
351        # Then (possibly) annotate the remaining expressions in the scope
352        self._maybe_annotate(scope.expression)
353
354        if self.schema.dialect == "bigquery" and isinstance(scope.expression, exp.Query):
355            struct_type = exp.DataType(
356                this=exp.DataType.Type.STRUCT,
357                expressions=[
358                    exp.ColumnDef(
359                        this=exp.to_identifier(select.output_name),
360                        kind=select.type.copy() if select.type else None,
361                    )
362                    for select in scope.expression.selects
363                ],
364                nested=True,
365            )
366
367            if not any(
368                cd.kind.is_type(exp.DataType.Type.UNKNOWN)
369                for cd in struct_type.expressions
370                if cd.kind
371            ):
372                # We don't use `_set_type` on purpose here. If we annotated the query directly, then
373                # using it in other contexts (e.g., ARRAY(<query>)) could result in incorrect type
374                # annotations, i.e., it shouldn't be interpreted as a STRUCT value.
375                scope.expression.meta["query_type"] = struct_type
376
377    def _maybe_annotate(self, expression: E) -> E:
378        if id(expression) in self._visited:
379            return expression  # We've already inferred the expression's type
380
381        annotator = self.annotators.get(expression.__class__)
382
383        return (
384            annotator(self, expression)
385            if annotator
386            else self._annotate_with_type(expression, exp.DataType.Type.UNKNOWN)
387        )
388
389    def _annotate_args(self, expression: E) -> E:
390        for value in expression.iter_expressions():
391            self._maybe_annotate(value)
392
393        return expression
394
395    def _maybe_coerce(
396        self,
397        type1: exp.DataType | exp.DataType.Type,
398        type2: exp.DataType | exp.DataType.Type,
399    ) -> exp.DataType | exp.DataType.Type:
400        """
401        Returns type2 if type1 can be coerced into it, otherwise type1.
402
403        If either type is parameterized (e.g. DECIMAL(18, 2) contains two parameters),
404        we assume type1 does not coerce into type2, so we also return it in this case.
405        """
406        if isinstance(type1, exp.DataType):
407            if type1.expressions:
408                return type1
409            type1_value = type1.this
410        else:
411            type1_value = type1
412
413        if isinstance(type2, exp.DataType):
414            if type2.expressions:
415                return type2
416            type2_value = type2.this
417        else:
418            type2_value = type2
419
420        # We propagate the UNKNOWN type upwards if found
421        if exp.DataType.Type.UNKNOWN in (type1_value, type2_value):
422            return exp.DataType.Type.UNKNOWN
423
424        return type2_value if type2_value in self.coerces_to.get(type1_value, {}) else type1_value
425
426    def _annotate_binary(self, expression: B) -> B:
427        self._annotate_args(expression)
428
429        left, right = expression.left, expression.right
430        if not left or not right:
431            expression_sql = expression.sql(self.schema.dialect)
432            logger.warning(f"Failed to annotate badly formed binary expression: {expression_sql}")
433            self._set_type(expression, None)
434            return expression
435
436        left_type, right_type = left.type.this, right.type.this  # type: ignore
437
438        if isinstance(expression, (exp.Connector, exp.Predicate)):
439            self._set_type(expression, exp.DataType.Type.BOOLEAN)
440        elif (left_type, right_type) in self.binary_coercions:
441            self._set_type(expression, self.binary_coercions[(left_type, right_type)](left, right))
442        else:
443            self._set_type(expression, self._maybe_coerce(left_type, right_type))
444
445        return expression
446
447    def _annotate_unary(self, expression: E) -> E:
448        self._annotate_args(expression)
449
450        if isinstance(expression, exp.Not):
451            self._set_type(expression, exp.DataType.Type.BOOLEAN)
452        else:
453            self._set_type(expression, expression.this.type)
454
455        return expression
456
457    def _annotate_literal(self, expression: exp.Literal) -> exp.Literal:
458        if expression.is_string:
459            self._set_type(expression, exp.DataType.Type.VARCHAR)
460        elif expression.is_int:
461            self._set_type(expression, exp.DataType.Type.INT)
462        else:
463            self._set_type(expression, exp.DataType.Type.DOUBLE)
464
465        return expression
466
467    def _annotate_with_type(
468        self, expression: E, target_type: exp.DataType | exp.DataType.Type
469    ) -> E:
470        self._set_type(expression, target_type)
471        return self._annotate_args(expression)
472
473    @t.no_type_check
474    def _annotate_by_args(
475        self,
476        expression: E,
477        *args: str,
478        promote: bool = False,
479        array: bool = False,
480    ) -> E:
481        self._annotate_args(expression)
482
483        expressions: t.List[exp.Expression] = []
484        for arg in args:
485            arg_expr = expression.args.get(arg)
486            expressions.extend(expr for expr in ensure_list(arg_expr) if expr)
487
488        last_datatype = None
489        for expr in expressions:
490            expr_type = expr.type
491
492            # Stop at the first nested data type found - we don't want to _maybe_coerce nested types
493            if expr_type.args.get("nested"):
494                last_datatype = expr_type
495                break
496
497            if not expr_type.is_type(exp.DataType.Type.UNKNOWN):
498                last_datatype = self._maybe_coerce(last_datatype or expr_type, expr_type)
499
500        self._set_type(expression, last_datatype or exp.DataType.Type.UNKNOWN)
501
502        if promote:
503            if expression.type.this in exp.DataType.INTEGER_TYPES:
504                self._set_type(expression, exp.DataType.Type.BIGINT)
505            elif expression.type.this in exp.DataType.FLOAT_TYPES:
506                self._set_type(expression, exp.DataType.Type.DOUBLE)
507
508        if array:
509            self._set_type(
510                expression,
511                exp.DataType(
512                    this=exp.DataType.Type.ARRAY, expressions=[expression.type], nested=True
513                ),
514            )
515
516        return expression
517
518    def _annotate_timeunit(
519        self, expression: exp.TimeUnit | exp.DateTrunc
520    ) -> exp.TimeUnit | exp.DateTrunc:
521        self._annotate_args(expression)
522
523        if expression.this.type.this in exp.DataType.TEXT_TYPES:
524            datatype = _coerce_date_literal(expression.this, expression.unit)
525        elif expression.this.type.this in exp.DataType.TEMPORAL_TYPES:
526            datatype = _coerce_date(expression.this, expression.unit)
527        else:
528            datatype = exp.DataType.Type.UNKNOWN
529
530        self._set_type(expression, datatype)
531        return expression
532
533    def _annotate_bracket(self, expression: exp.Bracket) -> exp.Bracket:
534        self._annotate_args(expression)
535
536        bracket_arg = expression.expressions[0]
537        this = expression.this
538
539        if isinstance(bracket_arg, exp.Slice):
540            self._set_type(expression, this.type)
541        elif this.type.is_type(exp.DataType.Type.ARRAY):
542            self._set_type(expression, seq_get(this.type.expressions, 0))
543        elif isinstance(this, (exp.Map, exp.VarMap)) and bracket_arg in this.keys:
544            index = this.keys.index(bracket_arg)
545            value = seq_get(this.values, index)
546            self._set_type(expression, value.type if value else None)
547        else:
548            self._set_type(expression, exp.DataType.Type.UNKNOWN)
549
550        return expression
551
552    def _annotate_div(self, expression: exp.Div) -> exp.Div:
553        self._annotate_args(expression)
554
555        left_type, right_type = expression.left.type.this, expression.right.type.this  # type: ignore
556
557        if (
558            expression.args.get("typed")
559            and left_type in exp.DataType.INTEGER_TYPES
560            and right_type in exp.DataType.INTEGER_TYPES
561        ):
562            self._set_type(expression, exp.DataType.Type.BIGINT)
563        else:
564            self._set_type(expression, self._maybe_coerce(left_type, right_type))
565            if expression.type and expression.type.this not in exp.DataType.REAL_TYPES:
566                self._set_type(
567                    expression, self._maybe_coerce(expression.type, exp.DataType.Type.DOUBLE)
568                )
569
570        return expression
571
572    def _annotate_dot(self, expression: exp.Dot) -> exp.Dot:
573        self._annotate_args(expression)
574        self._set_type(expression, None)
575        this_type = expression.this.type
576
577        if this_type and this_type.is_type(exp.DataType.Type.STRUCT):
578            for e in this_type.expressions:
579                if e.name == expression.expression.name:
580                    self._set_type(expression, e.kind)
581                    break
582
583        return expression
584
585    def _annotate_explode(self, expression: exp.Explode) -> exp.Explode:
586        self._annotate_args(expression)
587        self._set_type(expression, seq_get(expression.this.type.expressions, 0))
588        return expression
589
590    def _annotate_unnest(self, expression: exp.Unnest) -> exp.Unnest:
591        self._annotate_args(expression)
592        child = seq_get(expression.expressions, 0)
593
594        if child and child.is_type(exp.DataType.Type.ARRAY):
595            expr_type = seq_get(child.type.expressions, 0)
596        else:
597            expr_type = None
598
599        self._set_type(expression, expr_type)
600        return expression
601
602    def _annotate_struct_value(
603        self, expression: exp.Expression
604    ) -> t.Optional[exp.DataType] | exp.ColumnDef:
605        # Case: STRUCT(key AS value)
606        if alias := expression.args.get("alias"):
607            return exp.ColumnDef(this=alias.copy(), kind=expression.type)
608
609        # Case: STRUCT(key = value) or STRUCT(key := value)
610        if expression.expression:
611            return exp.ColumnDef(this=expression.this.copy(), kind=expression.expression.type)
612
613        # Case: STRUCT(c)
614        if isinstance(expression, exp.Column):
615            return exp.ColumnDef(this=expression.this.copy(), kind=expression.type)
616
617        return expression.type
618
619    def _annotate_struct(self, expression: exp.Struct) -> exp.Struct:
620        self._annotate_args(expression)
621        self._set_type(
622            expression,
623            exp.DataType(
624                this=exp.DataType.Type.STRUCT,
625                expressions=[self._annotate_struct_value(expr) for expr in expression.expressions],
626                nested=True,
627            ),
628        )
629        return expression
630
631    @t.overload
632    def _annotate_map(self, expression: exp.Map) -> exp.Map: ...
633
634    @t.overload
635    def _annotate_map(self, expression: exp.VarMap) -> exp.VarMap: ...
636
637    def _annotate_map(self, expression):
638        self._annotate_args(expression)
639
640        keys = expression.args.get("keys")
641        values = expression.args.get("values")
642
643        map_type = exp.DataType(this=exp.DataType.Type.MAP)
644        if isinstance(keys, exp.Array) and isinstance(values, exp.Array):
645            key_type = seq_get(keys.type.expressions, 0) or exp.DataType.Type.UNKNOWN
646            value_type = seq_get(values.type.expressions, 0) or exp.DataType.Type.UNKNOWN
647
648            if key_type != exp.DataType.Type.UNKNOWN and value_type != exp.DataType.Type.UNKNOWN:
649                map_type.set("expressions", [key_type, value_type])
650                map_type.set("nested", True)
651
652        self._set_type(expression, map_type)
653        return expression
654
655    def _annotate_to_map(self, expression: exp.ToMap) -> exp.ToMap:
656        self._annotate_args(expression)
657
658        map_type = exp.DataType(this=exp.DataType.Type.MAP)
659        arg = expression.this
660        if arg.is_type(exp.DataType.Type.STRUCT):
661            for coldef in arg.type.expressions:
662                kind = coldef.kind
663                if kind != exp.DataType.Type.UNKNOWN:
664                    map_type.set("expressions", [exp.DataType.build("varchar"), kind])
665                    map_type.set("nested", True)
666                    break
667
668        self._set_type(expression, map_type)
669        return expression
670
671    def _annotate_extract(self, expression: exp.Extract) -> exp.Extract:
672        self._annotate_args(expression)
673        part = expression.name
674        if part == "TIME":
675            self._set_type(expression, exp.DataType.Type.TIME)
676        elif part == "DATE":
677            self._set_type(expression, exp.DataType.Type.DATE)
678        else:
679            self._set_type(expression, exp.DataType.Type.INT)
680        return expression
681
682    def _annotate_by_array_element(self, expression: exp.Expression) -> exp.Expression:
683        self._annotate_args(expression)
684
685        array_arg = expression.this
686        if array_arg.type.is_type(exp.DataType.Type.ARRAY):
687            element_type = seq_get(array_arg.type.expressions, 0) or exp.DataType.Type.UNKNOWN
688            self._set_type(expression, element_type)
689        else:
690            self._set_type(expression, exp.DataType.Type.UNKNOWN)
691
692        return expression
logger = <Logger sqlglot (WARNING)>
def annotate_types( expression: ~E, schema: Union[Dict, sqlglot.schema.Schema, NoneType] = None, annotators: Optional[Dict[Type[~E], Callable[[TypeAnnotator, ~E], ~E]]] = None, coerces_to: Optional[Dict[sqlglot.expressions.DataType.Type, Set[sqlglot.expressions.DataType.Type]]] = None, dialect: Union[str, sqlglot.dialects.Dialect, Type[sqlglot.dialects.Dialect], NoneType] = None) -> ~E:
34def annotate_types(
35    expression: E,
36    schema: t.Optional[t.Dict | Schema] = None,
37    annotators: t.Optional[AnnotatorsType] = None,
38    coerces_to: t.Optional[t.Dict[exp.DataType.Type, t.Set[exp.DataType.Type]]] = None,
39    dialect: DialectType = None,
40) -> E:
41    """
42    Infers the types of an expression, annotating its AST accordingly.
43
44    Example:
45        >>> import sqlglot
46        >>> schema = {"y": {"cola": "SMALLINT"}}
47        >>> sql = "SELECT x.cola + 2.5 AS cola FROM (SELECT y.cola AS cola FROM y AS y) AS x"
48        >>> annotated_expr = annotate_types(sqlglot.parse_one(sql), schema=schema)
49        >>> annotated_expr.expressions[0].type.this  # Get the type of "x.cola + 2.5 AS cola"
50        <Type.DOUBLE: 'DOUBLE'>
51
52    Args:
53        expression: Expression to annotate.
54        schema: Database schema.
55        annotators: Maps expression type to corresponding annotation function.
56        coerces_to: Maps expression type to set of types that it can be coerced into.
57
58    Returns:
59        The expression annotated with types.
60    """
61
62    schema = ensure_schema(schema, dialect=dialect)
63
64    return TypeAnnotator(schema, annotators, coerces_to).annotate(expression)

Infers the types of an expression, annotating its AST accordingly.

Example:
>>> import sqlglot
>>> schema = {"y": {"cola": "SMALLINT"}}
>>> sql = "SELECT x.cola + 2.5 AS cola FROM (SELECT y.cola AS cola FROM y AS y) AS x"
>>> annotated_expr = annotate_types(sqlglot.parse_one(sql), schema=schema)
>>> annotated_expr.expressions[0].type.this  # Get the type of "x.cola + 2.5 AS cola"
<Type.DOUBLE: 'DOUBLE'>
Arguments:
  • expression: Expression to annotate.
  • schema: Database schema.
  • annotators: Maps expression type to corresponding annotation function.
  • coerces_to: Maps expression type to set of types that it can be coerced into.
Returns:

The expression annotated with types.

87def swap_args(func: BinaryCoercionFunc) -> BinaryCoercionFunc:
88    @functools.wraps(func)
89    def _swapped(l: exp.Expression, r: exp.Expression) -> exp.DataType.Type:
90        return func(r, l)
91
92    return _swapped
95def swap_all(coercions: BinaryCoercions) -> BinaryCoercions:
96    return {**coercions, **{(b, a): swap_args(func) for (a, b), func in coercions.items()}}
class TypeAnnotator:
145class TypeAnnotator(metaclass=_TypeAnnotator):
146    NESTED_TYPES = {
147        exp.DataType.Type.ARRAY,
148    }
149
150    # Specifies what types a given type can be coerced into (autofilled)
151    COERCES_TO: t.Dict[exp.DataType.Type, t.Set[exp.DataType.Type]] = {}
152
153    # Coercion functions for binary operations.
154    # Map of type pairs to a callable that takes both sides of the binary operation and returns the resulting type.
155    BINARY_COERCIONS: BinaryCoercions = {
156        **swap_all(
157            {
158                (t, exp.DataType.Type.INTERVAL): lambda l, r: _coerce_date_literal(
159                    l, r.args.get("unit")
160                )
161                for t in exp.DataType.TEXT_TYPES
162            }
163        ),
164        **swap_all(
165            {
166                # text + numeric will yield the numeric type to match most dialects' semantics
167                (text, numeric): lambda l, r: t.cast(
168                    exp.DataType.Type, l.type if l.type in exp.DataType.NUMERIC_TYPES else r.type
169                )
170                for text in exp.DataType.TEXT_TYPES
171                for numeric in exp.DataType.NUMERIC_TYPES
172            }
173        ),
174        **swap_all(
175            {
176                (exp.DataType.Type.DATE, exp.DataType.Type.INTERVAL): lambda l, r: _coerce_date(
177                    l, r.args.get("unit")
178                ),
179            }
180        ),
181    }
182
183    def __init__(
184        self,
185        schema: Schema,
186        annotators: t.Optional[AnnotatorsType] = None,
187        coerces_to: t.Optional[t.Dict[exp.DataType.Type, t.Set[exp.DataType.Type]]] = None,
188        binary_coercions: t.Optional[BinaryCoercions] = None,
189    ) -> None:
190        self.schema = schema
191        self.annotators = annotators or Dialect.get_or_raise(schema.dialect).ANNOTATORS
192        self.coerces_to = (
193            coerces_to or Dialect.get_or_raise(schema.dialect).COERCES_TO or self.COERCES_TO
194        )
195        self.binary_coercions = binary_coercions or self.BINARY_COERCIONS
196
197        # Caches the ids of annotated sub-Expressions, to ensure we only visit them once
198        self._visited: t.Set[int] = set()
199
200        # Caches NULL-annotated expressions to set them to UNKNOWN after type inference is completed
201        self._null_expressions: t.Dict[int, exp.Expression] = {}
202
203        # Databricks and Spark ≥v3 actually support NULL (i.e., VOID) as a type
204        self._supports_null_type = schema.dialect in ("databricks", "spark")
205
206        # Maps an exp.SetOperation's id (e.g. UNION) to its projection types. This is computed if the
207        # exp.SetOperation is the expression of a scope source, as selecting from it multiple times
208        # would reprocess the entire subtree to coerce the types of its operands' projections
209        self._setop_column_types: t.Dict[int, t.Dict[str, exp.DataType | exp.DataType.Type]] = {}
210
211    def _set_type(
212        self, expression: exp.Expression, target_type: t.Optional[exp.DataType | exp.DataType.Type]
213    ) -> None:
214        prev_type = expression.type
215        expression_id = id(expression)
216
217        expression.type = target_type or exp.DataType.Type.UNKNOWN  # type: ignore
218        self._visited.add(expression_id)
219
220        if (
221            not self._supports_null_type
222            and t.cast(exp.DataType, expression.type).this == exp.DataType.Type.NULL
223        ):
224            self._null_expressions[expression_id] = expression
225        elif prev_type and t.cast(exp.DataType, prev_type).this == exp.DataType.Type.NULL:
226            self._null_expressions.pop(expression_id, None)
227
228    def annotate(self, expression: E) -> E:
229        for scope in traverse_scope(expression):
230            self.annotate_scope(scope)
231
232        # This takes care of non-traversable expressions
233        expression = self._maybe_annotate(expression)
234
235        # Replace NULL type with UNKNOWN, since the former is not an actual type;
236        # it is mostly used to aid type coercion, e.g. in query set operations.
237        for expr in self._null_expressions.values():
238            expr.type = exp.DataType.Type.UNKNOWN
239
240        return expression
241
242    def annotate_scope(self, scope: Scope) -> None:
243        selects = {}
244        for name, source in scope.sources.items():
245            if not isinstance(source, Scope):
246                continue
247
248            expression = source.expression
249            if isinstance(expression, exp.UDTF):
250                values = []
251
252                if isinstance(expression, exp.Lateral):
253                    if isinstance(expression.this, exp.Explode):
254                        values = [expression.this.this]
255                elif isinstance(expression, exp.Unnest):
256                    values = [expression]
257                elif not isinstance(expression, exp.TableFromRows):
258                    values = expression.expressions[0].expressions
259
260                if not values:
261                    continue
262
263                selects[name] = {
264                    alias: column.type
265                    for alias, column in zip(expression.alias_column_names, values)
266                }
267            elif isinstance(expression, exp.SetOperation) and len(expression.left.selects) == len(
268                expression.right.selects
269            ):
270                selects[name] = col_types = self._setop_column_types.setdefault(id(expression), {})
271
272                if not col_types:
273                    # Process a chain / sub-tree of set operations
274                    for set_op in expression.walk(
275                        prune=lambda n: not isinstance(n, (exp.SetOperation, exp.Subquery))
276                    ):
277                        if not isinstance(set_op, exp.SetOperation):
278                            continue
279
280                        if set_op.args.get("by_name"):
281                            r_type_by_select = {
282                                s.alias_or_name: s.type for s in set_op.right.selects
283                            }
284                            setop_cols = {
285                                s.alias_or_name: self._maybe_coerce(
286                                    t.cast(exp.DataType, s.type),
287                                    r_type_by_select.get(s.alias_or_name)
288                                    or exp.DataType.Type.UNKNOWN,
289                                )
290                                for s in set_op.left.selects
291                            }
292                        else:
293                            setop_cols = {
294                                ls.alias_or_name: self._maybe_coerce(
295                                    t.cast(exp.DataType, ls.type), t.cast(exp.DataType, rs.type)
296                                )
297                                for ls, rs in zip(set_op.left.selects, set_op.right.selects)
298                            }
299
300                        # Coerce intermediate results with the previously registered types, if they exist
301                        for col_name, col_type in setop_cols.items():
302                            col_types[col_name] = self._maybe_coerce(
303                                col_type, col_types.get(col_name, exp.DataType.Type.NULL)
304                            )
305
306            else:
307                selects[name] = {s.alias_or_name: s.type for s in expression.selects}
308
309        # First annotate the current scope's column references
310        for col in scope.columns:
311            if not col.table:
312                continue
313
314            source = scope.sources.get(col.table)
315            if isinstance(source, exp.Table):
316                self._set_type(col, self.schema.get_column_type(source, col))
317            elif source:
318                if col.table in selects and col.name in selects[col.table]:
319                    self._set_type(col, selects[col.table][col.name])
320                elif isinstance(source.expression, exp.Unnest):
321                    self._set_type(col, source.expression.type)
322
323        if isinstance(self.schema, MappingSchema):
324            for table_column in scope.table_columns:
325                source = scope.sources.get(table_column.name)
326
327                if isinstance(source, exp.Table):
328                    schema = self.schema.find(
329                        source, raise_on_missing=False, ensure_data_types=True
330                    )
331                    if not isinstance(schema, dict):
332                        continue
333
334                    struct_type = exp.DataType(
335                        this=exp.DataType.Type.STRUCT,
336                        expressions=[
337                            exp.ColumnDef(this=exp.to_identifier(c), kind=kind)
338                            for c, kind in schema.items()
339                        ],
340                        nested=True,
341                    )
342                    self._set_type(table_column, struct_type)
343                elif (
344                    isinstance(source, Scope)
345                    and isinstance(source.expression, exp.Query)
346                    and (
347                        source.expression.meta.get("query_type") or exp.DataType.build("UNKNOWN")
348                    ).is_type(exp.DataType.Type.STRUCT)
349                ):
350                    self._set_type(table_column, source.expression.meta["query_type"])
351
352        # Then (possibly) annotate the remaining expressions in the scope
353        self._maybe_annotate(scope.expression)
354
355        if self.schema.dialect == "bigquery" and isinstance(scope.expression, exp.Query):
356            struct_type = exp.DataType(
357                this=exp.DataType.Type.STRUCT,
358                expressions=[
359                    exp.ColumnDef(
360                        this=exp.to_identifier(select.output_name),
361                        kind=select.type.copy() if select.type else None,
362                    )
363                    for select in scope.expression.selects
364                ],
365                nested=True,
366            )
367
368            if not any(
369                cd.kind.is_type(exp.DataType.Type.UNKNOWN)
370                for cd in struct_type.expressions
371                if cd.kind
372            ):
373                # We don't use `_set_type` on purpose here. If we annotated the query directly, then
374                # using it in other contexts (e.g., ARRAY(<query>)) could result in incorrect type
375                # annotations, i.e., it shouldn't be interpreted as a STRUCT value.
376                scope.expression.meta["query_type"] = struct_type
377
378    def _maybe_annotate(self, expression: E) -> E:
379        if id(expression) in self._visited:
380            return expression  # We've already inferred the expression's type
381
382        annotator = self.annotators.get(expression.__class__)
383
384        return (
385            annotator(self, expression)
386            if annotator
387            else self._annotate_with_type(expression, exp.DataType.Type.UNKNOWN)
388        )
389
390    def _annotate_args(self, expression: E) -> E:
391        for value in expression.iter_expressions():
392            self._maybe_annotate(value)
393
394        return expression
395
396    def _maybe_coerce(
397        self,
398        type1: exp.DataType | exp.DataType.Type,
399        type2: exp.DataType | exp.DataType.Type,
400    ) -> exp.DataType | exp.DataType.Type:
401        """
402        Returns type2 if type1 can be coerced into it, otherwise type1.
403
404        If either type is parameterized (e.g. DECIMAL(18, 2) contains two parameters),
405        we assume type1 does not coerce into type2, so we also return it in this case.
406        """
407        if isinstance(type1, exp.DataType):
408            if type1.expressions:
409                return type1
410            type1_value = type1.this
411        else:
412            type1_value = type1
413
414        if isinstance(type2, exp.DataType):
415            if type2.expressions:
416                return type2
417            type2_value = type2.this
418        else:
419            type2_value = type2
420
421        # We propagate the UNKNOWN type upwards if found
422        if exp.DataType.Type.UNKNOWN in (type1_value, type2_value):
423            return exp.DataType.Type.UNKNOWN
424
425        return type2_value if type2_value in self.coerces_to.get(type1_value, {}) else type1_value
426
427    def _annotate_binary(self, expression: B) -> B:
428        self._annotate_args(expression)
429
430        left, right = expression.left, expression.right
431        if not left or not right:
432            expression_sql = expression.sql(self.schema.dialect)
433            logger.warning(f"Failed to annotate badly formed binary expression: {expression_sql}")
434            self._set_type(expression, None)
435            return expression
436
437        left_type, right_type = left.type.this, right.type.this  # type: ignore
438
439        if isinstance(expression, (exp.Connector, exp.Predicate)):
440            self._set_type(expression, exp.DataType.Type.BOOLEAN)
441        elif (left_type, right_type) in self.binary_coercions:
442            self._set_type(expression, self.binary_coercions[(left_type, right_type)](left, right))
443        else:
444            self._set_type(expression, self._maybe_coerce(left_type, right_type))
445
446        return expression
447
448    def _annotate_unary(self, expression: E) -> E:
449        self._annotate_args(expression)
450
451        if isinstance(expression, exp.Not):
452            self._set_type(expression, exp.DataType.Type.BOOLEAN)
453        else:
454            self._set_type(expression, expression.this.type)
455
456        return expression
457
458    def _annotate_literal(self, expression: exp.Literal) -> exp.Literal:
459        if expression.is_string:
460            self._set_type(expression, exp.DataType.Type.VARCHAR)
461        elif expression.is_int:
462            self._set_type(expression, exp.DataType.Type.INT)
463        else:
464            self._set_type(expression, exp.DataType.Type.DOUBLE)
465
466        return expression
467
468    def _annotate_with_type(
469        self, expression: E, target_type: exp.DataType | exp.DataType.Type
470    ) -> E:
471        self._set_type(expression, target_type)
472        return self._annotate_args(expression)
473
474    @t.no_type_check
475    def _annotate_by_args(
476        self,
477        expression: E,
478        *args: str,
479        promote: bool = False,
480        array: bool = False,
481    ) -> E:
482        self._annotate_args(expression)
483
484        expressions: t.List[exp.Expression] = []
485        for arg in args:
486            arg_expr = expression.args.get(arg)
487            expressions.extend(expr for expr in ensure_list(arg_expr) if expr)
488
489        last_datatype = None
490        for expr in expressions:
491            expr_type = expr.type
492
493            # Stop at the first nested data type found - we don't want to _maybe_coerce nested types
494            if expr_type.args.get("nested"):
495                last_datatype = expr_type
496                break
497
498            if not expr_type.is_type(exp.DataType.Type.UNKNOWN):
499                last_datatype = self._maybe_coerce(last_datatype or expr_type, expr_type)
500
501        self._set_type(expression, last_datatype or exp.DataType.Type.UNKNOWN)
502
503        if promote:
504            if expression.type.this in exp.DataType.INTEGER_TYPES:
505                self._set_type(expression, exp.DataType.Type.BIGINT)
506            elif expression.type.this in exp.DataType.FLOAT_TYPES:
507                self._set_type(expression, exp.DataType.Type.DOUBLE)
508
509        if array:
510            self._set_type(
511                expression,
512                exp.DataType(
513                    this=exp.DataType.Type.ARRAY, expressions=[expression.type], nested=True
514                ),
515            )
516
517        return expression
518
519    def _annotate_timeunit(
520        self, expression: exp.TimeUnit | exp.DateTrunc
521    ) -> exp.TimeUnit | exp.DateTrunc:
522        self._annotate_args(expression)
523
524        if expression.this.type.this in exp.DataType.TEXT_TYPES:
525            datatype = _coerce_date_literal(expression.this, expression.unit)
526        elif expression.this.type.this in exp.DataType.TEMPORAL_TYPES:
527            datatype = _coerce_date(expression.this, expression.unit)
528        else:
529            datatype = exp.DataType.Type.UNKNOWN
530
531        self._set_type(expression, datatype)
532        return expression
533
534    def _annotate_bracket(self, expression: exp.Bracket) -> exp.Bracket:
535        self._annotate_args(expression)
536
537        bracket_arg = expression.expressions[0]
538        this = expression.this
539
540        if isinstance(bracket_arg, exp.Slice):
541            self._set_type(expression, this.type)
542        elif this.type.is_type(exp.DataType.Type.ARRAY):
543            self._set_type(expression, seq_get(this.type.expressions, 0))
544        elif isinstance(this, (exp.Map, exp.VarMap)) and bracket_arg in this.keys:
545            index = this.keys.index(bracket_arg)
546            value = seq_get(this.values, index)
547            self._set_type(expression, value.type if value else None)
548        else:
549            self._set_type(expression, exp.DataType.Type.UNKNOWN)
550
551        return expression
552
553    def _annotate_div(self, expression: exp.Div) -> exp.Div:
554        self._annotate_args(expression)
555
556        left_type, right_type = expression.left.type.this, expression.right.type.this  # type: ignore
557
558        if (
559            expression.args.get("typed")
560            and left_type in exp.DataType.INTEGER_TYPES
561            and right_type in exp.DataType.INTEGER_TYPES
562        ):
563            self._set_type(expression, exp.DataType.Type.BIGINT)
564        else:
565            self._set_type(expression, self._maybe_coerce(left_type, right_type))
566            if expression.type and expression.type.this not in exp.DataType.REAL_TYPES:
567                self._set_type(
568                    expression, self._maybe_coerce(expression.type, exp.DataType.Type.DOUBLE)
569                )
570
571        return expression
572
573    def _annotate_dot(self, expression: exp.Dot) -> exp.Dot:
574        self._annotate_args(expression)
575        self._set_type(expression, None)
576        this_type = expression.this.type
577
578        if this_type and this_type.is_type(exp.DataType.Type.STRUCT):
579            for e in this_type.expressions:
580                if e.name == expression.expression.name:
581                    self._set_type(expression, e.kind)
582                    break
583
584        return expression
585
586    def _annotate_explode(self, expression: exp.Explode) -> exp.Explode:
587        self._annotate_args(expression)
588        self._set_type(expression, seq_get(expression.this.type.expressions, 0))
589        return expression
590
591    def _annotate_unnest(self, expression: exp.Unnest) -> exp.Unnest:
592        self._annotate_args(expression)
593        child = seq_get(expression.expressions, 0)
594
595        if child and child.is_type(exp.DataType.Type.ARRAY):
596            expr_type = seq_get(child.type.expressions, 0)
597        else:
598            expr_type = None
599
600        self._set_type(expression, expr_type)
601        return expression
602
603    def _annotate_struct_value(
604        self, expression: exp.Expression
605    ) -> t.Optional[exp.DataType] | exp.ColumnDef:
606        # Case: STRUCT(key AS value)
607        if alias := expression.args.get("alias"):
608            return exp.ColumnDef(this=alias.copy(), kind=expression.type)
609
610        # Case: STRUCT(key = value) or STRUCT(key := value)
611        if expression.expression:
612            return exp.ColumnDef(this=expression.this.copy(), kind=expression.expression.type)
613
614        # Case: STRUCT(c)
615        if isinstance(expression, exp.Column):
616            return exp.ColumnDef(this=expression.this.copy(), kind=expression.type)
617
618        return expression.type
619
620    def _annotate_struct(self, expression: exp.Struct) -> exp.Struct:
621        self._annotate_args(expression)
622        self._set_type(
623            expression,
624            exp.DataType(
625                this=exp.DataType.Type.STRUCT,
626                expressions=[self._annotate_struct_value(expr) for expr in expression.expressions],
627                nested=True,
628            ),
629        )
630        return expression
631
632    @t.overload
633    def _annotate_map(self, expression: exp.Map) -> exp.Map: ...
634
635    @t.overload
636    def _annotate_map(self, expression: exp.VarMap) -> exp.VarMap: ...
637
638    def _annotate_map(self, expression):
639        self._annotate_args(expression)
640
641        keys = expression.args.get("keys")
642        values = expression.args.get("values")
643
644        map_type = exp.DataType(this=exp.DataType.Type.MAP)
645        if isinstance(keys, exp.Array) and isinstance(values, exp.Array):
646            key_type = seq_get(keys.type.expressions, 0) or exp.DataType.Type.UNKNOWN
647            value_type = seq_get(values.type.expressions, 0) or exp.DataType.Type.UNKNOWN
648
649            if key_type != exp.DataType.Type.UNKNOWN and value_type != exp.DataType.Type.UNKNOWN:
650                map_type.set("expressions", [key_type, value_type])
651                map_type.set("nested", True)
652
653        self._set_type(expression, map_type)
654        return expression
655
656    def _annotate_to_map(self, expression: exp.ToMap) -> exp.ToMap:
657        self._annotate_args(expression)
658
659        map_type = exp.DataType(this=exp.DataType.Type.MAP)
660        arg = expression.this
661        if arg.is_type(exp.DataType.Type.STRUCT):
662            for coldef in arg.type.expressions:
663                kind = coldef.kind
664                if kind != exp.DataType.Type.UNKNOWN:
665                    map_type.set("expressions", [exp.DataType.build("varchar"), kind])
666                    map_type.set("nested", True)
667                    break
668
669        self._set_type(expression, map_type)
670        return expression
671
672    def _annotate_extract(self, expression: exp.Extract) -> exp.Extract:
673        self._annotate_args(expression)
674        part = expression.name
675        if part == "TIME":
676            self._set_type(expression, exp.DataType.Type.TIME)
677        elif part == "DATE":
678            self._set_type(expression, exp.DataType.Type.DATE)
679        else:
680            self._set_type(expression, exp.DataType.Type.INT)
681        return expression
682
683    def _annotate_by_array_element(self, expression: exp.Expression) -> exp.Expression:
684        self._annotate_args(expression)
685
686        array_arg = expression.this
687        if array_arg.type.is_type(exp.DataType.Type.ARRAY):
688            element_type = seq_get(array_arg.type.expressions, 0) or exp.DataType.Type.UNKNOWN
689            self._set_type(expression, element_type)
690        else:
691            self._set_type(expression, exp.DataType.Type.UNKNOWN)
692
693        return expression
TypeAnnotator( schema: sqlglot.schema.Schema, annotators: Optional[Dict[Type[~E], Callable[[TypeAnnotator, ~E], ~E]]] = None, coerces_to: Optional[Dict[sqlglot.expressions.DataType.Type, Set[sqlglot.expressions.DataType.Type]]] = None, binary_coercions: Optional[Dict[Tuple[sqlglot.expressions.DataType.Type, sqlglot.expressions.DataType.Type], Callable[[sqlglot.expressions.Expression, sqlglot.expressions.Expression], sqlglot.expressions.DataType.Type]]] = None)
183    def __init__(
184        self,
185        schema: Schema,
186        annotators: t.Optional[AnnotatorsType] = None,
187        coerces_to: t.Optional[t.Dict[exp.DataType.Type, t.Set[exp.DataType.Type]]] = None,
188        binary_coercions: t.Optional[BinaryCoercions] = None,
189    ) -> None:
190        self.schema = schema
191        self.annotators = annotators or Dialect.get_or_raise(schema.dialect).ANNOTATORS
192        self.coerces_to = (
193            coerces_to or Dialect.get_or_raise(schema.dialect).COERCES_TO or self.COERCES_TO
194        )
195        self.binary_coercions = binary_coercions or self.BINARY_COERCIONS
196
197        # Caches the ids of annotated sub-Expressions, to ensure we only visit them once
198        self._visited: t.Set[int] = set()
199
200        # Caches NULL-annotated expressions to set them to UNKNOWN after type inference is completed
201        self._null_expressions: t.Dict[int, exp.Expression] = {}
202
203        # Databricks and Spark ≥v3 actually support NULL (i.e., VOID) as a type
204        self._supports_null_type = schema.dialect in ("databricks", "spark")
205
206        # Maps an exp.SetOperation's id (e.g. UNION) to its projection types. This is computed if the
207        # exp.SetOperation is the expression of a scope source, as selecting from it multiple times
208        # would reprocess the entire subtree to coerce the types of its operands' projections
209        self._setop_column_types: t.Dict[int, t.Dict[str, exp.DataType | exp.DataType.Type]] = {}
NESTED_TYPES = {<Type.ARRAY: 'ARRAY'>}
COERCES_TO: Dict[sqlglot.expressions.DataType.Type, Set[sqlglot.expressions.DataType.Type]] = {<Type.TEXT: 'TEXT'>: set(), <Type.NVARCHAR: 'NVARCHAR'>: {<Type.TEXT: 'TEXT'>}, <Type.VARCHAR: 'VARCHAR'>: {<Type.TEXT: 'TEXT'>, <Type.NVARCHAR: 'NVARCHAR'>}, <Type.NCHAR: 'NCHAR'>: {<Type.TEXT: 'TEXT'>, <Type.VARCHAR: 'VARCHAR'>, <Type.NVARCHAR: 'NVARCHAR'>}, <Type.CHAR: 'CHAR'>: {<Type.TEXT: 'TEXT'>, <Type.VARCHAR: 'VARCHAR'>, <Type.NCHAR: 'NCHAR'>, <Type.NVARCHAR: 'NVARCHAR'>}, <Type.DOUBLE: 'DOUBLE'>: set(), <Type.FLOAT: 'FLOAT'>: {<Type.DOUBLE: 'DOUBLE'>}, <Type.DECIMAL: 'DECIMAL'>: {<Type.DOUBLE: 'DOUBLE'>, <Type.BIGDECIMAL: 'BIGDECIMAL'>, <Type.FLOAT: 'FLOAT'>}, <Type.BIGINT: 'BIGINT'>: {<Type.DOUBLE: 'DOUBLE'>, <Type.DECIMAL: 'DECIMAL'>, <Type.BIGDECIMAL: 'BIGDECIMAL'>, <Type.FLOAT: 'FLOAT'>}, <Type.INT: 'INT'>: {<Type.DOUBLE: 'DOUBLE'>, <Type.DECIMAL: 'DECIMAL'>, <Type.FLOAT: 'FLOAT'>, <Type.BIGINT: 'BIGINT'>}, <Type.SMALLINT: 'SMALLINT'>: {<Type.DOUBLE: 'DOUBLE'>, <Type.DECIMAL: 'DECIMAL'>, <Type.INT: 'INT'>, <Type.BIGINT: 'BIGINT'>, <Type.FLOAT: 'FLOAT'>}, <Type.TINYINT: 'TINYINT'>: {<Type.DOUBLE: 'DOUBLE'>, <Type.DECIMAL: 'DECIMAL'>, <Type.INT: 'INT'>, <Type.BIGINT: 'BIGINT'>, <Type.FLOAT: 'FLOAT'>, <Type.SMALLINT: 'SMALLINT'>}, <Type.TIMESTAMPLTZ: 'TIMESTAMPLTZ'>: set(), <Type.TIMESTAMPTZ: 'TIMESTAMPTZ'>: {<Type.TIMESTAMPLTZ: 'TIMESTAMPLTZ'>}, <Type.TIMESTAMP: 'TIMESTAMP'>: {<Type.TIMESTAMPLTZ: 'TIMESTAMPLTZ'>, <Type.TIMESTAMPTZ: 'TIMESTAMPTZ'>}, <Type.DATETIME: 'DATETIME'>: {<Type.TIMESTAMP: 'TIMESTAMP'>, <Type.TIMESTAMPLTZ: 'TIMESTAMPLTZ'>, <Type.TIMESTAMPTZ: 'TIMESTAMPTZ'>}, <Type.DATE: 'DATE'>: {<Type.TIMESTAMP: 'TIMESTAMP'>, <Type.TIMESTAMPLTZ: 'TIMESTAMPLTZ'>, <Type.TIMESTAMPTZ: 'TIMESTAMPTZ'>, <Type.DATETIME: 'DATETIME'>}, <Type.NULL: 'NULL'>: {<Type.TIMESTAMP: 'TIMESTAMP'>, <Type.INT: 'INT'>, <Type.BIGINT: 'BIGINT'>, <Type.NCHAR: 'NCHAR'>, <Type.TEXT: 'TEXT'>, <Type.FLOAT: 'FLOAT'>, <Type.VARCHAR: 'VARCHAR'>, <Type.SMALLINT: 'SMALLINT'>, <Type.DOUBLE: 'DOUBLE'>, <Type.DECIMAL: 'DECIMAL'>, <Type.DATE: 'DATE'>, <Type.TIMESTAMPLTZ: 'TIMESTAMPLTZ'>, <Type.TIMESTAMPTZ: 'TIMESTAMPTZ'>, <Type.NVARCHAR: 'NVARCHAR'>, <Type.DATETIME: 'DATETIME'>, <Type.CHAR: 'CHAR'>, <Type.TINYINT: 'TINYINT'>}}
BINARY_COERCIONS: Dict[Tuple[sqlglot.expressions.DataType.Type, sqlglot.expressions.DataType.Type], Callable[[sqlglot.expressions.Expression, sqlglot.expressions.Expression], sqlglot.expressions.DataType.Type]] = {(<Type.NCHAR: 'NCHAR'>, <Type.INTERVAL: 'INTERVAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.INTERVAL: 'INTERVAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.INTERVAL: 'INTERVAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.INTERVAL: 'INTERVAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.INTERVAL: 'INTERVAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.INTERVAL: 'INTERVAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INTERVAL: 'INTERVAL'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INTERVAL: 'INTERVAL'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INTERVAL: 'INTERVAL'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INTERVAL: 'INTERVAL'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INTERVAL: 'INTERVAL'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INTERVAL: 'INTERVAL'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.UINT256: 'UINT256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.UBIGINT: 'UBIGINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.MEDIUMINT: 'MEDIUMINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.SMALLINT: 'SMALLINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.INT256: 'INT256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.DECIMAL256: 'DECIMAL256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.BIT: 'BIT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.DECIMAL: 'DECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.DECIMAL32: 'DECIMAL32'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.SMALLMONEY: 'SMALLMONEY'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.BIGDECIMAL: 'BIGDECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.UINT128: 'UINT128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.DECIMAL64: 'DECIMAL64'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.DECIMAL128: 'DECIMAL128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.TINYINT: 'TINYINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.UDOUBLE: 'UDOUBLE'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.USMALLINT: 'USMALLINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.MONEY: 'MONEY'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.INT: 'INT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.BIGINT: 'BIGINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.UINT: 'UINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.FLOAT: 'FLOAT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.UTINYINT: 'UTINYINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.UDECIMAL: 'UDECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.UMEDIUMINT: 'UMEDIUMINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.INT128: 'INT128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NCHAR: 'NCHAR'>, <Type.DOUBLE: 'DOUBLE'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.UINT256: 'UINT256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.UBIGINT: 'UBIGINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.MEDIUMINT: 'MEDIUMINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.SMALLINT: 'SMALLINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.INT256: 'INT256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.DECIMAL256: 'DECIMAL256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.BIT: 'BIT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.DECIMAL: 'DECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.DECIMAL32: 'DECIMAL32'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.SMALLMONEY: 'SMALLMONEY'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.BIGDECIMAL: 'BIGDECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.UINT128: 'UINT128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.DECIMAL64: 'DECIMAL64'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.DECIMAL128: 'DECIMAL128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.TINYINT: 'TINYINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.UDOUBLE: 'UDOUBLE'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.USMALLINT: 'USMALLINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.MONEY: 'MONEY'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.INT: 'INT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.BIGINT: 'BIGINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.UINT: 'UINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.FLOAT: 'FLOAT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.UTINYINT: 'UTINYINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.UDECIMAL: 'UDECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.UMEDIUMINT: 'UMEDIUMINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.INT128: 'INT128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TEXT: 'TEXT'>, <Type.DOUBLE: 'DOUBLE'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.UINT256: 'UINT256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.UBIGINT: 'UBIGINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.MEDIUMINT: 'MEDIUMINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.SMALLINT: 'SMALLINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.INT256: 'INT256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.DECIMAL256: 'DECIMAL256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.BIT: 'BIT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.DECIMAL: 'DECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.DECIMAL32: 'DECIMAL32'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.SMALLMONEY: 'SMALLMONEY'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.BIGDECIMAL: 'BIGDECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.UINT128: 'UINT128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.DECIMAL64: 'DECIMAL64'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.DECIMAL128: 'DECIMAL128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.TINYINT: 'TINYINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.UDOUBLE: 'UDOUBLE'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.USMALLINT: 'USMALLINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.MONEY: 'MONEY'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.INT: 'INT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.BIGINT: 'BIGINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.UINT: 'UINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.FLOAT: 'FLOAT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.UTINYINT: 'UTINYINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.UDECIMAL: 'UDECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.UMEDIUMINT: 'UMEDIUMINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.INT128: 'INT128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.VARCHAR: 'VARCHAR'>, <Type.DOUBLE: 'DOUBLE'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.UINT256: 'UINT256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.UBIGINT: 'UBIGINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.MEDIUMINT: 'MEDIUMINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.SMALLINT: 'SMALLINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.INT256: 'INT256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.DECIMAL256: 'DECIMAL256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.BIT: 'BIT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.DECIMAL: 'DECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.DECIMAL32: 'DECIMAL32'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.SMALLMONEY: 'SMALLMONEY'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.BIGDECIMAL: 'BIGDECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.UINT128: 'UINT128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.DECIMAL64: 'DECIMAL64'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.DECIMAL128: 'DECIMAL128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.TINYINT: 'TINYINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.UDOUBLE: 'UDOUBLE'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.USMALLINT: 'USMALLINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.MONEY: 'MONEY'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.INT: 'INT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.BIGINT: 'BIGINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.UINT: 'UINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.FLOAT: 'FLOAT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.UTINYINT: 'UTINYINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.UDECIMAL: 'UDECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.UMEDIUMINT: 'UMEDIUMINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.INT128: 'INT128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NAME: 'NAME'>, <Type.DOUBLE: 'DOUBLE'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.UINT256: 'UINT256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.UBIGINT: 'UBIGINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.MEDIUMINT: 'MEDIUMINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.SMALLINT: 'SMALLINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.INT256: 'INT256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.DECIMAL256: 'DECIMAL256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.BIT: 'BIT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.DECIMAL: 'DECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.DECIMAL32: 'DECIMAL32'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.SMALLMONEY: 'SMALLMONEY'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.BIGDECIMAL: 'BIGDECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.UINT128: 'UINT128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.DECIMAL64: 'DECIMAL64'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.DECIMAL128: 'DECIMAL128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.TINYINT: 'TINYINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.UDOUBLE: 'UDOUBLE'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.USMALLINT: 'USMALLINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.MONEY: 'MONEY'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.INT: 'INT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.BIGINT: 'BIGINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.UINT: 'UINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.FLOAT: 'FLOAT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.UTINYINT: 'UTINYINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.UDECIMAL: 'UDECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.UMEDIUMINT: 'UMEDIUMINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.INT128: 'INT128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.NVARCHAR: 'NVARCHAR'>, <Type.DOUBLE: 'DOUBLE'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.UINT256: 'UINT256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.UBIGINT: 'UBIGINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.MEDIUMINT: 'MEDIUMINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.SMALLINT: 'SMALLINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.INT256: 'INT256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.DECIMAL256: 'DECIMAL256'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.BIT: 'BIT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.DECIMAL: 'DECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.DECIMAL32: 'DECIMAL32'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.SMALLMONEY: 'SMALLMONEY'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.BIGDECIMAL: 'BIGDECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.UINT128: 'UINT128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.DECIMAL64: 'DECIMAL64'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.DECIMAL128: 'DECIMAL128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.TINYINT: 'TINYINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.UDOUBLE: 'UDOUBLE'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.USMALLINT: 'USMALLINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.MONEY: 'MONEY'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.INT: 'INT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.BIGINT: 'BIGINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.UINT: 'UINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.FLOAT: 'FLOAT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.UTINYINT: 'UTINYINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.UDECIMAL: 'UDECIMAL'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.UMEDIUMINT: 'UMEDIUMINT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.INT128: 'INT128'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.CHAR: 'CHAR'>, <Type.DOUBLE: 'DOUBLE'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT256: 'UINT256'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UBIGINT: 'UBIGINT'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.MEDIUMINT: 'MEDIUMINT'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.SMALLINT: 'SMALLINT'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT256: 'INT256'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL256: 'DECIMAL256'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIT: 'BIT'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL: 'DECIMAL'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL32: 'DECIMAL32'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.SMALLMONEY: 'SMALLMONEY'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIGDECIMAL: 'BIGDECIMAL'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT128: 'UINT128'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL64: 'DECIMAL64'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL128: 'DECIMAL128'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TINYINT: 'TINYINT'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UDOUBLE: 'UDOUBLE'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.USMALLINT: 'USMALLINT'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.MONEY: 'MONEY'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT: 'INT'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIGINT: 'BIGINT'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT: 'UINT'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.FLOAT: 'FLOAT'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UTINYINT: 'UTINYINT'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UDECIMAL: 'UDECIMAL'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UMEDIUMINT: 'UMEDIUMINT'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT128: 'INT128'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DOUBLE: 'DOUBLE'>, <Type.NCHAR: 'NCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT256: 'UINT256'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UBIGINT: 'UBIGINT'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.MEDIUMINT: 'MEDIUMINT'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.SMALLINT: 'SMALLINT'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT256: 'INT256'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL256: 'DECIMAL256'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIT: 'BIT'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL: 'DECIMAL'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL32: 'DECIMAL32'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.SMALLMONEY: 'SMALLMONEY'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIGDECIMAL: 'BIGDECIMAL'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT128: 'UINT128'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL64: 'DECIMAL64'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL128: 'DECIMAL128'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TINYINT: 'TINYINT'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UDOUBLE: 'UDOUBLE'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.USMALLINT: 'USMALLINT'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.MONEY: 'MONEY'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT: 'INT'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIGINT: 'BIGINT'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT: 'UINT'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.FLOAT: 'FLOAT'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UTINYINT: 'UTINYINT'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UDECIMAL: 'UDECIMAL'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UMEDIUMINT: 'UMEDIUMINT'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT128: 'INT128'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DOUBLE: 'DOUBLE'>, <Type.TEXT: 'TEXT'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT256: 'UINT256'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UBIGINT: 'UBIGINT'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.MEDIUMINT: 'MEDIUMINT'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.SMALLINT: 'SMALLINT'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT256: 'INT256'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL256: 'DECIMAL256'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIT: 'BIT'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL: 'DECIMAL'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL32: 'DECIMAL32'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.SMALLMONEY: 'SMALLMONEY'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIGDECIMAL: 'BIGDECIMAL'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT128: 'UINT128'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL64: 'DECIMAL64'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL128: 'DECIMAL128'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TINYINT: 'TINYINT'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UDOUBLE: 'UDOUBLE'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.USMALLINT: 'USMALLINT'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.MONEY: 'MONEY'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT: 'INT'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIGINT: 'BIGINT'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT: 'UINT'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.FLOAT: 'FLOAT'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UTINYINT: 'UTINYINT'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UDECIMAL: 'UDECIMAL'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UMEDIUMINT: 'UMEDIUMINT'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT128: 'INT128'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DOUBLE: 'DOUBLE'>, <Type.VARCHAR: 'VARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT256: 'UINT256'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UBIGINT: 'UBIGINT'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.MEDIUMINT: 'MEDIUMINT'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.SMALLINT: 'SMALLINT'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT256: 'INT256'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL256: 'DECIMAL256'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIT: 'BIT'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL: 'DECIMAL'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL32: 'DECIMAL32'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.SMALLMONEY: 'SMALLMONEY'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIGDECIMAL: 'BIGDECIMAL'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT128: 'UINT128'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL64: 'DECIMAL64'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL128: 'DECIMAL128'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TINYINT: 'TINYINT'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UDOUBLE: 'UDOUBLE'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.USMALLINT: 'USMALLINT'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.MONEY: 'MONEY'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT: 'INT'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIGINT: 'BIGINT'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT: 'UINT'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.FLOAT: 'FLOAT'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UTINYINT: 'UTINYINT'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UDECIMAL: 'UDECIMAL'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UMEDIUMINT: 'UMEDIUMINT'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT128: 'INT128'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DOUBLE: 'DOUBLE'>, <Type.NAME: 'NAME'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT256: 'UINT256'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UBIGINT: 'UBIGINT'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.MEDIUMINT: 'MEDIUMINT'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.SMALLINT: 'SMALLINT'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT256: 'INT256'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL256: 'DECIMAL256'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIT: 'BIT'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL: 'DECIMAL'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL32: 'DECIMAL32'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.SMALLMONEY: 'SMALLMONEY'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIGDECIMAL: 'BIGDECIMAL'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT128: 'UINT128'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL64: 'DECIMAL64'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL128: 'DECIMAL128'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TINYINT: 'TINYINT'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UDOUBLE: 'UDOUBLE'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.USMALLINT: 'USMALLINT'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.MONEY: 'MONEY'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT: 'INT'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIGINT: 'BIGINT'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT: 'UINT'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.FLOAT: 'FLOAT'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UTINYINT: 'UTINYINT'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UDECIMAL: 'UDECIMAL'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UMEDIUMINT: 'UMEDIUMINT'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT128: 'INT128'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DOUBLE: 'DOUBLE'>, <Type.NVARCHAR: 'NVARCHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT256: 'UINT256'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UBIGINT: 'UBIGINT'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.MEDIUMINT: 'MEDIUMINT'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.SMALLINT: 'SMALLINT'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT256: 'INT256'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL256: 'DECIMAL256'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIT: 'BIT'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL: 'DECIMAL'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL32: 'DECIMAL32'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.SMALLMONEY: 'SMALLMONEY'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIGDECIMAL: 'BIGDECIMAL'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT128: 'UINT128'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL64: 'DECIMAL64'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DECIMAL128: 'DECIMAL128'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.TINYINT: 'TINYINT'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UDOUBLE: 'UDOUBLE'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.USMALLINT: 'USMALLINT'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.MONEY: 'MONEY'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT: 'INT'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.BIGINT: 'BIGINT'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UINT: 'UINT'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.FLOAT: 'FLOAT'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UTINYINT: 'UTINYINT'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UDECIMAL: 'UDECIMAL'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.UMEDIUMINT: 'UMEDIUMINT'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.INT128: 'INT128'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DOUBLE: 'DOUBLE'>, <Type.CHAR: 'CHAR'>): <function TypeAnnotator.<dictcomp>.<lambda>>, (<Type.DATE: 'DATE'>, <Type.INTERVAL: 'INTERVAL'>): <function TypeAnnotator.<lambda>>, (<Type.INTERVAL: 'INTERVAL'>, <Type.DATE: 'DATE'>): <function TypeAnnotator.<lambda>>}
schema
annotators
coerces_to
binary_coercions
def annotate(self, expression: ~E) -> ~E:
228    def annotate(self, expression: E) -> E:
229        for scope in traverse_scope(expression):
230            self.annotate_scope(scope)
231
232        # This takes care of non-traversable expressions
233        expression = self._maybe_annotate(expression)
234
235        # Replace NULL type with UNKNOWN, since the former is not an actual type;
236        # it is mostly used to aid type coercion, e.g. in query set operations.
237        for expr in self._null_expressions.values():
238            expr.type = exp.DataType.Type.UNKNOWN
239
240        return expression
def annotate_scope(self, scope: sqlglot.optimizer.scope.Scope) -> None:
242    def annotate_scope(self, scope: Scope) -> None:
243        selects = {}
244        for name, source in scope.sources.items():
245            if not isinstance(source, Scope):
246                continue
247
248            expression = source.expression
249            if isinstance(expression, exp.UDTF):
250                values = []
251
252                if isinstance(expression, exp.Lateral):
253                    if isinstance(expression.this, exp.Explode):
254                        values = [expression.this.this]
255                elif isinstance(expression, exp.Unnest):
256                    values = [expression]
257                elif not isinstance(expression, exp.TableFromRows):
258                    values = expression.expressions[0].expressions
259
260                if not values:
261                    continue
262
263                selects[name] = {
264                    alias: column.type
265                    for alias, column in zip(expression.alias_column_names, values)
266                }
267            elif isinstance(expression, exp.SetOperation) and len(expression.left.selects) == len(
268                expression.right.selects
269            ):
270                selects[name] = col_types = self._setop_column_types.setdefault(id(expression), {})
271
272                if not col_types:
273                    # Process a chain / sub-tree of set operations
274                    for set_op in expression.walk(
275                        prune=lambda n: not isinstance(n, (exp.SetOperation, exp.Subquery))
276                    ):
277                        if not isinstance(set_op, exp.SetOperation):
278                            continue
279
280                        if set_op.args.get("by_name"):
281                            r_type_by_select = {
282                                s.alias_or_name: s.type for s in set_op.right.selects
283                            }
284                            setop_cols = {
285                                s.alias_or_name: self._maybe_coerce(
286                                    t.cast(exp.DataType, s.type),
287                                    r_type_by_select.get(s.alias_or_name)
288                                    or exp.DataType.Type.UNKNOWN,
289                                )
290                                for s in set_op.left.selects
291                            }
292                        else:
293                            setop_cols = {
294                                ls.alias_or_name: self._maybe_coerce(
295                                    t.cast(exp.DataType, ls.type), t.cast(exp.DataType, rs.type)
296                                )
297                                for ls, rs in zip(set_op.left.selects, set_op.right.selects)
298                            }
299
300                        # Coerce intermediate results with the previously registered types, if they exist
301                        for col_name, col_type in setop_cols.items():
302                            col_types[col_name] = self._maybe_coerce(
303                                col_type, col_types.get(col_name, exp.DataType.Type.NULL)
304                            )
305
306            else:
307                selects[name] = {s.alias_or_name: s.type for s in expression.selects}
308
309        # First annotate the current scope's column references
310        for col in scope.columns:
311            if not col.table:
312                continue
313
314            source = scope.sources.get(col.table)
315            if isinstance(source, exp.Table):
316                self._set_type(col, self.schema.get_column_type(source, col))
317            elif source:
318                if col.table in selects and col.name in selects[col.table]:
319                    self._set_type(col, selects[col.table][col.name])
320                elif isinstance(source.expression, exp.Unnest):
321                    self._set_type(col, source.expression.type)
322
323        if isinstance(self.schema, MappingSchema):
324            for table_column in scope.table_columns:
325                source = scope.sources.get(table_column.name)
326
327                if isinstance(source, exp.Table):
328                    schema = self.schema.find(
329                        source, raise_on_missing=False, ensure_data_types=True
330                    )
331                    if not isinstance(schema, dict):
332                        continue
333
334                    struct_type = exp.DataType(
335                        this=exp.DataType.Type.STRUCT,
336                        expressions=[
337                            exp.ColumnDef(this=exp.to_identifier(c), kind=kind)
338                            for c, kind in schema.items()
339                        ],
340                        nested=True,
341                    )
342                    self._set_type(table_column, struct_type)
343                elif (
344                    isinstance(source, Scope)
345                    and isinstance(source.expression, exp.Query)
346                    and (
347                        source.expression.meta.get("query_type") or exp.DataType.build("UNKNOWN")
348                    ).is_type(exp.DataType.Type.STRUCT)
349                ):
350                    self._set_type(table_column, source.expression.meta["query_type"])
351
352        # Then (possibly) annotate the remaining expressions in the scope
353        self._maybe_annotate(scope.expression)
354
355        if self.schema.dialect == "bigquery" and isinstance(scope.expression, exp.Query):
356            struct_type = exp.DataType(
357                this=exp.DataType.Type.STRUCT,
358                expressions=[
359                    exp.ColumnDef(
360                        this=exp.to_identifier(select.output_name),
361                        kind=select.type.copy() if select.type else None,
362                    )
363                    for select in scope.expression.selects
364                ],
365                nested=True,
366            )
367
368            if not any(
369                cd.kind.is_type(exp.DataType.Type.UNKNOWN)
370                for cd in struct_type.expressions
371                if cd.kind
372            ):
373                # We don't use `_set_type` on purpose here. If we annotated the query directly, then
374                # using it in other contexts (e.g., ARRAY(<query>)) could result in incorrect type
375                # annotations, i.e., it shouldn't be interpreted as a STRUCT value.
376                scope.expression.meta["query_type"] = struct_type