diff --git a/.github/workflows/ibis-backends-cloud.yml b/.github/workflows/ibis-backends-cloud.yml index 677cda613d35..6292529714cf 100644 --- a/.github/workflows/ibis-backends-cloud.yml +++ b/.github/workflows/ibis-backends-cloud.yml @@ -46,8 +46,8 @@ jobs: - "3.9" - "3.11" backend: - # - name: bigquery - # title: BigQuery + - name: bigquery + title: BigQuery - name: snowflake title: Snowflake steps: diff --git a/ibis/backends/base/sqlglot/compiler.py b/ibis/backends/base/sqlglot/compiler.py index 1cc732ac89a0..f8e14222c3fe 100644 --- a/ibis/backends/base/sqlglot/compiler.py +++ b/ibis/backends/base/sqlglot/compiler.py @@ -20,6 +20,7 @@ import ibis.expr.datatypes as dt import ibis.expr.operations as ops from ibis.backends.base.sqlglot.rewrites import Select, Window, sqlize +from ibis.expr.operations.udf import InputType from ibis.expr.rewrites import ( add_one_to_nth_value_input, add_order_by_to_empty_ranking_window_functions, @@ -874,9 +875,20 @@ def visit_RowID(self, op, *, table): # TODO(kszucs): this should be renamed to something UDF related def __sql_name__(self, op: ops.ScalarUDF | ops.AggUDF) -> str: + # for builtin functions use the exact function name, otherwise use the + # generated name to handle the case of redefinition + funcname = ( + op.__func_name__ + if op.__input_type__ == InputType.BUILTIN + else type(op).__name__ + ) + # not actually a table, but easier to quote individual namespace # components this way - return sg.table(op.__func_name__, db=op.__udf_namespace__).sql(self.dialect) + namespace = op.__udf_namespace__ + return sg.table(funcname, db=namespace.schema, catalog=namespace.database).sql( + self.dialect + ) @visit_node.register(ops.ScalarUDF) def visit_ScalarUDF(self, op, **kw): @@ -919,6 +931,23 @@ def _dedup_name( else value.as_(key, quoted=self.quoted) ) + @staticmethod + def _gen_valid_name(name: str) -> str: + """Generate a valid name for a value expression. + + Override this method if the dialect has restrictions on valid + identifiers even when quoted. + + See the BigQuery backend's implementation for an example. + """ + return name + + def _cleanup_names(self, exprs: Mapping[str, sge.Expression]): + """Compose `_gen_valid_name` and `_dedup_name` to clean up names in projections.""" + return starmap( + self._dedup_name, toolz.keymap(self._gen_valid_name, exprs).items() + ) + @visit_node.register(Select) def visit_Select(self, op, *, parent, selections, predicates, sort_keys): # if we've constructed a useless projection return the parent relation @@ -928,9 +957,7 @@ def visit_Select(self, op, *, parent, selections, predicates, sort_keys): result = parent if selections: - result = sg.select(*starmap(self._dedup_name, selections.items())).from_( - result - ) + result = sg.select(*self._cleanup_names(selections)).from_(result) if predicates: result = result.where(*predicates) @@ -942,7 +969,7 @@ def visit_Select(self, op, *, parent, selections, predicates, sort_keys): @visit_node.register(ops.DummyTable) def visit_DummyTable(self, op, *, values): - return sg.select(*starmap(self._dedup_name, values.items())) + return sg.select(*self._cleanup_names(values)) @visit_node.register(ops.UnboundTable) def visit_UnboundTable( @@ -978,7 +1005,7 @@ def visit_SelfReference(self, op, *, parent, identifier): @visit_node.register(ops.JoinChain) def visit_JoinChain(self, op, *, first, rest, values): - result = sg.select(*starmap(self._dedup_name, values.items())).from_(first) + result = sg.select(*self._cleanup_names(values)).from_(first) for link in rest: if isinstance(link, sge.Alias): @@ -1019,15 +1046,9 @@ def visit_JoinLink(self, op, *, how, table, predicates): on = sg.and_(*predicates) if predicates else None return sge.Join(this=table, side=sides[how], kind=kinds[how], on=on) - @staticmethod - def _gen_valid_name(name: str) -> str: - return name - @visit_node.register(ops.Project) def visit_Project(self, op, *, parent, values): - # needs_alias should never be true here in explicitly, but it may get - # passed via a (recursive) call to translate_val - return sg.select(*starmap(self._dedup_name, values.items())).from_(parent) + return sg.select(*self._cleanup_names(values)).from_(parent) @staticmethod def _generate_groups(groups): @@ -1036,12 +1057,7 @@ def _generate_groups(groups): @visit_node.register(ops.Aggregate) def visit_Aggregate(self, op, *, parent, groups, metrics): sel = sg.select( - *starmap( - self._dedup_name, toolz.keymap(self._gen_valid_name, groups).items() - ), - *starmap( - self._dedup_name, toolz.keymap(self._gen_valid_name, metrics).items() - ), + *self._cleanup_names(groups), *self._cleanup_names(metrics) ).from_(parent) if groups: @@ -1190,21 +1206,15 @@ def visit_FillNa(self, op, *, parent, replacements): for name, dtype in op.schema.items() if dtype.nullable } - exprs = [ - ( - sg.alias( - sge.Coalesce( - this=sg.column(col, quoted=self.quoted), - expressions=[sge.convert(alt)], - ), - col, - ) + exprs = { + col: ( + self.f.coalesce(sg.column(col, quoted=self.quoted), sge.convert(alt)) if (alt := mapping.get(col)) is not None else sg.column(col, quoted=self.quoted) ) for col in op.schema.keys() - ] - return sg.select(*exprs).from_(parent) + } + return sg.select(*self._cleanup_names(exprs)).from_(parent) @visit_node.register(ops.View) def visit_View(self, op, *, child, name: str): diff --git a/ibis/backends/base/sqlglot/datatypes.py b/ibis/backends/base/sqlglot/datatypes.py index b1d4c336739e..e09e582f9671 100644 --- a/ibis/backends/base/sqlglot/datatypes.py +++ b/ibis/backends/base/sqlglot/datatypes.py @@ -620,3 +620,122 @@ class PySparkType(SqlglotType): default_decimal_precision = 38 default_decimal_scale = 18 + + +class BigQueryType(SqlglotType): + dialect = "bigquery" + + default_decimal_precision = 38 + default_decimal_scale = 9 + + @classmethod + def _from_sqlglot_NUMERIC(cls) -> dt.Decimal: + return dt.Decimal( + cls.default_decimal_precision, + cls.default_decimal_scale, + nullable=cls.default_nullable, + ) + + @classmethod + def _from_sqlglot_BIGNUMERIC(cls) -> dt.Decimal: + return dt.Decimal(76, 38, nullable=cls.default_nullable) + + @classmethod + def _from_sqlglot_DATETIME(cls) -> dt.Decimal: + return dt.Timestamp(timezone=None, nullable=cls.default_nullable) + + @classmethod + def _from_sqlglot_TIMESTAMP(cls) -> dt.Decimal: + return dt.Timestamp(timezone="UTC", nullable=cls.default_nullable) + + @classmethod + def _from_sqlglot_GEOGRAPHY(cls) -> dt.Decimal: + return dt.GeoSpatial( + geotype="geography", srid=4326, nullable=cls.default_nullable + ) + + @classmethod + def _from_sqlglot_TINYINT(cls) -> dt.Int64: + return dt.Int64(nullable=cls.default_nullable) + + _from_sqlglot_UINT = ( + _from_sqlglot_USMALLINT + ) = ( + _from_sqlglot_UTINYINT + ) = _from_sqlglot_INT = _from_sqlglot_SMALLINT = _from_sqlglot_TINYINT + + @classmethod + def _from_sqlglot_UBIGINT(cls) -> dt.Int64: + raise TypeError("Unsigned BIGINT isn't representable in BigQuery INT64") + + @classmethod + def _from_sqlglot_FLOAT(cls) -> dt.Double: + return dt.Float64(nullable=cls.default_nullable) + + @classmethod + def _from_sqlglot_MAP(cls) -> dt.Map: + raise NotImplementedError( + "Cannot convert sqlglot Map type to ibis type: maps are not supported in BigQuery" + ) + + @classmethod + def _from_ibis_Map(cls, dtype: dt.Map) -> sge.DataType: + raise NotImplementedError( + "Cannot convert Ibis Map type to BigQuery type: maps are not supported in BigQuery" + ) + + @classmethod + def _from_ibis_Timestamp(cls, dtype: dt.Timestamp) -> sge.DataType: + if dtype.timezone is None: + return sge.DataType(this=sge.DataType.Type.DATETIME) + elif dtype.timezone == "UTC": + return sge.DataType(this=sge.DataType.Type.TIMESTAMPTZ) + else: + raise TypeError( + "BigQuery does not support timestamps with timezones other than 'UTC'" + ) + + @classmethod + def _from_ibis_Decimal(cls, dtype: dt.Decimal) -> sge.DataType: + precision = dtype.precision + scale = dtype.scale + if (precision, scale) == (76, 38): + return sge.DataType(this=sge.DataType.Type.BIGDECIMAL) + elif (precision, scale) in ((38, 9), (None, None)): + return sge.DataType(this=sge.DataType.Type.DECIMAL) + else: + raise TypeError( + "BigQuery only supports decimal types with precision of 38 and " + f"scale of 9 (NUMERIC) or precision of 76 and scale of 38 (BIGNUMERIC). " + f"Current precision: {dtype.precision}. Current scale: {dtype.scale}" + ) + + @classmethod + def _from_ibis_UInt64(cls, dtype: dt.UInt64) -> sge.DataType: + raise TypeError( + f"Conversion from {dtype} to BigQuery integer type (Int64) is lossy" + ) + + @classmethod + def _from_ibis_UInt32(cls, dtype: dt.UInt32) -> sge.DataType: + return sge.DataType(this=sge.DataType.Type.BIGINT) + + _from_ibis_UInt8 = _from_ibis_UInt16 = _from_ibis_UInt32 + + @classmethod + def _from_ibis_GeoSpatial(cls, dtype: dt.GeoSpatial) -> sge.DataType: + if (dtype.geotype, dtype.srid) == ("geography", 4326): + return sge.DataType(this=sge.DataType.Type.GEOGRAPHY) + else: + raise TypeError( + "BigQuery geography uses points on WGS84 reference ellipsoid." + f"Current geotype: {dtype.geotype}, Current srid: {dtype.srid}" + ) + + +class BigQueryUDFType(BigQueryType): + @classmethod + def _from_ibis_Int64(cls, dtype: dt.Int64) -> sge.DataType: + raise com.UnsupportedBackendType( + "int64 is not a supported input or output type in BigQuery UDFs; use float64 instead" + ) diff --git a/ibis/backends/bigquery/__init__.py b/ibis/backends/bigquery/__init__.py index 6d37cc87bc45..5e0cddf43f9a 100644 --- a/ibis/backends/bigquery/__init__.py +++ b/ibis/backends/bigquery/__init__.py @@ -3,11 +3,10 @@ from __future__ import annotations import concurrent.futures -import contextlib import glob import os import re -from functools import partial +from contextlib import contextmanager from typing import TYPE_CHECKING, Any, Callable, Optional from urllib.parse import parse_qs, urlparse @@ -17,6 +16,7 @@ import pandas as pd import pydata_google_auth import sqlglot as sg +import sqlglot.expressions as sge from pydata_google_auth import cache import ibis @@ -25,7 +25,8 @@ import ibis.expr.types as ir from ibis import util from ibis.backends.base import CanCreateSchema, Database -from ibis.backends.base.sql import BaseSQLBackend +from ibis.backends.base.sqlglot import SQLGlotBackend +from ibis.backends.base.sqlglot.datatypes import BigQueryType from ibis.backends.bigquery.client import ( BigQueryCursor, bigquery_param, @@ -34,19 +35,17 @@ schema_from_bigquery_table, ) from ibis.backends.bigquery.compiler import BigQueryCompiler -from ibis.backends.bigquery.datatypes import BigQuerySchema, BigQueryType - -with contextlib.suppress(ImportError): - from ibis.backends.bigquery.udf import udf # noqa: F401 +from ibis.backends.bigquery.datatypes import BigQuerySchema +from ibis.backends.bigquery.udf.core import PythonToJavaScriptTranslator if TYPE_CHECKING: - from collections.abc import Iterable, Mapping + from collections.abc import Iterable, Iterator, Mapping from pathlib import Path import pyarrow as pa from google.cloud.bigquery.table import RowIterator - import ibis.expr.schema as sch + import ibis.expr.types as dt SCOPES = ["https://www.googleapis.com/auth/bigquery"] EXTERNAL_DATA_SCOPES = [ @@ -82,36 +81,40 @@ def _create_client_info_gapic(application_name): return ClientInfo(user_agent=_create_user_agent(application_name)) -def _anonymous_unnest_to_explode(node: sg.exp.Expression): - """Convert `ANONYMOUS` `unnest` function calls to `EXPLODE` calls. - - This allows us to generate DuckDB-like `UNNEST` calls and let sqlglot do - the work of transforming those into the correct BigQuery SQL. - """ - if isinstance(node, sg.exp.Anonymous) and node.this.lower() == "unnest": - return sg.exp.Explode(this=node.expressions[0]) - return node - - _MEMTABLE_PATTERN = re.compile(r"^_?ibis_(?:pandas|pyarrow)_memtable_[a-z0-9]{26}$") def _qualify_memtable( - node: sg.exp.Expression, *, dataset: str | None, project: str | None -) -> sg.exp.Expression: + node: sge.Expression, *, dataset: str | None, project: str | None +) -> sge.Expression: """Add a BigQuery dataset and project to memtable references.""" - if ( - isinstance(node, sg.exp.Table) - and _MEMTABLE_PATTERN.match(node.name) is not None - ): + if isinstance(node, sge.Table) and _MEMTABLE_PATTERN.match(node.name) is not None: node.args["db"] = dataset node.args["catalog"] = project return node -class Backend(BaseSQLBackend, CanCreateSchema): +def _remove_nulls_first_from_invalid_window_orderings( + node: sge.Expression, +) -> sge.Expression: + if isinstance(node, sge.Window): + order = node.args.get("order") + if order is not None: + for key in order.args["expressions"]: + kargs = key.args + if kargs.get("desc") is True and kargs.get("nulls_first", False): + kargs["nulls_first"] = False + elif kargs.get("desc") is False and not kargs.setdefault( + "nulls_first", True + ): + kargs["nulls_first"] = True + + return node + + +class Backend(SQLGlotBackend, CanCreateSchema): name = "bigquery" - compiler = BigQueryCompiler + compiler = BigQueryCompiler() supports_in_memory_tables = True supports_python_udfs = False @@ -455,20 +458,20 @@ def create_schema( **options: Any, ) -> None: properties = [ - sg.exp.Property(this=sg.to_identifier(name), value=sg.exp.convert(value)) + sge.Property(this=sg.to_identifier(name), value=sge.convert(value)) for name, value in (options or {}).items() ] if collate is not None: properties.append( - sg.exp.CollateProperty(this=sg.exp.convert(collate), default=True) + sge.CollateProperty(this=sge.convert(collate), default=True) ) - stmt = sg.exp.Create( + stmt = sge.Create( kind="SCHEMA", this=sg.table(name, db=database), exists=force, - properties=sg.exp.Properties(expressions=properties), + properties=sge.Properties(expressions=properties), ) self.raw_sql(stmt.sql(self.name)) @@ -481,7 +484,7 @@ def drop_schema( cascade: bool = False, ) -> None: """Drop a BigQuery dataset.""" - stmt = sg.exp.Drop( + stmt = sge.Drop( kind="SCHEMA", this=sg.table(name, db=database), exists=force, @@ -499,7 +502,7 @@ def table( "`database` specifier. Include a `schema` argument." ) - table = sg.parse_one(name, into=sg.exp.Table, read=self.name) + table = sg.parse_one(name, into=sge.Table, read=self.name) # table.catalog will be the empty string if table.catalog: @@ -520,11 +523,11 @@ def table( schema = table.db if database is not None and schema is None: - database = sg.parse_one(database, into=sg.exp.Table, read=self.name) + database = sg.parse_one(database, into=sge.Table, read=self.name) database.args["quoted"] = False database = database.sql(dialect=self.name) elif database is None and schema is not None: - database = sg.parse_one(schema, into=sg.exp.Table, read=self.name) + database = sg.parse_one(schema, into=sge.Table, read=self.name) database.args["quoted"] = False database = database.sql(dialect=self.name) else: @@ -567,7 +570,7 @@ def _make_session(self) -> tuple[str, str]: dataset_id=query.destination.dataset_id, ) - def _get_schema_using_query(self, query: str) -> sch.Schema: + def _metadata(self, query: str) -> Iterator[tuple[name, dt.DataType]]: self._make_session() job = self.client.query( @@ -575,9 +578,11 @@ def _get_schema_using_query(self, query: str) -> sch.Schema: job_config=bq.QueryJobConfig(dry_run=True, use_query_cache=False), project=self.billing_project, ) - return BigQuerySchema.to_ibis(job.schema) + return ( + (f.name, BigQuerySchema._dtype_from_bigquery_field(f)) for f in job.schema + ) - def _execute(self, stmt, results=True, query_parameters=None): + def _execute(self, stmt, query_parameters=None): self._make_session() job_config = bq.job.QueryJobConfig(query_parameters=query_parameters or []) @@ -587,12 +592,12 @@ def _execute(self, stmt, results=True, query_parameters=None): query.result() # blocks until finished return BigQueryCursor(query) - def compile( + def _to_sqlglot( self, expr: ir.Expr, limit: str | None = None, params: Mapping[ir.Expr, Any] | None = None, - **_, + **kwargs, ) -> Any: """Compile an Ibis expression. @@ -605,6 +610,8 @@ def compile( of values/rows. Overrides any limit already set on the expression. params Named unbound parameters + kwargs + Keyword arguments passed to the compiler Returns ------- @@ -614,24 +621,15 @@ def compile( """ self._make_session() self._define_udf_translation_rules(expr) - sql = self.compiler.to_ast_ensure_limit(expr, limit, params=params).compile() - - return ";\n\n".join( - # convert unnest function calls to explode - query.transform(_anonymous_unnest_to_explode) - # add dataset and project to memtable references - .transform( - partial( - _qualify_memtable, - dataset=getattr(self._session_dataset, "dataset_id", None), - project=getattr(self._session_dataset, "project", None), - ) - ) - .sql(dialect=self.name, pretty=True) - for query in sg.parse(sql, read=self.name) - ) + sql = super()._to_sqlglot(expr, limit=limit, params=params, **kwargs) - def raw_sql(self, query: str, results=False, params=None): + return sql.transform( + _qualify_memtable, + dataset=getattr(self._session_dataset, "dataset_id", None), + project=getattr(self._session_dataset, "project", None), + ).transform(_remove_nulls_first_from_invalid_window_orderings) + + def raw_sql(self, query: str, params=None): query_parameters = [ bigquery_param( param.type(), @@ -644,7 +642,7 @@ def raw_sql(self, query: str, results=False, params=None): ) for param, value in (params or {}).items() ] - return self._execute(query, results=results, query_parameters=query_parameters) + return self._execute(query, query_parameters=query_parameters) @property def current_database(self) -> str: @@ -663,6 +661,23 @@ def database(self, name=None): ) return Database(name or self.dataset, self) + def compile( + self, expr: ir.Expr, limit: str | None = None, params=None, **kwargs: Any + ): + """Compile an Ibis expression to a SQL string.""" + query = self._to_sqlglot(expr, limit=limit, params=params, **kwargs) + udf_sources = [] + for udf_node in expr.op().find(ops.ScalarUDF): + compile_func = getattr( + self, f"_compile_{udf_node.__input_type__.name.lower()}_udf" + ) + if sql := compile_func(udf_node): + udf_sources.append(sql.sql(self.name, pretty=True)) + + sql = ";\n".join([*udf_sources, query.sql(dialect=self.name, pretty=True)]) + self._log(sql) + return sql + def execute(self, expr, params=None, limit="default", **kwargs): """Compile and execute the given Ibis expression. @@ -700,11 +715,11 @@ def execute(self, expr, params=None, limit="default", **kwargs): return expr.__pandas_result__(result) def fetch_from_cursor(self, cursor, schema): - from ibis.formats.pandas import PandasData + from ibis.backends.bigquery.converter import BigQueryPandasData arrow_t = self._cursor_to_arrow(cursor) df = arrow_t.to_pandas(timestamp_as_object=True) - return PandasData.convert_table(df, schema) + return BigQueryPandasData.convert_table(df, schema) def _cursor_to_arrow( self, @@ -830,7 +845,7 @@ def list_tables( "Include a `schema` argument." ) elif database is None and schema is not None: - database = sg.parse_one(schema, into=sg.exp.Table, read=self.name) + database = sg.parse_one(schema, into=sge.Table, read=self.name) database.args["quoted"] = False database = database.sql(dialect=self.name) else: @@ -918,15 +933,13 @@ def create_table( if default_collate is not None: properties.append( - sg.exp.CollateProperty( - this=sg.exp.convert(default_collate), default=True - ) + sge.CollateProperty(this=sge.convert(default_collate), default=True) ) if partition_by is not None: properties.append( - sg.exp.PartitionedByProperty( - this=sg.exp.Tuple( + sge.PartitionedByProperty( + this=sge.Tuple( expressions=list(map(sg.to_identifier, partition_by)) ) ) @@ -934,11 +947,11 @@ def create_table( if cluster_by is not None: properties.append( - sg.exp.Cluster(expressions=list(map(sg.to_identifier, cluster_by))) + sge.Cluster(expressions=list(map(sg.to_identifier, cluster_by))) ) properties.extend( - sg.exp.Property(this=sg.to_identifier(name), value=sg.exp.convert(value)) + sge.Property(this=sg.to_identifier(name), value=sge.convert(value)) for name, value in (options or {}).items() ) @@ -960,7 +973,7 @@ def create_table( dataset = database or self.current_schema try: - table = sg.parse_one(name, into=sg.exp.Table, read="bigquery") + table = sg.parse_one(name, into=sge.Table, read="bigquery") except sg.ParseError: table = sg.table(name, db=dataset, catalog=project_id) else: @@ -971,25 +984,23 @@ def create_table( table.args["catalog"] = project_id column_defs = [ - sg.exp.ColumnDef( + sge.ColumnDef( this=name, kind=BigQueryType.from_ibis(typ), constraints=( None if typ.nullable or typ.is_array() - else [ - sg.exp.ColumnConstraint(kind=sg.exp.NotNullColumnConstraint()) - ] + else [sge.ColumnConstraint(kind=sge.NotNullColumnConstraint())] ), ) for name, typ in (schema or {}).items() ] - stmt = sg.exp.Create( + stmt = sge.Create( kind="TABLE", - this=sg.exp.Schema(this=table, expressions=column_defs or None), + this=sge.Schema(this=table, expressions=column_defs or None), replace=overwrite, - properties=sg.exp.Properties(expressions=properties), + properties=sge.Properties(expressions=properties), expression=None if obj is None else self.compile(obj), ) @@ -1006,7 +1017,7 @@ def drop_table( database: str | None = None, force: bool = False, ) -> None: - stmt = sg.exp.Drop( + stmt = sge.Drop( kind="TABLE", this=sg.table( name, @@ -1026,7 +1037,7 @@ def create_view( database: str | None = None, overwrite: bool = False, ) -> ir.Table: - stmt = sg.exp.Create( + stmt = sge.Create( kind="VIEW", this=sg.table( name, @@ -1048,7 +1059,7 @@ def drop_view( database: str | None = None, force: bool = False, ) -> None: - stmt = sg.exp.Drop( + stmt = sge.Drop( kind="VIEW", this=sg.table( name, @@ -1069,6 +1080,84 @@ def _clean_up_cached_table(self, op): database=self._session_dataset.project, ) + def _get_udf_source(self, udf_node: ops.ScalarUDF): + name = type(udf_node).__name__ + type_mapper = self.compiler.udf_type_mapper + + body = PythonToJavaScriptTranslator(udf_node.__func__).compile() + config = udf_node.__config__ + libraries = config.get("libraries", []) + + signature = [ + sge.ColumnDef( + this=sg.to_identifier(name), + kind=type_mapper.from_ibis(param.annotation.pattern.dtype), + ) + for name, param in udf_node.__signature__.parameters.items() + ] + + lines = ['"""'] + + if config.get("strict", True): + lines.append('"use strict";') + + lines += [ + body, + "", + f"return {udf_node.__func_name__}({', '.join(udf_node.argnames)});", + '"""', + ] + + func = sge.Create( + kind="FUNCTION", + this=sge.UserDefinedFunction( + this=sg.to_identifier(name), expressions=signature, wrapped=True + ), + # not exactly what I had in mind, but it works + # + # quoting is too simplistic to handle multiline strings + expression=sge.Var(this="\n".join(lines)), + exists=False, + properties=sge.Properties( + expressions=[ + sge.TemporaryProperty(), + sge.ReturnsProperty(this=type_mapper.from_ibis(udf_node.dtype)), + sge.StabilityProperty( + this="IMMUTABLE" if config.get("determinism") else "VOLATILE" + ), + sge.LanguageProperty(this=sg.to_identifier("js")), + ] + + [ + sge.Property( + this=sg.to_identifier("library"), + value=self.compiler.f.array(*libraries), + ) + ] + * bool(libraries) + ), + ) + + return func + + def _compile_builtin_udf(self, udf_node: ops.ScalarUDF) -> None: + """No op.""" + + def _compile_python_udf(self, udf_node: ops.ScalarUDF) -> None: + return self._get_udf_source(udf_node) + + def _compile_pyarrow_udf(self, udf_node: ops.ScalarUDF) -> None: + raise NotImplementedError("PyArrow UDFs are not supported in BigQuery") + + def _compile_pandas_udf(self, udf_node: ops.ScalarUDF) -> str: + raise NotImplementedError("Pandas UDFs are not supported in BigQuery") + + def _register_udfs(self, expr: ir.Expr) -> None: + """No op because UDFs made with CREATE TEMPORARY FUNCTION must be followed by a query.""" + + @contextmanager + def _safe_raw_sql(self, *args, **kwargs): + yield self.raw_sql(*args, **kwargs) + def compile(expr, params=None, **kwargs): """Compile an expression for BigQuery.""" diff --git a/ibis/backends/bigquery/client.py b/ibis/backends/bigquery/client.py index eadf83384898..d785e99ddfa7 100644 --- a/ibis/backends/bigquery/client.py +++ b/ibis/backends/bigquery/client.py @@ -24,7 +24,7 @@ def schema_from_bigquery_table(table): partition_field = partition_info.field or NATIVE_PARTITION_COL # Only add a new column if it's not already a column in the schema if partition_field not in schema: - schema |= {partition_field: dt.timestamp} + schema |= {partition_field: dt.Timestamp(timezone="UTC")} return schema diff --git a/ibis/backends/bigquery/compiler.py b/ibis/backends/bigquery/compiler.py index c5e101426f5b..9b0bd63fac3c 100644 --- a/ibis/backends/bigquery/compiler.py +++ b/ibis/backends/bigquery/compiler.py @@ -3,145 +3,825 @@ from __future__ import annotations import re -from functools import partial +from functools import singledispatchmethod import sqlglot as sg -import toolz +import sqlglot.expressions as sge -import ibis.common.graph as lin +import ibis.common.exceptions as com +import ibis.expr.datatypes as dt import ibis.expr.operations as ops -import ibis.expr.types as ir -from ibis.backends.base.sql import compiler as sql_compiler -from ibis.backends.bigquery import operations, registry, rewrites - - -class BigQueryUDFDefinition(sql_compiler.DDL): - """Represents definition of a temporary UDF.""" - - def __init__(self, expr, context): - self.expr = expr - self.context = context - - def compile(self): - """Generate UDF string from definition.""" - op = expr.op() if isinstance(expr := self.expr, ir.Expr) else expr - return op.sql - - -class BigQueryUnion(sql_compiler.Union): - """Union of tables.""" - - @classmethod - def keyword(cls, distinct): - """Use distinct UNION if distinct is True.""" - return "UNION DISTINCT" if distinct else "UNION ALL" - - -class BigQueryIntersection(sql_compiler.Intersection): - """Intersection of tables.""" - - @classmethod - def keyword(cls, distinct): - return "INTERSECT DISTINCT" if distinct else "INTERSECT ALL" - - -class BigQueryDifference(sql_compiler.Difference): - """Difference of tables.""" - - @classmethod - def keyword(cls, distinct): - return "EXCEPT DISTINCT" if distinct else "EXCEPT ALL" - - -def find_bigquery_udf(op): - """Filter which includes only UDFs from expression tree.""" - if type(op) in BigQueryExprTranslator._rewrites: - op = BigQueryExprTranslator._rewrites[type(op)](op) - if isinstance(op, operations.BigQueryUDFNode): - result = op - else: - result = None - return lin.proceed, result - +from ibis import util +from ibis.backends.base.sqlglot.compiler import NULL, STAR, SQLGlotCompiler, paren +from ibis.backends.base.sqlglot.datatypes import BigQueryType, BigQueryUDFType +from ibis.backends.base.sqlglot.rewrites import ( + exclude_unsupported_window_frame_from_ops, + exclude_unsupported_window_frame_from_row_number, + rewrite_first_to_first_value, + rewrite_last_to_last_value, +) +from ibis.common.patterns import replace +from ibis.common.temporal import DateUnit, IntervalUnit, TimestampUnit, TimeUnit +from ibis.expr.rewrites import p, rewrite_sample, y _NAME_REGEX = re.compile(r'[^!"$()*,./;?@[\\\]^`{}~\n]+') -class BigQueryExprTranslator(sql_compiler.ExprTranslator): - """Translate expressions to strings.""" - - _registry = registry.OPERATION_REGISTRY - _rewrites = rewrites.REWRITES - - _forbids_frame_clause = ( - *sql_compiler.ExprTranslator._forbids_frame_clause, - ops.Lag, - ops.Lead, +@replace(p.WindowFunction(p.MinRank | p.DenseRank, y @ p.WindowFrame(start=None))) +def exclude_unsupported_window_frame_from_rank(_, y): + return ops.Subtract( + _.copy(frame=y.copy(start=None, end=0, order_by=y.order_by or (ops.NULL,))), 1 ) - _unsupported_reductions = (ops.ApproxMedian, ops.ApproxCountDistinct) - _dialect_name = "bigquery" - - @staticmethod - def _gen_valid_name(name: str) -> str: - name = "_".join(_NAME_REGEX.findall(name)) or "tmp" - return f"`{name}`" - - def name(self, translated: str, name: str): - # replace invalid characters in automatically generated names - valid_name = self._gen_valid_name(name) - if translated == valid_name: - return translated - return f"{translated} AS {valid_name}" - - @classmethod - def compiles(cls, klass): - def decorator(f): - cls._registry[klass] = f - return f - - return decorator - - def _trans_param(self, op): - if op not in self.context.params: - raise KeyError(op) - return f"@{op.name}" +class BigQueryCompiler(SQLGlotCompiler): + dialect = "bigquery" + type_mapper = BigQueryType + udf_type_mapper = BigQueryUDFType + rewrites = ( + rewrite_sample, + rewrite_first_to_first_value, + rewrite_last_to_last_value, + exclude_unsupported_window_frame_from_ops, + exclude_unsupported_window_frame_from_row_number, + exclude_unsupported_window_frame_from_rank, + *SQLGlotCompiler.rewrites, + ) -compiles = BigQueryExprTranslator.compiles - + NAN = sge.Cast( + this=sge.convert("NaN"), to=sge.DataType(this=sge.DataType.Type.DOUBLE) + ) + POS_INF = sge.Cast( + this=sge.convert("Infinity"), to=sge.DataType(this=sge.DataType.Type.DOUBLE) + ) + NEG_INF = sge.Cast( + this=sge.convert("-Infinity"), to=sge.DataType(this=sge.DataType.Type.DOUBLE) + ) -class BigQueryTableSetFormatter(sql_compiler.TableSetFormatter): - def _quote_identifier(self, name): - return sg.to_identifier(name).sql("bigquery") + def _aggregate(self, funcname: str, *args, where): + func = self.f[funcname] + if where is not None: + args = tuple(self.if_(where, arg, NULL) for arg in args) -class BigQueryCompiler(sql_compiler.Compiler): - translator_class = BigQueryExprTranslator - table_set_formatter_class = BigQueryTableSetFormatter - union_class = BigQueryUnion - intersect_class = BigQueryIntersection - difference_class = BigQueryDifference + return func(*args, dialect=self.dialect) - support_values_syntax_in_select = False - null_limit = None - cheap_in_memory_tables = True + @staticmethod + def _minimize_spec(start, end, spec): + if ( + start is None + and isinstance(getattr(end, "value", None), ops.Literal) + and end.value.value == 0 + and end.following + ): + return None + return spec + + @singledispatchmethod + def visit_node(self, op, **kw): + return super().visit_node(op, **kw) + + @visit_node.register(ops.GeoXMax) + @visit_node.register(ops.GeoXMin) + @visit_node.register(ops.GeoYMax) + @visit_node.register(ops.GeoYMin) + def visit_BoundingBox(self, op, *, arg): + name = type(op).__name__[len("Geo") :].lower() + return sge.Dot( + this=self.f.st_boundingbox(arg), expression=sg.to_identifier(name) + ) + + @visit_node.register(ops.GeoSimplify) + def visit_GeoSimplify(self, op, *, arg, tolerance, preserve_collapsed): + if ( + not isinstance(op.preserve_collapsed, ops.Literal) + or op.preserve_collapsed.value + ): + raise com.UnsupportedOperationError( + "BigQuery simplify does not support preserving collapsed geometries, " + "pass preserve_collapsed=False" + ) + return self.f.st_simplify(arg, tolerance) + + @visit_node.register(ops.ApproxMedian) + def visit_ApproxMedian(self, op, *, arg, where): + return self.agg.approx_quantiles(arg, 2, where=where)[self.f.offset(1)] + + @visit_node.register(ops.Pi) + def visit_Pi(self, op): + return self.f.acos(-1) + + @visit_node.register(ops.E) + def visit_E(self, op): + return self.f.exp(1) + + @visit_node.register(ops.TimeDelta) + def visit_TimeDelta(self, op, *, left, right, part): + return self.f.time_diff(left, right, part, dialect=self.dialect) + + @visit_node.register(ops.DateDelta) + def visit_DateDelta(self, op, *, left, right, part): + return self.f.date_diff(left, right, part, dialect=self.dialect) + + @visit_node.register(ops.TimestampDelta) + def visit_TimestampDelta(self, op, *, left, right, part): + left_tz = op.left.dtype.timezone + right_tz = op.right.dtype.timezone + + if left_tz is None and right_tz is None: + return self.f.datetime_diff(left, right, part) + elif left_tz is not None and right_tz is not None: + return self.f.timestamp_diff(left, right, part) + + raise com.UnsupportedOperationError( + "timestamp difference with mixed timezone/timezoneless values is not implemented" + ) + + @visit_node.register(ops.GroupConcat) + def visit_GroupConcat(self, op, *, arg, sep, where): + if where is not None: + arg = self.if_(where, arg, NULL) + return self.f.string_agg(arg, sep) + + @visit_node.register(ops.FloorDivide) + def visit_FloorDivide(self, op, *, left, right): + return self.cast(self.f.floor(self.f.ieee_divide(left, right)), op.dtype) + + @visit_node.register(ops.Log2) + def visit_Log2(self, op, *, arg): + return self.f.log(arg, 2, dialect=self.dialect) + + @visit_node.register(ops.Log) + def visit_Log(self, op, *, arg, base): + if base is None: + return self.f.ln(arg) + return self.f.log(arg, base, dialect=self.dialect) + + @visit_node.register(ops.ArrayRepeat) + def visit_ArrayRepeat(self, op, *, arg, times): + start = step = 1 + array_length = self.f.array_length(arg) + stop = self.f.greatest(times, 0) * array_length + i = sg.to_identifier("i") + idx = self.f.coalesce( + self.f.nullif(self.f.mod(i, array_length), 0), array_length + ) + series = self.f.generate_array(start, stop, step) + return self.f.array( + sg.select(arg[self.f.safe_ordinal(idx)]).from_(self._unnest(series, as_=i)) + ) + + @visit_node.register(ops.Capitalize) + def visit_Capitalize(self, op, *, arg): + return self.f.concat( + self.f.upper(self.f.substr(arg, 1, 1)), self.f.lower(self.f.substr(arg, 2)) + ) + + @visit_node.register(ops.NthValue) + def visit_NthValue(self, op, *, arg, nth): + if not isinstance(op.nth, ops.Literal): + raise com.UnsupportedOperationError( + f"BigQuery `nth` must be a literal; got {type(op.nth)}" + ) + return self.f.nth_value(arg, nth) + + @visit_node.register(ops.StrRight) + def visit_StrRight(self, op, *, arg, nchars): + return self.f.substr(arg, -self.f.least(self.f.length(arg), nchars)) + + @visit_node.register(ops.StringJoin) + def visit_StringJoin(self, op, *, arg, sep): + return self.f.array_to_string(self.f.array(*arg), sep) + + @visit_node.register(ops.DayOfWeekIndex) + def visit_DayOfWeekIndex(self, op, *, arg): + return self.f.mod(self.f.extract(self.v.dayofweek, arg) + 5, 7) + + @visit_node.register(ops.DayOfWeekName) + def visit_DayOfWeekName(self, op, *, arg): + return self.f.initcap(sge.Cast(this=arg, to="STRING FORMAT 'DAY'")) + + @visit_node.register(ops.StringToTimestamp) + def visit_StringToTimestamp(self, op, *, arg, format_str): + if (timezone := op.dtype.timezone) is not None: + return self.f.parse_timestamp(format_str, arg, timezone) + return self.f.parse_datetime(format_str, arg) + + @visit_node.register(ops.Floor) + def visit_Floor(self, op, *, arg): + return self.cast(self.f.floor(arg), op.dtype) + + @visit_node.register(ops.ArrayCollect) + def visit_ArrayCollect(self, op, *, arg, where): + if where is not None: + arg = self.if_(where, arg, NULL) + return self.f.array_agg(sge.IgnoreNulls(this=arg)) + + def _neg_idx_to_pos(self, arg, idx): + return self.if_(idx < 0, self.f.array_length(arg) + idx, idx) + + @visit_node.register(ops.ArraySlice) + def visit_ArraySlice(self, op, *, arg, start, stop): + index = sg.to_identifier("bq_arr_slice") + cond = [index >= self._neg_idx_to_pos(arg, start)] + + if stop is not None: + cond.append(index < self._neg_idx_to_pos(arg, stop)) + + el = sg.to_identifier("el") + return self.f.array( + sg.select(el).from_(self._unnest(arg, as_=el, offset=index)).where(*cond) + ) + + @visit_node.register(ops.ArrayIndex) + def visit_ArrayIndex(self, op, *, arg, index): + return arg[self.f.safe_offset(index)] + + @visit_node.register(ops.ArrayContains) + def visit_ArrayContains(self, op, *, arg, other): + name = sg.to_identifier(util.gen_name("bq_arr_contains")) + return sge.Exists( + this=sg.select(sge.convert(1)) + .from_(self._unnest(arg, as_=name)) + .where(name.eq(other)) + ) + + @visit_node.register(ops.StringContains) + def visit_StringContains(self, op, *, haystack, needle): + return self.f.strpos(haystack, needle) > 0 + + @visit_node.register(ops.StringFind) + def visti_StringFind(self, op, *, arg, substr, start, end): + if start is not None: + raise NotImplementedError( + "`start` not implemented for BigQuery string find" + ) + if end is not None: + raise NotImplementedError("`end` not implemented for BigQuery string find") + return self.f.strpos(arg, substr) + + def visit_NonNullLiteral(self, op, *, value, dtype): + if dtype.is_string(): + return sge.convert( + str(value) + # Escape \ first so we don't double escape other characters. + .replace("\\", "\\\\") + # ASCII escape sequences that are recognized in Python: + # https://docs.python.org/3/reference/lexical_analysis.html#string-and-bytes-literals + .replace("\a", "\\a") # Bell + .replace("\b", "\\b") # Backspace + .replace("\f", "\\f") # Formfeed + .replace("\n", "\\n") # Newline / Linefeed + .replace("\r", "\\r") # Carriage return + .replace("\t", "\\t") # Tab + .replace("\v", "\\v") # Vertical tab + ) + elif dtype.is_inet() or dtype.is_macaddr(): + return sge.convert(str(value)) + elif dtype.is_timestamp(): + funcname = "datetime" if dtype.timezone is None else "timestamp" + return self.f[funcname](value.isoformat()) + elif dtype.is_date(): + return self.f.datefromparts(value.year, value.month, value.day) + elif dtype.is_time(): + return self.f.time(value.hour, value.minute, value.second) + elif dtype.is_binary(): + return sge.Cast( + this=sge.convert(value.hex()), + to=sge.DataType(this=sge.DataType.Type.BINARY), + format=sge.convert("HEX"), + ) + elif dtype.is_interval(): + if dtype.unit == IntervalUnit.NANOSECOND: + raise com.UnsupportedOperationError( + "BigQuery does not support nanosecond intervals" + ) + elif dtype.is_uuid(): + return sge.convert(str(value)) + return None + + @visit_node.register(ops.IntervalFromInteger) + def visit_IntervalFromInteger(self, op, *, arg, unit): + if unit == IntervalUnit.NANOSECOND: + raise com.UnsupportedOperationError( + "BigQuery does not support nanosecond intervals" + ) + return sge.Interval(this=arg, unit=self.v[unit.singular]) + + @visit_node.register(ops.Strftime) + def visit_Strftime(self, op, *, arg, format_str): + arg_dtype = op.arg.dtype + if arg_dtype.is_timestamp(): + if (timezone := arg_dtype.timezone) is None: + return self.f.format_datetime(format_str, arg) + else: + return self.f.format_timestamp(format_str, arg, timezone) + elif arg_dtype.is_date(): + return self.f.format_date(format_str, arg) + else: + assert arg_dtype.is_time(), arg_dtype + return self.f.format_time(format_str, arg) + + @visit_node.register(ops.IntervalMultiply) + def visit_IntervalMultiply(self, op, *, left, right): + unit = self.v[op.left.dtype.resolution.upper()] + return sge.Interval(this=self.f.extract(unit, left) * right, unit=unit) + + @visit_node.register(ops.TimestampFromUNIX) + def visit_TimestampFromUNIX(self, op, *, arg, unit): + unit = op.unit + if unit == TimestampUnit.SECOND: + return self.f.timestamp_seconds(arg) + elif unit == TimestampUnit.MILLISECOND: + return self.f.timestamp_millis(arg) + elif unit == TimestampUnit.MICROSECOND: + return self.f.timestamp_micros(arg) + elif unit == TimestampUnit.NANOSECOND: + return self.f.timestamp_micros( + self.cast(self.f.round(arg / 1_000), dt.int64) + ) + else: + raise com.UnsupportedOperationError(f"Unit not supported: {unit}") + + @visit_node.register(ops.Cast) + def visit_Cast(self, op, *, arg, to): + from_ = op.arg.dtype + if from_.is_timestamp() and to.is_integer(): + return self.f.unix_micros(arg) + elif from_.is_integer() and to.is_timestamp(): + return self.f.timestamp_seconds(arg) + elif from_.is_interval() and to.is_integer(): + if from_.unit in { + IntervalUnit.WEEK, + IntervalUnit.QUARTER, + IntervalUnit.NANOSECOND, + }: + raise com.UnsupportedOperationError( + f"BigQuery does not allow extracting date part `{from_.unit}` from intervals" + ) + return self.f.extract(self.v[to.resolution.upper()], arg) + elif from_.is_integer() and to.is_interval(): + return sge.Interval(this=arg, unit=self.v[to.unit.singular]) + elif from_.is_floating() and to.is_integer(): + return self.cast(self.f.trunc(arg), dt.int64) + return super().visit_Cast(op, arg=arg, to=to) + + @visit_node.register(ops.JSONGetItem) + def visit_JSONGetItem(self, op, *, arg, index): + return arg[index] + + @visit_node.register(ops.ExtractEpochSeconds) + def visit_ExtractEpochSeconds(self, op, *, arg): + return self.f.unix_seconds(arg) + + @visit_node.register(ops.ExtractWeekOfYear) + def visit_ExtractWeekOfYear(self, op, *, arg): + return self.f.extract(self.v.isoweek, arg) + + @visit_node.register(ops.ExtractYear) + @visit_node.register(ops.ExtractQuarter) + @visit_node.register(ops.ExtractMonth) + @visit_node.register(ops.ExtractDay) + @visit_node.register(ops.ExtractDayOfYear) + @visit_node.register(ops.ExtractHour) + @visit_node.register(ops.ExtractMinute) + @visit_node.register(ops.ExtractSecond) + @visit_node.register(ops.ExtractMicrosecond) + @visit_node.register(ops.ExtractMillisecond) + def visit_ExtractDateField(self, op, *, arg): + name = type(op).__name__[len("Extract") :].upper() + return self.f.extract(self.v[name], arg) + + @visit_node.register(ops.TimestampTruncate) + def visit_Timestamp(self, op, *, arg, unit): + if unit == IntervalUnit.NANOSECOND: + raise com.UnsupportedOperationError( + f"BigQuery does not support truncating {op.arg.dtype} values to unit {unit!r}" + ) + elif unit == IntervalUnit.WEEK: + unit = "WEEK(MONDAY)" + else: + unit = unit.name + return self.f.timestamp_trunc(arg, self.v[unit], dialect=self.dialect) + + @visit_node.register(ops.DateTruncate) + def visit_DateTruncate(self, op, *, arg, unit): + if unit == DateUnit.WEEK: + unit = "WEEK(MONDAY)" + else: + unit = unit.name + return self.f.date_trunc(arg, self.v[unit], dialect=self.dialect) + + @visit_node.register(ops.TimeTruncate) + def visit_TimeTruncate(self, op, *, arg, unit): + if unit == TimeUnit.NANOSECOND: + raise com.UnsupportedOperationError( + f"BigQuery does not support truncating {op.arg.dtype} values to unit {unit!r}" + ) + else: + unit = unit.name + return self.f.time_trunc(arg, self.v[unit], dialect=self.dialect) + + def _nullifzero(self, step, zero, step_dtype): + if step_dtype.is_interval(): + return self.if_(step.eq(zero), NULL, step) + return self.f.nullif(step, zero) + + def _zero(self, dtype): + if dtype.is_interval(): + return self.f.make_interval() + return sge.convert(0) + + def _sign(self, value, dtype): + if dtype.is_interval(): + zero = self._zero(dtype) + return sge.Case( + ifs=[ + self.if_(value < zero, -1), + self.if_(value.eq(zero), 0), + self.if_(value > zero, 1), + ], + default=NULL, + ) + return self.f.sign(value) + + def _make_range(self, func, start, stop, step, step_dtype): + step_sign = self._sign(step, step_dtype) + delta_sign = self._sign(stop - start, step_dtype) + zero = self._zero(step_dtype) + nullifzero = self._nullifzero(step, zero, step_dtype) + condition = sg.and_(sg.not_(nullifzero.is_(NULL)), step_sign.eq(delta_sign)) + gen_array = func(start, stop, step) + name = sg.to_identifier(util.gen_name("bq_arr_range")) + inner = ( + sg.select(name) + .from_(self._unnest(gen_array, as_=name)) + .where(name.neq(stop)) + ) + return self.if_(condition, self.f.array(inner), self.f.array()) + + @visit_node.register(ops.IntegerRange) + def visit_IntegerRange(self, op, *, start, stop, step): + return self._make_range(self.f.generate_array, start, stop, step, op.step.dtype) + + @visit_node.register(ops.TimestampRange) + def visit_TimestampRange(self, op, *, start, stop, step): + if op.start.dtype.timezone is None or op.stop.dtype.timezone is None: + raise com.IbisTypeError( + "Timestamps without timezone values are not supported when generating timestamp ranges" + ) + return self._make_range( + self.f.generate_timestamp_array, start, stop, step, op.step.dtype + ) + + @visit_node.register(ops.First) + def visit_First(self, op, *, arg, where): + if where is not None: + arg = self.if_(where, arg, NULL) + array = self.f.array_agg( + sge.Limit(this=sge.IgnoreNulls(this=arg), expression=sge.convert(1)), + ) + return array[self.f.safe_offset(0)] + + @visit_node.register(ops.Last) + def visit_Last(self, op, *, arg, where): + if where is not None: + arg = self.if_(where, arg, NULL) + array = self.f.array_reverse(self.f.array_agg(sge.IgnoreNulls(this=arg))) + return array[self.f.safe_offset(0)] + + @visit_node.register(ops.Arbitrary) + def _arbitrary(self, op, *, arg, how, where): + if how != "first": + raise com.UnsupportedOperationError( + f"{how!r} value not supported for arbitrary in BigQuery" + ) + + return self.agg.any_value(arg, where=where) + + @visit_node.register(ops.ArrayFilter) + def visit_ArrayFilter(self, op, *, arg, body, param): + return self.f.array( + sg.select(param).from_(self._unnest(arg, as_=param)).where(body) + ) + + @visit_node.register(ops.ArrayMap) + def visit_ArrayMap(self, op, *, arg, body, param): + return self.f.array(sg.select(body).from_(self._unnest(arg, as_=param))) + + @visit_node.register(ops.ArrayZip) + def visit_ArrayZip(self, op, *, arg): + lengths = [self.f.array_length(arr) - 1 for arr in arg] + idx = sg.to_identifier(util.gen_name("bq_arr_idx")) + indices = self._unnest( + self.f.generate_array(0, self.f.greatest(*lengths)), as_=idx + ) + struct_fields = [ + arr[self.f.safe_offset(idx)].as_(name) + for name, arr in zip(op.dtype.value_type.names, arg) + ] + return self.f.array( + sge.Select(kind="STRUCT", expressions=struct_fields).from_(indices) + ) + + @visit_node.register(ops.ArrayPosition) + def visit_ArrayPosition(self, op, *, arg, other): + name = sg.to_identifier(util.gen_name("bq_arr")) + idx = sg.to_identifier(util.gen_name("bq_arr_idx")) + unnest = self._unnest(arg, as_=name, offset=idx) + return self.f.coalesce( + sg.select(idx + 1).from_(unnest).where(name.eq(other)).limit(1).subquery(), + 0, + ) + + def _unnest(self, expression, *, as_, offset=None): + alias = sge.TableAlias(columns=[sg.to_identifier(as_)]) + return sge.Unnest(expressions=[expression], alias=alias, offset=offset) + + @visit_node.register(ops.ArrayRemove) + def visit_ArrayRemove(self, op, *, arg, other): + name = sg.to_identifier(util.gen_name("bq_arr")) + unnest = self._unnest(arg, as_=name) + return self.f.array(sg.select(name).from_(unnest).where(name.neq(other))) + + @visit_node.register(ops.ArrayDistinct) + def visit_ArrayDistinct(self, op, *, arg): + name = util.gen_name("bq_arr") + return self.f.array( + sg.select(name).distinct().from_(self._unnest(arg, as_=name)) + ) + + @visit_node.register(ops.ArraySort) + def visit_ArraySort(self, op, *, arg): + name = util.gen_name("bq_arr") + return self.f.array( + sg.select(name).from_(self._unnest(arg, as_=name)).order_by(name) + ) + + @visit_node.register(ops.ArrayUnion) + def visit_ArrayUnion(self, op, *, left, right): + lname = util.gen_name("bq_arr_left") + rname = util.gen_name("bq_arr_right") + lhs = sg.select(lname).from_(self._unnest(left, as_=lname)) + rhs = sg.select(rname).from_(self._unnest(right, as_=rname)) + return self.f.array(sg.union(lhs, rhs, distinct=True)) + + @visit_node.register(ops.ArrayIntersect) + def visit_ArrayIntersect(self, op, *, left, right): + lname = util.gen_name("bq_arr_left") + rname = util.gen_name("bq_arr_right") + lhs = sg.select(lname).from_(self._unnest(left, as_=lname)) + rhs = sg.select(rname).from_(self._unnest(right, as_=rname)) + return self.f.array(sg.intersect(lhs, rhs, distinct=True)) + + @visit_node.register(ops.Substring) + def visit_Substring(self, op, *, arg, start, length): + if isinstance(op.length, ops.Literal) and (value := op.length.value) < 0: + raise com.IbisInputError( + f"Length parameter must be a non-negative value; got {value}" + ) + suffix = (length,) * (length is not None) + if_pos = self.f.substr(arg, start + 1, *suffix) + if_neg = self.f.substr(arg, self.f.length(arg) + start + 1, *suffix) + return self.if_(start >= 0, if_pos, if_neg) + + @visit_node.register(ops.RegexExtract) + def visit_RegexExtract(self, op, *, arg, pattern, index): + matches = self.f.regexp_contains(arg, pattern) + nonzero_index_replace = self.f.regexp_replace( + arg, + self.f.concat(".*?", pattern, ".*"), + self.f.concat("\\\\", self.cast(index, dt.string)), + ) + zero_index_replace = self.f.regexp_replace( + arg, self.f.concat(".*?", self.f.concat("(", pattern, ")"), ".*"), "\\\\1" + ) + extract = self.if_(index.eq(0), zero_index_replace, nonzero_index_replace) + return self.if_(matches, extract, NULL) + + @visit_node.register(ops.TimestampAdd) + @visit_node.register(ops.TimestampSub) + def visit_TimestampAddSub(self, op, *, left, right): + if not isinstance(right, sge.Interval): + raise com.OperationNotDefinedError( + "BigQuery does not support non-literals on the right side of timestamp add/subtract" + ) + if (unit := op.right.dtype.unit) == IntervalUnit.NANOSECOND: + raise com.UnsupportedOperationError( + f"BigQuery does not allow binary operation {type(op).__name__} with " + f"INTERVAL offset {unit}" + ) + + opname = type(op).__name__[len("Timestamp") :] + funcname = f"TIMESTAMP_{opname.upper()}" + return self.f.anon[funcname](left, right) + + @visit_node.register(ops.DateAdd) + @visit_node.register(ops.DateSub) + def visit_DateAddSub(self, op, *, left, right): + if not isinstance(right, sge.Interval): + raise com.OperationNotDefinedError( + "BigQuery does not support non-literals on the right side of date add/subtract" + ) + if not (unit := op.right.dtype.unit).is_date(): + raise com.UnsupportedOperationError( + f"BigQuery does not allow binary operation {type(op).__name__} with " + f"INTERVAL offset {unit}" + ) + opname = type(op).__name__[len("Date") :] + funcname = f"DATE_{opname.upper()}" + return self.f.anon[funcname](left, right) + + @visit_node.register(ops.Covariance) + def visit_Covariance(self, op, *, left, right, how, where): + if where is not None: + left = self.if_(where, left, NULL) + right = self.if_(where, right, NULL) + + if op.left.dtype.is_boolean(): + left = self.cast(left, dt.int64) + + if op.right.dtype.is_boolean(): + right = self.cast(right, dt.int64) + + how = op.how[:4].upper() + assert how in ("POP", "SAMP"), 'how not in ("POP", "SAMP")' + return self.agg[f"COVAR_{how}"](left, right, where=where) + + @visit_node.register(ops.Correlation) + def visit_Correlation(self, op, *, left, right, how, where): + if how == "sample": + raise ValueError(f"Correlation with how={how!r} is not supported.") + + if where is not None: + left = self.if_(where, left, NULL) + right = self.if_(where, right, NULL) + + if op.left.dtype.is_boolean(): + left = self.cast(left, dt.int64) + + if op.right.dtype.is_boolean(): + right = self.cast(right, dt.int64) + + return self.agg.corr(left, right, where=where) + + @visit_node.register(ops.TypeOf) + def visit_TypeOf(self, op, *, arg): + name = sg.to_identifier(util.gen_name("bq_typeof")) + from_ = self._unnest(self.f.array(self.f.format("%T", arg)), as_=name) + ifs = [ + self.if_( + self.f.regexp_contains(name, '^[A-Z]+ "'), + self.f.regexp_extract(name, '^([A-Z]+) "'), + ), + self.if_(self.f.regexp_contains(name, "^-?[0-9]*$"), "INT64"), + self.if_( + self.f.regexp_contains( + name, r'^(-?[0-9]+[.e].*|CAST\\("([^"]*)" AS FLOAT64\\))$' + ), + "FLOAT64", + ), + self.if_(name.isin(sge.convert("true"), sge.convert("false")), "BOOL"), + self.if_( + sg.or_(self.f.starts_with(name, '"'), self.f.starts_with(name, "'")), + "STRING", + ), + self.if_(self.f.starts_with(name, 'b"'), "BYTES"), + self.if_(self.f.starts_with(name, "["), "ARRAY"), + self.if_(self.f.regexp_contains(name, r"^(STRUCT)?\\("), "STRUCT"), + self.if_(self.f.starts_with(name, "ST_"), "GEOGRAPHY"), + self.if_(name.eq(sge.convert("NULL")), "NULL"), + ] + case = sge.Case(ifs=ifs, default=sge.convert("UNKNOWN")) + return sg.select(case).from_(from_).subquery() + + @visit_node.register(ops.Xor) + def visit_Xor(self, op, *, left, right): + return sg.or_(sg.and_(left, sg.not_(right)), sg.and_(sg.not_(left), right)) + + @visit_node.register(ops.HashBytes) + def visit_HashBytes(self, op, *, arg, how): + if how not in ("md5", "sha1", "sha256", "sha512"): + raise NotImplementedError(how) + return self.f[how](arg) @staticmethod - def _generate_setup_queries(expr, context): - """Generate DDL for temporary resources.""" - nodes = lin.traverse(find_bigquery_udf, expr) - queries = map(partial(BigQueryUDFDefinition, context=context), nodes) + def _gen_valid_name(name: str) -> str: + return "_".join(_NAME_REGEX.findall(name)) or "tmp" + + @visit_node.register(ops.CountStar) + def visit_CountStar(self, op, *, arg, where): + if where is not None: + return self.f.countif(where) + return self.f.count(STAR) + + @visit_node.register(ops.Degrees) + def visit_Degrees(self, op, *, arg): + return paren(180 * arg / self.f.acos(-1)) + + @visit_node.register(ops.Radians) + def visit_Radians(self, op, *, arg): + return paren(self.f.acos(-1) * arg / 180) + + @visit_node.register(ops.CountDistinct) + def visit_CountDistinct(self, op, *, arg, where): + if where is not None: + arg = self.if_(where, arg, NULL) + return self.f.count(sge.Distinct(expressions=[arg])) + + @visit_node.register(ops.CountDistinctStar) + @visit_node.register(ops.DateDiff) + @visit_node.register(ops.ExtractAuthority) + @visit_node.register(ops.ExtractFile) + @visit_node.register(ops.ExtractFragment) + @visit_node.register(ops.ExtractHost) + @visit_node.register(ops.ExtractPath) + @visit_node.register(ops.ExtractProtocol) + @visit_node.register(ops.ExtractQuery) + @visit_node.register(ops.ExtractUserInfo) + @visit_node.register(ops.FindInSet) + @visit_node.register(ops.Median) + @visit_node.register(ops.Quantile) + @visit_node.register(ops.MultiQuantile) + @visit_node.register(ops.RegexSplit) + @visit_node.register(ops.RowID) + @visit_node.register(ops.TimestampBucket) + @visit_node.register(ops.TimestampDiff) + def visit_Undefined(self, op, **_): + raise com.OperationNotDefinedError(type(op).__name__) + + +_SIMPLE_OPS = { + ops.StringAscii: "ascii", + ops.BitAnd: "bit_and", + ops.BitOr: "bit_or", + ops.BitXor: "bit_xor", + ops.DateFromYMD: "date", + ops.Divide: "ieee_divide", + ops.EndsWith: "ends_with", + ops.GeoArea: "st_area", + ops.GeoAsBinary: "st_asbinary", + ops.GeoAsText: "st_astext", + ops.GeoAzimuth: "st_azimuth", + ops.GeoBuffer: "st_buffer", + ops.GeoCentroid: "st_centroid", + ops.GeoContains: "st_contains", + ops.GeoCoveredBy: "st_coveredby", + ops.GeoCovers: "st_covers", + ops.GeoDWithin: "st_dwithin", + ops.GeoDifference: "st_difference", + ops.GeoDisjoint: "st_disjoint", + ops.GeoDistance: "st_distance", + ops.GeoEndPoint: "st_endpoint", + ops.GeoEquals: "st_equals", + ops.GeoGeometryType: "st_geometrytype", + ops.GeoIntersection: "st_intersection", + ops.GeoIntersects: "st_intersects", + ops.GeoLength: "st_length", + ops.GeoMaxDistance: "st_maxdistance", + ops.GeoNPoints: "st_numpoints", + ops.GeoPerimeter: "st_perimeter", + ops.GeoPoint: "st_geogpoint", + ops.GeoPointN: "st_pointn", + ops.GeoStartPoint: "st_startpoint", + ops.GeoTouches: "st_touches", + ops.GeoUnaryUnion: "st_union_agg", + ops.GeoUnion: "st_union", + ops.GeoWithin: "st_within", + ops.GeoX: "st_x", + ops.GeoY: "st_y", + ops.Hash: "farm_fingerprint", + ops.IsInf: "is_inf", + ops.IsNan: "is_nan", + ops.Log10: "log10", + ops.LPad: "lpad", + ops.RPad: "rpad", + ops.Levenshtein: "edit_distance", + ops.Modulus: "mod", + ops.RandomScalar: "rand", + ops.RegexReplace: "regexp_replace", + ops.RegexSearch: "regexp_contains", + ops.Time: "time", + ops.TimeFromHMS: "time", + ops.TimestampFromYMDHMS: "datetime", + ops.TimestampNow: "current_timestamp", +} + + +for _op, _name in _SIMPLE_OPS.items(): + assert isinstance(type(_op), type), type(_op) + if issubclass(_op, ops.Reduction): + + @BigQueryCompiler.visit_node.register(_op) + def _fmt(self, op, *, _name: str = _name, where, **kw): + return self.agg[_name](*kw.values(), where=where) - # UDFs are uniquely identified by the name of the Node subclass we - # generate. - def key(x): - expr = x.expr - op = expr.op() if isinstance(expr, ir.Expr) else expr - return op.__class__.__name__ + else: - return list(toolz.unique(queries, key=key)) + @BigQueryCompiler.visit_node.register(_op) + def _fmt(self, op, *, _name: str = _name, **kw): + return self.f[_name](*kw.values()) + setattr(BigQueryCompiler, f"visit_{_op.__name__}", _fmt) -# Register custom UDFs -import ibis.backends.bigquery.custom_udfs # noqa: F401, E402 +del _op, _name, _fmt diff --git a/ibis/backends/bigquery/converter.py b/ibis/backends/bigquery/converter.py new file mode 100644 index 000000000000..613260188267 --- /dev/null +++ b/ibis/backends/bigquery/converter.py @@ -0,0 +1,20 @@ +from __future__ import annotations + +from ibis.formats.pandas import PandasData + + +class BigQueryPandasData(PandasData): + @classmethod + def convert_GeoSpatial(cls, s, dtype, pandas_type): + import geopandas as gpd + import shapely as shp + + return gpd.GeoSeries(shp.from_wkt(s)) + + convert_Point = ( + convert_LineString + ) = ( + convert_Polygon + ) = ( + convert_MultiLineString + ) = convert_MultiPoint = convert_MultiPolygon = convert_GeoSpatial diff --git a/ibis/backends/bigquery/custom_udfs.py b/ibis/backends/bigquery/custom_udfs.py deleted file mode 100644 index 95b27992ca4d..000000000000 --- a/ibis/backends/bigquery/custom_udfs.py +++ /dev/null @@ -1,41 +0,0 @@ -from __future__ import annotations - -import ibis.expr.datatypes as dt -import ibis.expr.operations as ops -from ibis.backends.bigquery.compiler import BigQueryExprTranslator -from ibis.backends.bigquery.udf import udf - -# Based on: -# https://github.com/GoogleCloudPlatform/bigquery-utils/blob/45e1ac51367ab6209f68e04b1660d5b00258c131/udfs/community/typeof.sqlx#L1 -typeof_ = udf.sql( - name="typeof", - params={"input": "ANY TYPE"}, - output_type=dt.str, - sql_expression=r""" - ( - SELECT - CASE - -- Process NUMERIC, DATE, DATETIME, TIME, TIMESTAMP, - WHEN REGEXP_CONTAINS(literal, r'^[A-Z]+ "') THEN REGEXP_EXTRACT(literal, r'^([A-Z]+) "') - WHEN REGEXP_CONTAINS(literal, r'^-?[0-9]*$') THEN 'INT64' - WHEN - REGEXP_CONTAINS(literal, r'^(-?[0-9]+[.e].*|CAST\("([^"]*)" AS FLOAT64\))$') - THEN - 'FLOAT64' - WHEN literal IN ('true', 'false') THEN 'BOOL' - WHEN literal LIKE '"%' OR literal LIKE "'%" THEN 'STRING' - WHEN literal LIKE 'b"%' THEN 'BYTES' - WHEN literal LIKE '[%' THEN 'ARRAY' - WHEN REGEXP_CONTAINS(literal, r'^(STRUCT)?\(') THEN 'STRUCT' - WHEN literal LIKE 'ST_%' THEN 'GEOGRAPHY' - WHEN literal = 'NULL' THEN 'NULL' - ELSE - 'UNKNOWN' - END - FROM - UNNEST([FORMAT('%T', input)]) AS literal - ) - """, -) - -BigQueryExprTranslator.rewrites(ops.TypeOf)(lambda op: typeof_(op.arg).op()) diff --git a/ibis/backends/bigquery/datatypes.py b/ibis/backends/bigquery/datatypes.py index 130d2500a74c..10aef9e1c943 100644 --- a/ibis/backends/bigquery/datatypes.py +++ b/ibis/backends/bigquery/datatypes.py @@ -1,126 +1,14 @@ from __future__ import annotations import google.cloud.bigquery as bq -import sqlglot.expressions as sge import ibis import ibis.expr.datatypes as dt import ibis.expr.schema as sch -from ibis.backends.base.sqlglot.datatypes import SqlglotType +from ibis.backends.base.sqlglot.datatypes import BigQueryType from ibis.formats import SchemaMapper -class BigQueryType(SqlglotType): - dialect = "bigquery" - - default_decimal_precision = 38 - default_decimal_scale = 9 - - @classmethod - def _from_sqlglot_NUMERIC(cls) -> dt.Decimal: - return dt.Decimal( - cls.default_decimal_precision, - cls.default_decimal_scale, - nullable=cls.default_nullable, - ) - - @classmethod - def _from_sqlglot_BIGNUMERIC(cls) -> dt.Decimal: - return dt.Decimal(76, 38, nullable=cls.default_nullable) - - @classmethod - def _from_sqlglot_DATETIME(cls) -> dt.Decimal: - return dt.Timestamp(timezone=None, nullable=cls.default_nullable) - - @classmethod - def _from_sqlglot_TIMESTAMP(cls) -> dt.Decimal: - return dt.Timestamp(timezone="UTC", nullable=cls.default_nullable) - - @classmethod - def _from_sqlglot_GEOGRAPHY(cls) -> dt.Decimal: - return dt.GeoSpatial( - geotype="geography", srid=4326, nullable=cls.default_nullable - ) - - @classmethod - def _from_sqlglot_TINYINT(cls) -> dt.Int64: - return dt.Int64(nullable=cls.default_nullable) - - _from_sqlglot_UINT = ( - _from_sqlglot_USMALLINT - ) = ( - _from_sqlglot_UTINYINT - ) = _from_sqlglot_INT = _from_sqlglot_SMALLINT = _from_sqlglot_TINYINT - - @classmethod - def _from_sqlglot_UBIGINT(cls) -> dt.Int64: - raise TypeError("Unsigned BIGINT isn't representable in BigQuery INT64") - - @classmethod - def _from_sqlglot_FLOAT(cls) -> dt.Double: - return dt.Float64(nullable=cls.default_nullable) - - @classmethod - def _from_sqlglot_MAP(cls) -> dt.Map: - raise NotImplementedError( - "Cannot convert sqlglot Map type to ibis type: maps are not supported in BigQuery" - ) - - @classmethod - def _from_ibis_Map(cls, dtype: dt.Map) -> sge.DataType: - raise NotImplementedError( - "Cannot convert Ibis Map type to BigQuery type: maps are not supported in BigQuery" - ) - - @classmethod - def _from_ibis_Timestamp(cls, dtype: dt.Timestamp) -> sge.DataType: - if dtype.timezone is None: - return sge.DataType(this=sge.DataType.Type.DATETIME) - elif dtype.timezone == "UTC": - return sge.DataType(this=sge.DataType.Type.TIMESTAMPTZ) - else: - raise TypeError( - "BigQuery does not support timestamps with timezones other than 'UTC'" - ) - - @classmethod - def _from_ibis_Decimal(cls, dtype: dt.Decimal) -> sge.DataType: - precision = dtype.precision - scale = dtype.scale - if (precision, scale) == (76, 38): - return sge.DataType(this=sge.DataType.Type.BIGDECIMAL) - elif (precision, scale) in ((38, 9), (None, None)): - return sge.DataType(this=sge.DataType.Type.DECIMAL) - else: - raise TypeError( - "BigQuery only supports decimal types with precision of 38 and " - f"scale of 9 (NUMERIC) or precision of 76 and scale of 38 (BIGNUMERIC). " - f"Current precision: {dtype.precision}. Current scale: {dtype.scale}" - ) - - @classmethod - def _from_ibis_UInt64(cls, dtype: dt.UInt64) -> sge.DataType: - raise TypeError( - f"Conversion from {dtype} to BigQuery integer type (Int64) is lossy" - ) - - @classmethod - def _from_ibis_UInt32(cls, dtype: dt.UInt32) -> sge.DataType: - return sge.DataType(this=sge.DataType.Type.BIGINT) - - _from_ibis_UInt8 = _from_ibis_UInt16 = _from_ibis_UInt32 - - @classmethod - def _from_ibis_GeoSpatial(cls, dtype: dt.GeoSpatial) -> sge.DataType: - if (dtype.geotype, dtype.srid) == ("geography", 4326): - return sge.DataType(this=sge.DataType.Type.GEOGRAPHY) - else: - raise TypeError( - "BigQuery geography uses points on WGS84 reference ellipsoid." - f"Current geotype: {dtype.geotype}, Current srid: {dtype.srid}" - ) - - class BigQuerySchema(SchemaMapper): @classmethod def from_ibis(cls, schema: sch.Schema) -> list[bq.SchemaField]: diff --git a/ibis/backends/bigquery/operations.py b/ibis/backends/bigquery/operations.py deleted file mode 100644 index b37c8cb1ee1d..000000000000 --- a/ibis/backends/bigquery/operations.py +++ /dev/null @@ -1,9 +0,0 @@ -"""Ibis operations specific to BigQuery.""" - -from __future__ import annotations - -import ibis.expr.operations as ops - - -class BigQueryUDFNode(ops.ValueOp): - """Represents use of a UDF.""" diff --git a/ibis/backends/bigquery/registry.py b/ibis/backends/bigquery/registry.py deleted file mode 100644 index 7afc889d9ac8..000000000000 --- a/ibis/backends/bigquery/registry.py +++ /dev/null @@ -1,1020 +0,0 @@ -"""Module to convert from Ibis expression to SQL string.""" - -from __future__ import annotations - -import contextlib -from typing import TYPE_CHECKING, Literal - -import numpy as np -import sqlglot as sg -from multipledispatch import Dispatcher - -import ibis -import ibis.common.exceptions as com -import ibis.expr.datatypes as dt -import ibis.expr.operations as ops -from ibis import util -from ibis.backends.base.sql.registry import ( - fixed_arity, - helpers, - operation_registry, - reduction, - unary, -) -from ibis.backends.base.sql.registry.literal import _string_literal_format -from ibis.backends.base.sql.registry.main import table_array_view -from ibis.backends.bigquery.datatypes import BigQueryType -from ibis.common.temporal import DateUnit, IntervalUnit, TimeUnit - -if TYPE_CHECKING: - from ibis.backends.base.sql import compiler - - -def _extract_field(sql_attr): - def extract_field_formatter(translator, op): - arg = translator.translate(op.args[0]) - if sql_attr == "epochseconds": - return f"UNIX_SECONDS({arg})" - else: - return f"EXTRACT({sql_attr} from {arg})" - - return extract_field_formatter - - -bigquery_cast = Dispatcher("bigquery_cast") - - -@bigquery_cast.register(str, dt.Timestamp, dt.Integer) -def bigquery_cast_timestamp_to_integer(compiled_arg, from_, to): - """Convert TIMESTAMP to INT64 (seconds since Unix epoch).""" - return f"UNIX_MICROS({compiled_arg})" - - -@bigquery_cast.register(str, dt.Integer, dt.Timestamp) -def bigquery_cast_integer_to_timestamp(compiled_arg, from_, to): - """Convert INT64 (seconds since Unix epoch) to Timestamp.""" - return f"TIMESTAMP_SECONDS({compiled_arg})" - - -@bigquery_cast.register(str, dt.Interval, dt.Integer) -def bigquery_cast_interval_to_integer(compiled_arg, from_, to): - if from_.unit in {IntervalUnit.WEEK, IntervalUnit.QUARTER, IntervalUnit.NANOSECOND}: - raise com.UnsupportedOperationError( - f"BigQuery does not allow extracting date part `{from_.unit}` from intervals" - ) - - return f"EXTRACT({from_.resolution.upper()} from {compiled_arg})" - - -@bigquery_cast.register(str, dt.Floating, dt.Integer) -def bigquery_cast_floating_to_integer(compiled_arg, from_, to): - """Convert FLOAT64 to INT64 without rounding.""" - return f"CAST(TRUNC({compiled_arg}) AS INT64)" - - -@bigquery_cast.register(str, dt.DataType, dt.DataType) -def bigquery_cast_generate(compiled_arg, from_, to): - """Cast to desired type.""" - sql_type = BigQueryType.to_string(to) - return f"CAST({compiled_arg} AS {sql_type})" - - -@bigquery_cast.register(str, dt.DataType) -def bigquery_cast_generate_simple(compiled_arg, to): - return bigquery_cast(compiled_arg, to, to) - - -def _cast(translator, op): - arg, target_type = op.args - arg_formatted = translator.translate(arg) - input_dtype = arg.dtype - return bigquery_cast(arg_formatted, input_dtype, target_type) - - -def integer_to_timestamp(translator: compiler.ExprTranslator, op) -> str: - """Interprets an integer as a timestamp.""" - arg = translator.translate(op.arg) - unit = op.unit.short - - if unit == "s": - return f"TIMESTAMP_SECONDS({arg})" - elif unit == "ms": - return f"TIMESTAMP_MILLIS({arg})" - elif unit == "us": - return f"TIMESTAMP_MICROS({arg})" - elif unit == "ns": - # Timestamps are represented internally as elapsed microseconds, so some - # rounding is required if an integer represents nanoseconds. - # https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types#timestamp_type - return f"TIMESTAMP_MICROS(CAST(ROUND({arg} / 1000) AS INT64))" - - raise NotImplementedError(f"cannot cast unit {op.unit}") - - -def _struct_field(translator, op): - arg = translator.translate(op.arg) - return f"{arg}.`{op.field}`" - - -def _struct_column(translator, op): - cols = ( - f"{translator.translate(value)} AS {name}" - for name, value in zip(op.names, op.values) - ) - return "STRUCT({})".format(", ".join(cols)) - - -def _array_concat(translator, op): - return "ARRAY_CONCAT({})".format(", ".join(map(translator.translate, op.arg))) - - -def _array_column(translator, op): - return "[{}]".format(", ".join(map(translator.translate, op.exprs))) - - -def _array_index(translator, op): - # SAFE_OFFSET returns NULL if out of bounds - arg = translator.translate(op.arg) - index = translator.translate(op.index) - return f"{arg}[SAFE_OFFSET({index})]" - - -def _array_contains(translator, op): - arg = translator.translate(op.arg) - other = translator.translate(op.other) - name = util.gen_name("bq_arr") - return f"(SELECT LOGICAL_OR({name} = {other}) FROM UNNEST({arg}) {name})" - - -def _array_position(translator, op): - arg = translator.translate(op.arg) - other = translator.translate(op.other) - name = util.gen_name("bq_arr") - idx = util.gen_name("bq_arr_idx") - unnest = f"UNNEST({arg}) {name} WITH OFFSET AS {idx}" - return f"COALESCE((SELECT {idx} FROM {unnest} WHERE {name} = {other} LIMIT 1), -1)" - - -def _array_remove(translator, op): - arg = translator.translate(op.arg) - other = translator.translate(op.other) - name = util.gen_name("bq_arr") - return f"ARRAY(SELECT {name} FROM UNNEST({arg}) {name} WHERE {name} <> {other})" - - -def _array_distinct(translator, op): - arg = translator.translate(op.arg) - name = util.gen_name("bq_arr") - return f"ARRAY(SELECT DISTINCT {name} FROM UNNEST({arg}) {name})" - - -def _array_sort(translator, op): - arg = translator.translate(op.arg) - name = util.gen_name("bq_arr") - return f"ARRAY(SELECT {name} FROM UNNEST({arg}) {name} ORDER BY {name})" - - -def _array_union(translator, op): - left = translator.translate(op.left) - right = translator.translate(op.right) - - lname = util.gen_name("bq_arr_left") - rname = util.gen_name("bq_arr_right") - - left_expr = f"SELECT {lname} FROM UNNEST({left}) {lname}" - right_expr = f"SELECT {rname} FROM UNNEST({right}) {rname}" - - return f"ARRAY({left_expr} UNION DISTINCT {right_expr})" - - -def _array_intersect(translator, op): - left = translator.translate(op.left) - right = translator.translate(op.right) - - lname = util.gen_name("bq_arr_left") - rname = util.gen_name("bq_arr_right") - - left_expr = f"SELECT {lname} FROM UNNEST({left}) {lname}" - right_expr = f"SELECT {rname} FROM UNNEST({right}) {rname}" - - return f"ARRAY({left_expr} INTERSECT DISTINCT {right_expr})" - - -def _array_zip(translator, op): - arg = list(map(translator.translate, op.arg)) - lengths = ", ".join(map("ARRAY_LENGTH({}) - 1".format, arg)) - indices = f"UNNEST(GENERATE_ARRAY(0, GREATEST({lengths})))" - idx = util.gen_name("bq_arr_idx") - struct_fields = ", ".join( - f"{arr}[SAFE_OFFSET({idx})] AS {name}" - for name, arr in zip(op.dtype.value_type.names, arg) - ) - return f"ARRAY(SELECT AS STRUCT {struct_fields} FROM {indices} {idx})" - - -def _array_map(translator, op): - arg = translator.translate(op.arg) - result = translator.translate(op.body) - param = op.param - return f"ARRAY(SELECT {result} FROM UNNEST({arg}) {param})" - - -def _array_filter(translator, op): - arg = translator.translate(op.arg) - result = translator.translate(op.body) - param = op.param - return f"ARRAY(SELECT {param} FROM UNNEST({arg}) {param} WHERE {result})" - - -def _hash(translator, op): - arg_formatted = translator.translate(op.arg) - return f"farm_fingerprint({arg_formatted})" - - -def _string_find(translator, op): - haystack, needle, start, end = op.args - - if start is not None: - raise NotImplementedError("start not implemented for string find") - if end is not None: - raise NotImplementedError("end not implemented for string find") - - return "STRPOS({}, {}) - 1".format( - translator.translate(haystack), translator.translate(needle) - ) - - -def _regex_search(translator, op): - arg = translator.translate(op.arg) - regex = translator.translate(op.pattern) - return f"REGEXP_CONTAINS({arg}, {regex})" - - -def _regex_extract(translator, op): - arg = translator.translate(op.arg) - regex = translator.translate(op.pattern) - index = translator.translate(op.index) - matches = f"REGEXP_CONTAINS({arg}, {regex})" - # non-greedily match the regex's prefix so the regex can match as much as possible - nonzero_index_replace = rf"REGEXP_REPLACE({arg}, CONCAT('.*?', {regex}, '.*'), CONCAT('\\', CAST({index} AS STRING)))" - # zero index replacement means capture everything matched by the regex, so - # we wrap the regex in an outer group - zero_index_replace = ( - rf"REGEXP_REPLACE({arg}, CONCAT('.*?', CONCAT('(', {regex}, ')'), '.*'), '\\1')" - ) - extract = f"IF({index} = 0, {zero_index_replace}, {nonzero_index_replace})" - return f"IF({matches}, {extract}, NULL)" - - -def _regex_replace(translator, op): - arg = translator.translate(op.arg) - regex = translator.translate(op.pattern) - replacement = translator.translate(op.replacement) - return f"REGEXP_REPLACE({arg}, {regex}, {replacement})" - - -def _string_concat(translator, op): - args = ", ".join(map(translator.translate, op.arg)) - return f"CONCAT({args})" - - -def _string_join(translator, op): - sep, args = op.args - return "ARRAY_TO_STRING([{}], {})".format( - ", ".join(map(translator.translate, args)), translator.translate(sep) - ) - - -def _string_ascii(translator, op): - arg = translator.translate(op.arg) - return f"TO_CODE_POINTS({arg})[SAFE_OFFSET(0)]" - - -def _string_right(translator, op): - arg, nchars = map(translator.translate, op.args) - return f"SUBSTR({arg}, -LEAST(LENGTH({arg}), {nchars}))" - - -def _string_substring(translator, op): - length = op.length - if (length := getattr(length, "value", None)) is not None and length < 0: - raise ValueError("Length parameter must be a non-negative value.") - - arg = translator.translate(op.arg) - start = translator.translate(op.start) - - arg_length = f"LENGTH({arg})" - if op.length is not None: - suffix = f", {translator.translate(op.length)}" - else: - suffix = "" - - if_pos = f"SUBSTR({arg}, {start} + 1{suffix})" - if_neg = f"SUBSTR({arg}, {arg_length} + {start} + 1{suffix})" - return f"IF({start} >= 0, {if_pos}, {if_neg})" - - -def _log(translator, op): - arg, base = op.args - arg_formatted = translator.translate(arg) - - if base is None: - return f"ln({arg_formatted})" - - base_formatted = translator.translate(base) - return f"log({arg_formatted}, {base_formatted})" - - -def _sg_literal(val) -> str: - return sg.exp.Literal(this=str(val), is_string=isinstance(val, str)).sql( - dialect="bigquery" - ) - - -def _literal(t, op): - dtype = op.dtype - value = op.value - - if value is None: - if not dtype.is_null(): - return f"CAST(NULL AS {BigQueryType.to_string(dtype)})" - return "NULL" - elif dtype.is_boolean(): - return str(value).upper() - elif dtype.is_string() or dtype.is_inet() or dtype.is_macaddr(): - return _string_literal_format(t, op) - elif dtype.is_decimal(): - if value.is_nan(): - return "CAST('NaN' AS FLOAT64)" - elif value.is_infinite(): - prefix = "-" * value.is_signed() - return f"CAST('{prefix}inf' AS FLOAT64)" - else: - return f"{BigQueryType.to_string(dtype)} '{value}'" - elif dtype.is_uuid(): - return _sg_literal(str(value)) - elif dtype.is_numeric(): - if not np.isfinite(value): - return f"CAST({str(value)!r} AS FLOAT64)" - return _sg_literal(value) - elif dtype.is_date(): - with contextlib.suppress(AttributeError): - value = value.date() - return f"DATE {_sg_literal(str(value))}" - elif dtype.is_timestamp(): - typename = "DATETIME" if dtype.timezone is None else "TIMESTAMP" - return f"{typename} {_sg_literal(str(value))}" - elif dtype.is_time(): - # TODO: define extractors on TimeValue expressions - return f"TIME {_sg_literal(str(value))}" - elif dtype.is_binary(): - return repr(value) - elif dtype.is_struct(): - cols = ", ".join( - f"{t.translate(ops.Literal(value[name], dtype=typ))} AS `{name}`" - for name, typ in dtype.items() - ) - return f"STRUCT({cols})" - elif dtype.is_array(): - val_type = dtype.value_type - values = ", ".join( - t.translate(ops.Literal(element, dtype=val_type)) for element in value - ) - return f"[{values}]" - elif dtype.is_interval(): - return f"INTERVAL {value} {dtype.resolution.upper()}" - else: - raise NotImplementedError(f"Unsupported type for BigQuery literal: {dtype}") - - -def _arbitrary(translator, op): - arg, how, where = op.args - - if where is not None: - arg = ops.IfElse(where, arg, ibis.NA) - - if how != "first": - raise com.UnsupportedOperationError( - f"{how!r} value not supported for arbitrary in BigQuery" - ) - - return f"ANY_VALUE({translator.translate(arg)})" - - -def _first(translator, op): - arg = op.arg - where = op.where - - if where is not None: - arg = ops.IfElse(where, arg, ibis.NA) - - arg = translator.translate(arg) - return f"ARRAY_AGG({arg} IGNORE NULLS)[SAFE_OFFSET(0)]" - - -def _last(translator, op): - arg = op.arg - where = op.where - - if where is not None: - arg = ops.IfElse(where, arg, ibis.NA) - - arg = translator.translate(arg) - return f"ARRAY_REVERSE(ARRAY_AGG({arg} IGNORE NULLS))[SAFE_OFFSET(0)]" - - -def _truncate(kind, units): - def truncator(translator, op): - arg, unit = op.args - trans_arg = translator.translate(arg) - if unit not in units: - raise com.UnsupportedOperationError( - f"BigQuery does not support truncating {arg.dtype} values to unit {unit!r}" - ) - if unit.name == "WEEK": - unit = "WEEK(MONDAY)" - else: - unit = unit.name - return f"{kind}_TRUNC({trans_arg}, {unit})" - - return truncator - - -# BigQuery doesn't support nanosecond intervals -_date_truncate = _truncate("DATE", DateUnit) -_time_truncate = _truncate("TIME", set(TimeUnit) - {TimeUnit.NANOSECOND}) -_timestamp_truncate = _truncate( - "TIMESTAMP", set(IntervalUnit) - {IntervalUnit.NANOSECOND} -) - - -def _date_binary(func): - def _formatter(translator, op): - arg, offset = op.left, op.right - - unit = offset.dtype.unit - if not unit.is_date(): - raise com.UnsupportedOperationError( - f"BigQuery does not allow binary operation {func} with INTERVAL offset {unit}" - ) - - formatted_arg = translator.translate(arg) - formatted_offset = translator.translate(offset) - return f"{func}({formatted_arg}, {formatted_offset})" - - return _formatter - - -def _timestamp_binary(func): - def _formatter(translator, op): - arg, offset = op.left, op.right - - unit = offset.dtype.unit - if unit == IntervalUnit.NANOSECOND: - raise com.UnsupportedOperationError( - f"BigQuery does not allow binary operation {func} with INTERVAL offset {unit}" - ) - - if unit.is_date(): - try: - offset = offset.to_expr().to_unit("h").op() - except ValueError: - raise com.UnsupportedOperationError( - f"BigQuery does not allow binary operation {func} with INTERVAL offset {unit}" - ) - - formatted_arg = translator.translate(arg) - formatted_offset = translator.translate(offset) - return f"{func}({formatted_arg}, {formatted_offset})" - - return _formatter - - -def _geo_boundingbox(dimension_name): - def _formatter(translator, op): - geog = op.args[0] - geog_formatted = translator.translate(geog) - return f"ST_BOUNDINGBOX({geog_formatted}).{dimension_name}" - - return _formatter - - -def _geo_simplify(translator, op): - geog, tolerance, preserve_collapsed = op.args - if preserve_collapsed.value: - raise com.UnsupportedOperationError( - "BigQuery simplify does not support preserving collapsed geometries, " - "must pass preserve_collapsed=False" - ) - geog, tolerance = map(translator.translate, (geog, tolerance)) - return f"ST_SIMPLIFY({geog}, {tolerance})" - - -STRFTIME_FORMAT_FUNCTIONS = { - dt.date: "DATE", - dt.time: "TIME", - dt.Timestamp(timezone=None): "DATETIME", - dt.Timestamp(timezone="UTC"): "TIMESTAMP", -} - - -def bigquery_day_of_week_index(t, op): - """Convert timestamp to day-of-week integer.""" - arg = op.args[0] - arg_formatted = t.translate(arg) - return f"MOD(EXTRACT(DAYOFWEEK FROM {arg_formatted}) + 5, 7)" - - -def bigquery_day_of_week_name(t, op): - """Convert timestamp to day-of-week name.""" - return f"INITCAP(CAST({t.translate(op.arg)} AS STRING FORMAT 'DAY'))" - - -def bigquery_compiles_divide(t, op): - """Floating point division.""" - return f"IEEE_DIVIDE({t.translate(op.left)}, {t.translate(op.right)})" - - -def compiles_strftime(translator, op): - """Timestamp formatting.""" - arg = op.arg - format_str = op.format_str - arg_type = arg.dtype - strftime_format_func_name = STRFTIME_FORMAT_FUNCTIONS[arg_type] - fmt_string = translator.translate(format_str) - arg_formatted = translator.translate(arg) - if isinstance(arg_type, dt.Timestamp) and arg_type.timezone is None: - return f"FORMAT_{strftime_format_func_name}({fmt_string}, {arg_formatted})" - elif isinstance(arg_type, dt.Timestamp): - return "FORMAT_{}({}, {}, {!r})".format( - strftime_format_func_name, - fmt_string, - arg_formatted, - arg_type.timezone, - ) - else: - return f"FORMAT_{strftime_format_func_name}({fmt_string}, {arg_formatted})" - - -def compiles_string_to_timestamp(translator, op): - """Timestamp parsing.""" - fmt_string = translator.translate(op.format_str) - arg_formatted = translator.translate(op.arg) - return f"PARSE_TIMESTAMP({fmt_string}, {arg_formatted})" - - -def compiles_floor(t, op): - bigquery_type = BigQueryType.to_string(op.dtype) - arg = op.arg - return f"CAST(FLOOR({t.translate(arg)}) AS {bigquery_type})" - - -def compiles_approx(translator, op): - arg = op.arg - where = op.where - - if where is not None: - arg = ops.IfElse(where, arg, ibis.NA) - - return f"APPROX_QUANTILES({translator.translate(arg)}, 2)[OFFSET(1)]" - - -def compiles_covar_corr(func): - def translate(translator, op): - left = op.left - right = op.right - - if (where := op.where) is not None: - left = ops.IfElse(where, left, None) - right = ops.IfElse(where, right, None) - - left = translator.translate( - ops.Cast(left, dt.int64) if left.dtype.is_boolean() else left - ) - right = translator.translate( - ops.Cast(right, dt.int64) if right.dtype.is_boolean() else right - ) - return f"{func}({left}, {right})" - - return translate - - -def _covar(translator, op): - how = op.how[:4].upper() - assert how in ("POP", "SAMP"), 'how not in ("POP", "SAMP")' - return compiles_covar_corr(f"COVAR_{how}")(translator, op) - - -def _corr(translator, op): - if (how := op.how) == "sample": - raise ValueError(f"Correlation with how={how!r} is not supported.") - return compiles_covar_corr("CORR")(translator, op) - - -def _identical_to(t, op): - left = t.translate(op.left) - right = t.translate(op.right) - return f"{left} IS NOT DISTINCT FROM {right}" - - -def _floor_divide(t, op): - left = t.translate(op.left) - right = t.translate(op.right) - return bigquery_cast(f"FLOOR(IEEE_DIVIDE({left}, {right}))", op.dtype) - - -def _log2(t, op): - return f"LOG({t.translate(op.arg)}, 2)" - - -def _is_nan(t, op): - return f"IS_NAN({t.translate(op.arg)})" - - -def _is_inf(t, op): - return f"IS_INF({t.translate(op.arg)})" - - -def _array_agg(t, op): - arg = op.arg - if (where := op.where) is not None: - arg = ops.IfElse(where, arg, ibis.NA) - return f"ARRAY_AGG({t.translate(arg)} IGNORE NULLS)" - - -def _arg_min_max(sort_dir: Literal["ASC", "DESC"]): - def translate(t, op: ops.ArgMin | ops.ArgMax) -> str: - arg = op.arg - if (where := op.where) is not None: - arg = ops.IfElse(where, arg, None) - arg = t.translate(arg) - key = t.translate(op.key) - return f"ARRAY_AGG({arg} IGNORE NULLS ORDER BY {key} {sort_dir} LIMIT 1)[SAFE_OFFSET(0)]" - - return translate - - -def _array_repeat(t, op): - start = step = 1 - times = t.translate(op.times) - arg = t.translate(op.arg) - array_length = f"ARRAY_LENGTH({arg})" - stop = f"GREATEST({times}, 0) * {array_length}" - idx = f"COALESCE(NULLIF(MOD(i, {array_length}), 0), {array_length})" - series = f"GENERATE_ARRAY({start}, {stop}, {step})" - return f"ARRAY(SELECT {arg}[SAFE_ORDINAL({idx})] FROM UNNEST({series}) AS i)" - - -def _neg_idx_to_pos(array, idx): - return f"IF({idx} < 0, ARRAY_LENGTH({array}) + {idx}, {idx})" - - -def _array_slice(t, op): - arg = t.translate(op.arg) - cond = [f"index >= {_neg_idx_to_pos(arg, t.translate(op.start))}"] - if stop := op.stop: - cond.append(f"index < {_neg_idx_to_pos(arg, t.translate(stop))}") - return ( - f"ARRAY(" - f"SELECT el " - f"FROM UNNEST({arg}) AS el WITH OFFSET index " - f"WHERE {' AND '.join(cond)}" - f")" - ) - - -def _capitalize(t, op): - arg = t.translate(op.arg) - return f"CONCAT(UPPER(SUBSTR({arg}, 1, 1)), LOWER(SUBSTR({arg}, 2)))" - - -def _nth_value(t, op): - arg = t.translate(op.arg) - - if not isinstance(nth_op := op.nth, ops.Literal): - raise TypeError(f"Bigquery nth must be a literal; got {type(op.nth)}") - - return f"NTH_VALUE({arg}, {nth_op.value + 1})" - - -def _interval_multiply(t, op): - if isinstance(op.left, ops.Literal) and isinstance(op.right, ops.Literal): - value = op.left.value * op.right.value - literal = ops.Literal(value, op.left.dtype) - return t.translate(literal) - - left, right = t.translate(op.left), t.translate(op.right) - unit = op.left.dtype.resolution.upper() - return f"INTERVAL EXTRACT({unit} from {left}) * {right} {unit}" - - -def table_column(translator, op): - """Override column references to adjust names for BigQuery.""" - quoted_name = translator._gen_valid_name( - helpers.quote_identifier(op.name, force=True) - ) - - ctx = translator.context - - # If the column does not originate from the table set in the current SELECT - # context, we should format as a subquery - if translator.permit_subquery and ctx.is_foreign_expr(op.table): - # TODO(kszucs): avoid the expression roundtrip - proj_expr = op.table.to_expr().select([op.name]).to_array().op() - return table_array_view(translator, proj_expr) - - alias = ctx.get_ref(op.table, search_parents=True) - if alias is not None: - quoted_name = f"{alias}.{quoted_name}" - - return quoted_name - - -def _count_distinct_star(t, op): - raise com.UnsupportedOperationError( - "BigQuery doesn't support COUNT(DISTINCT ...) with multiple columns" - ) - - -def _time_delta(t, op): - left = t.translate(op.left) - right = t.translate(op.right) - return f"TIME_DIFF({left}, {right}, {op.part.value.upper()})" - - -def _date_delta(t, op): - left = t.translate(op.left) - right = t.translate(op.right) - return f"DATE_DIFF({left}, {right}, {op.part.value.upper()})" - - -def _timestamp_delta(t, op): - left = t.translate(op.left) - right = t.translate(op.right) - left_tz = op.left.dtype.timezone - right_tz = op.right.dtype.timezone - args = f"{left}, {right}, {op.part.value.upper()}" - if left_tz is None and right_tz is None: - return f"DATETIME_DIFF({args})" - elif left_tz is not None and right_tz is not None: - return f"TIMESTAMP_DIFF({args})" - else: - raise NotImplementedError( - "timestamp difference with mixed timezone/timezoneless values is not implemented" - ) - - -def _group_concat(translator, op): - arg = op.arg - where = op.where - - if where is not None: - arg = ops.IfElse(where, arg, ibis.NA) - - arg = translator.translate(arg) - sep = translator.translate(op.sep) - return f"STRING_AGG({arg}, {sep})" - - -def _zero(dtype): - if dtype.is_interval(): - return "MAKE_INTERVAL()" - return "0" - - -def _sign(value, dtype): - if dtype.is_interval(): - zero = _zero(dtype) - return f"""\ -CASE - WHEN {value} < {zero} THEN -1 - WHEN {value} = {zero} THEN 0 - WHEN {value} > {zero} THEN 1 - ELSE NULL -END""" - return f"SIGN({value})" - - -def _nullifzero(step, zero, step_dtype): - if step_dtype.is_interval(): - return f"IF({step} = {zero}, NULL, {step})" - return f"NULLIF({step}, {zero})" - - -def _make_range(func): - def _range(translator, op): - start = translator.translate(op.start) - stop = translator.translate(op.stop) - step = translator.translate(op.step) - - step_dtype = op.step.dtype - step_sign = _sign(step, step_dtype) - delta_sign = _sign(step, step_dtype) - zero = _zero(step_dtype) - nullifzero = _nullifzero(step, zero, step_dtype) - - condition = f"{nullifzero} IS NOT NULL AND {step_sign} = {delta_sign}" - gen_array = f"{func}({start}, {stop}, {step})" - inner = f"SELECT x FROM UNNEST({gen_array}) x WHERE x <> {stop}" - return f"IF({condition}, ARRAY({inner}), [])" - - return _range - - -def _timestamp_range(translator, op): - start = op.start - stop = op.stop - - if start.dtype.timezone is None or stop.dtype.timezone is None: - raise com.IbisTypeError( - "Timestamps without timezone values are not supported when generating timestamp ranges" - ) - - rule = _make_range("GENERATE_TIMESTAMP_ARRAY") - return rule(translator, op) - - -OPERATION_REGISTRY = { - **operation_registry, - # Literal - ops.Literal: _literal, - # Logical - ops.Any: reduction("LOGICAL_OR"), - ops.All: reduction("LOGICAL_AND"), - ops.NullIf: fixed_arity("NULLIF", 2), - # Reductions - ops.ApproxMedian: compiles_approx, - ops.Covariance: _covar, - ops.Correlation: _corr, - # Math - ops.Divide: bigquery_compiles_divide, - ops.Floor: compiles_floor, - ops.Modulus: fixed_arity("MOD", 2), - ops.Sign: unary("SIGN"), - ops.BitwiseNot: lambda t, op: f"~ {t.translate(op.arg)}", - ops.BitwiseXor: lambda t, op: f"{t.translate(op.left)} ^ {t.translate(op.right)}", - ops.BitwiseOr: lambda t, op: f"{t.translate(op.left)} | {t.translate(op.right)}", - ops.BitwiseAnd: lambda t, op: f"{t.translate(op.left)} & {t.translate(op.right)}", - ops.BitwiseLeftShift: lambda t, - op: f"{t.translate(op.left)} << {t.translate(op.right)}", - ops.BitwiseRightShift: lambda t, - op: f"{t.translate(op.left)} >> {t.translate(op.right)}", - # Temporal functions - ops.Date: unary("DATE"), - ops.DateFromYMD: fixed_arity("DATE", 3), - ops.DateAdd: _date_binary("DATE_ADD"), - ops.DateSub: _date_binary("DATE_SUB"), - ops.DateTruncate: _date_truncate, - ops.DayOfWeekIndex: bigquery_day_of_week_index, - ops.DayOfWeekName: bigquery_day_of_week_name, - ops.ExtractEpochSeconds: _extract_field("epochseconds"), - ops.ExtractYear: _extract_field("year"), - ops.ExtractQuarter: _extract_field("quarter"), - ops.ExtractMonth: _extract_field("month"), - ops.ExtractWeekOfYear: _extract_field("isoweek"), - ops.ExtractDay: _extract_field("day"), - ops.ExtractDayOfYear: _extract_field("dayofyear"), - ops.ExtractHour: _extract_field("hour"), - ops.ExtractMinute: _extract_field("minute"), - ops.ExtractSecond: _extract_field("second"), - ops.ExtractMicrosecond: _extract_field("microsecond"), - ops.ExtractMillisecond: _extract_field("millisecond"), - ops.Strftime: compiles_strftime, - ops.StringToTimestamp: compiles_string_to_timestamp, - ops.Time: unary("TIME"), - ops.TimeFromHMS: fixed_arity("TIME", 3), - ops.TimeTruncate: _time_truncate, - ops.TimestampAdd: _timestamp_binary("TIMESTAMP_ADD"), - ops.TimestampFromUNIX: integer_to_timestamp, - ops.TimestampFromYMDHMS: fixed_arity("DATETIME", 6), - ops.TimestampNow: fixed_arity("CURRENT_TIMESTAMP", 0), - ops.TimestampSub: _timestamp_binary("TIMESTAMP_SUB"), - ops.TimestampTruncate: _timestamp_truncate, - ops.IntervalMultiply: _interval_multiply, - ops.Hash: _hash, - ops.StringReplace: fixed_arity("REPLACE", 3), - ops.StringSplit: fixed_arity("SPLIT", 2), - ops.StringConcat: _string_concat, - ops.StringJoin: _string_join, - ops.StringAscii: _string_ascii, - ops.StringFind: _string_find, - ops.Substring: _string_substring, - ops.StrRight: _string_right, - ops.Capitalize: _capitalize, - ops.Translate: fixed_arity("TRANSLATE", 3), - ops.Repeat: fixed_arity("REPEAT", 2), - ops.RegexSearch: _regex_search, - ops.RegexExtract: _regex_extract, - ops.RegexReplace: _regex_replace, - ops.GroupConcat: _group_concat, - ops.Cast: _cast, - ops.StructField: _struct_field, - ops.StructColumn: _struct_column, - ops.ArrayCollect: _array_agg, - ops.ArrayConcat: _array_concat, - ops.Array: _array_column, - ops.ArrayIndex: _array_index, - ops.ArrayLength: unary("ARRAY_LENGTH"), - ops.ArrayRepeat: _array_repeat, - ops.ArraySlice: _array_slice, - ops.ArrayContains: _array_contains, - ops.ArrayPosition: _array_position, - ops.ArrayRemove: _array_remove, - ops.ArrayDistinct: _array_distinct, - ops.ArraySort: _array_sort, - ops.ArrayUnion: _array_union, - ops.ArrayIntersect: _array_intersect, - ops.ArrayZip: _array_zip, - ops.ArrayMap: _array_map, - ops.ArrayFilter: _array_filter, - ops.Log: _log, - ops.Log2: _log2, - ops.Arbitrary: _arbitrary, - ops.First: _first, - ops.Last: _last, - # Geospatial Columnar - ops.GeoUnaryUnion: unary("ST_UNION_AGG"), - # Geospatial - ops.GeoArea: unary("ST_AREA"), - ops.GeoAsBinary: unary("ST_ASBINARY"), - ops.GeoAsText: unary("ST_ASTEXT"), - ops.GeoAzimuth: fixed_arity("ST_AZIMUTH", 2), - ops.GeoBuffer: fixed_arity("ST_BUFFER", 2), - ops.GeoCentroid: unary("ST_CENTROID"), - ops.GeoContains: fixed_arity("ST_CONTAINS", 2), - ops.GeoCovers: fixed_arity("ST_COVERS", 2), - ops.GeoCoveredBy: fixed_arity("ST_COVEREDBY", 2), - ops.GeoDWithin: fixed_arity("ST_DWITHIN", 3), - ops.GeoDifference: fixed_arity("ST_DIFFERENCE", 2), - ops.GeoDisjoint: fixed_arity("ST_DISJOINT", 2), - ops.GeoDistance: fixed_arity("ST_DISTANCE", 2), - ops.GeoEndPoint: unary("ST_ENDPOINT"), - ops.GeoEquals: fixed_arity("ST_EQUALS", 2), - ops.GeoGeometryType: unary("ST_GEOMETRYTYPE"), - ops.GeoIntersection: fixed_arity("ST_INTERSECTION", 2), - ops.GeoIntersects: fixed_arity("ST_INTERSECTS", 2), - ops.GeoLength: unary("ST_LENGTH"), - ops.GeoMaxDistance: fixed_arity("ST_MAXDISTANCE", 2), - ops.GeoNPoints: unary("ST_NUMPOINTS"), - ops.GeoPerimeter: unary("ST_PERIMETER"), - ops.GeoPoint: fixed_arity("ST_GEOGPOINT", 2), - ops.GeoPointN: fixed_arity("ST_POINTN", 2), - ops.GeoSimplify: _geo_simplify, - ops.GeoStartPoint: unary("ST_STARTPOINT"), - ops.GeoTouches: fixed_arity("ST_TOUCHES", 2), - ops.GeoUnion: fixed_arity("ST_UNION", 2), - ops.GeoWithin: fixed_arity("ST_WITHIN", 2), - ops.GeoX: unary("ST_X"), - ops.GeoXMax: _geo_boundingbox("xmax"), - ops.GeoXMin: _geo_boundingbox("xmin"), - ops.GeoY: unary("ST_Y"), - ops.GeoYMax: _geo_boundingbox("ymax"), - ops.GeoYMin: _geo_boundingbox("ymin"), - ops.BitAnd: reduction("BIT_AND"), - ops.BitOr: reduction("BIT_OR"), - ops.BitXor: reduction("BIT_XOR"), - ops.ApproxCountDistinct: reduction("APPROX_COUNT_DISTINCT"), - ops.ApproxMedian: compiles_approx, - ops.IdenticalTo: _identical_to, - ops.FloorDivide: _floor_divide, - ops.IsNan: _is_nan, - ops.IsInf: _is_inf, - ops.ArgMin: _arg_min_max("ASC"), - ops.ArgMax: _arg_min_max("DESC"), - ops.Pi: lambda *_: "ACOS(-1)", - ops.E: lambda *_: "EXP(1)", - ops.RandomScalar: fixed_arity("RAND", 0), - ops.NthValue: _nth_value, - ops.JSONGetItem: lambda t, op: f"{t.translate(op.arg)}[{t.translate(op.index)}]", - ops.ArrayStringJoin: lambda t, - op: f"ARRAY_TO_STRING({t.translate(op.arg)}, {t.translate(op.sep)})", - ops.StartsWith: fixed_arity("STARTS_WITH", 2), - ops.EndsWith: fixed_arity("ENDS_WITH", 2), - ops.TableColumn: table_column, - ops.CountDistinctStar: _count_distinct_star, - ops.Argument: lambda _, op: op.param, - ops.Unnest: unary("UNNEST"), - ops.TimeDelta: _time_delta, - ops.DateDelta: _date_delta, - ops.TimestampDelta: _timestamp_delta, - ops.IntegerRange: _make_range("GENERATE_ARRAY"), - ops.TimestampRange: _timestamp_range, -} - -_invalid_operations = { - ops.FindInSet, - ops.DateDiff, - ops.TimestampDiff, - ops.ExtractAuthority, - ops.ExtractFile, - ops.ExtractFragment, - ops.ExtractHost, - ops.ExtractPath, - ops.ExtractProtocol, - ops.ExtractQuery, - ops.ExtractUserInfo, -} - -OPERATION_REGISTRY = { - k: v for k, v in OPERATION_REGISTRY.items() if k not in _invalid_operations -} diff --git a/ibis/backends/bigquery/rewrites.py b/ibis/backends/bigquery/rewrites.py deleted file mode 100644 index 2fe34ccc37bf..000000000000 --- a/ibis/backends/bigquery/rewrites.py +++ /dev/null @@ -1,32 +0,0 @@ -"""Methods to translate BigQuery expressions before compilation.""" - -from __future__ import annotations - -import toolz - -import ibis.expr.datatypes as dt -import ibis.expr.operations as ops -from ibis.backends.base.sql import compiler as sql_compiler - - -def bq_sum(op): - if isinstance((arg := op.arg).dtype, dt.Boolean): - return ops.Sum(ops.Cast(arg, dt.int64), where=op.where) - else: - return op - - -def bq_mean(op): - if isinstance((arg := op.arg).dtype, dt.Boolean): - return ops.Mean(ops.Cast(arg, dt.int64), where=op.where) - else: - return op - - -REWRITES = { - **sql_compiler.ExprTranslator._rewrites, - ops.Sum: bq_sum, - ops.Mean: bq_mean, - ops.Any: toolz.identity, - ops.All: toolz.identity, -} diff --git a/ibis/backends/bigquery/tests/system/snapshots/test_client/test_cross_project_query/out.sql b/ibis/backends/bigquery/tests/system/snapshots/test_client/test_cross_project_query/out.sql index 225b5770f6fe..da12e567f6b5 100644 --- a/ibis/backends/bigquery/tests/system/snapshots/test_client/test_cross_project_query/out.sql +++ b/ibis/backends/bigquery/tests/system/snapshots/test_client/test_cross_project_query/out.sql @@ -1,10 +1,6 @@ SELECT - t0.`title`, - t0.`tags` -FROM ( - SELECT - t1.* - FROM `bigquery-public-data`.stackoverflow.posts_questions AS t1 - WHERE - STRPOS(t1.`tags`, 'ibis') - 1 >= 0 -) AS t0 \ No newline at end of file + t0.title, + t0.tags +FROM `bigquery-public-data`.stackoverflow.posts_questions AS t0 +WHERE + strpos(t0.tags, 'ibis') > 0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/system/snapshots/test_client/test_multiple_project_queries/out.sql b/ibis/backends/bigquery/tests/system/snapshots/test_client/test_multiple_project_queries/out.sql index f9d06ecd8b53..9832d461eb58 100644 --- a/ibis/backends/bigquery/tests/system/snapshots/test_client/test_multiple_project_queries/out.sql +++ b/ibis/backends/bigquery/tests/system/snapshots/test_client/test_multiple_project_queries/out.sql @@ -1,5 +1,5 @@ SELECT - t0.`title` -FROM `bigquery-public-data`.stackoverflow.posts_questions AS t0 -INNER JOIN `nyc-tlc`.yellow.trips AS t1 - ON t0.`tags` = t1.`rate_code` \ No newline at end of file + t2.title +FROM `bigquery-public-data`.stackoverflow.posts_questions AS t2 +INNER JOIN `nyc-tlc`.yellow.trips AS t3 + ON t2.tags = t3.rate_code \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/system/snapshots/test_client/test_subquery_scalar_params/out.sql b/ibis/backends/bigquery/tests/system/snapshots/test_client/test_subquery_scalar_params/out.sql index cc99fcc1a0d3..ec28287c9f13 100644 --- a/ibis/backends/bigquery/tests/system/snapshots/test_client/test_subquery_scalar_params/out.sql +++ b/ibis/backends/bigquery/tests/system/snapshots/test_client/test_subquery_scalar_params/out.sql @@ -1,20 +1,19 @@ -WITH t0 AS ( - SELECT - t2.`float_col`, - t2.`timestamp_col`, - t2.`int_col`, - t2.`string_col` - FROM `ibis-gbq`.ibis_gbq_testing.functional_alltypes AS t2 - WHERE - t2.`timestamp_col` < @param_0 -) SELECT - count(t1.`foo`) AS `count` + COUNT(t2.foo) AS count FROM ( SELECT - t0.`string_col`, - sum(t0.`float_col`) AS `foo` - FROM t0 + t1.string_col, + SUM(t1.float_col) AS foo + FROM ( + SELECT + t0.float_col, + t0.timestamp_col, + t0.int_col, + t0.string_col + FROM `ibis-gbq`.ibis_gbq_testing.functional_alltypes AS t0 + WHERE + t0.timestamp_col < datetime('2014-01-01T00:00:00') + ) AS t1 GROUP BY 1 -) AS t1 \ No newline at end of file +) AS t2 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/system/test_client.py b/ibis/backends/bigquery/tests/system/test_client.py index ac7eedee9094..cbb4a3c3c0aa 100644 --- a/ibis/backends/bigquery/tests/system/test_client.py +++ b/ibis/backends/bigquery/tests/system/test_client.py @@ -190,13 +190,13 @@ def test_raw_sql(con): def test_parted_column_rename(parted_alltypes): assert "PARTITIONTIME" in parted_alltypes.columns - assert "_PARTITIONTIME" in parted_alltypes.op().table.schema.names + assert "_PARTITIONTIME" in parted_alltypes.op().parent.schema.names def test_scalar_param_partition_time(parted_alltypes): assert "PARTITIONTIME" in parted_alltypes.columns assert "PARTITIONTIME" in parted_alltypes.schema() - param = ibis.param("timestamp").name("time_param") + param = ibis.param("timestamp('UTC')") expr = parted_alltypes[param > parted_alltypes.PARTITIONTIME] df = expr.execute(params={param: "2017-01-01"}) assert df.empty diff --git a/ibis/backends/bigquery/tests/system/udf/test_udf_execute.py b/ibis/backends/bigquery/tests/system/udf/test_udf_execute.py index 3020a02b58c4..66226ae70e74 100644 --- a/ibis/backends/bigquery/tests/system/udf/test_udf_execute.py +++ b/ibis/backends/bigquery/tests/system/udf/test_udf_execute.py @@ -9,7 +9,7 @@ import ibis import ibis.expr.datatypes as dt -from ibis.backends.bigquery import udf +from ibis import udf PROJECT_ID = os.environ.get("GOOGLE_BIGQUERY_PROJECT_ID", "ibis-gbq") DATASET_ID = "testing" @@ -28,12 +28,8 @@ def df(alltypes): def test_udf(alltypes, df): - @udf( - input_type=[dt.double, dt.double], - output_type=dt.double, - determinism=True, - ) - def my_add(a, b): + @udf.scalar.python(determinism=True) + def my_add(a: float, b: float) -> float: return a + b expr = my_add(alltypes.double_col, alltypes.double_col) @@ -49,13 +45,10 @@ def my_add(a, b): def test_udf_with_struct(alltypes, df, snapshot): - @udf( - input_type=[dt.double, dt.double], - output_type=dt.Struct.from_tuples( - [("width", dt.double), ("height", dt.double)] - ), - ) - def my_struct_thing(a, b): + @udf.scalar.python + def my_struct_thing(a: float, b: float) -> dt.Struct( + {"width": float, "height": float} + ): class Rectangle: def __init__(self, width, height): self.width = width @@ -63,9 +56,6 @@ def __init__(self, width, height): return Rectangle(a, b) - result = my_struct_thing.sql - snapshot.assert_match(result, "out.sql") - expr = my_struct_thing(alltypes.double_col, alltypes.double_col) result = expr.execute() assert not result.empty @@ -75,12 +65,12 @@ def __init__(self, width, height): def test_udf_compose(alltypes, df): - @udf([dt.double], dt.double) - def add_one(x): + @udf.scalar.python + def add_one(x: float) -> float: return x + 1.0 - @udf([dt.double], dt.double) - def times_two(x): + @udf.scalar.python + def times_two(x: float) -> float: return x * 2.0 t = alltypes @@ -91,8 +81,8 @@ def times_two(x): def test_udf_scalar(con): - @udf([dt.double, dt.double], dt.double) - def my_add(x, y): + @udf.scalar.python + def my_add(x: float, y: float) -> float: return x + y expr = my_add(1, 2) @@ -101,29 +91,23 @@ def my_add(x, y): def test_multiple_calls_has_one_definition(con): - @udf([dt.string], dt.double) - def my_str_len(s): + @udf.scalar.python + def my_str_len(s: str) -> float: return s.length s = ibis.literal("abcd") expr = my_str_len(s) + my_str_len(s) - add = expr.op() - - # generated javascript is identical - assert add.left.sql == add.right.sql assert con.execute(expr) == 8.0 def test_udf_libraries(con): - @udf( - [dt.Array(dt.string)], - dt.double, + @udf.scalar.python( # whatever symbols are exported in the library are visible inside the # UDF, in this case lodash defines _ and we use that here - libraries=["gs://ibis-testing-libraries/lodash.min.js"], + libraries=("gs://ibis-testing-libraries/lodash.min.js",), ) - def string_length(strings): + def string_length(strings: list[str]) -> float: return _.sum(_.map(strings, lambda x: x.length)) # noqa: F821 raw_data = ["aaa", "bb", "c"] @@ -135,45 +119,18 @@ def string_length(strings): def test_udf_with_len(con): - @udf([dt.string], dt.double) - def my_str_len(x): + @udf.scalar.python + def my_str_len(x: str) -> float: return len(x) - @udf([dt.Array(dt.string)], dt.double) - def my_array_len(x): + @udf.scalar.python + def my_array_len(x: list[str]) -> float: return len(x) assert con.execute(my_str_len("aaa")) == 3 assert con.execute(my_array_len(["aaa", "bb"])) == 2 -@pytest.mark.parametrize( - ("argument_type",), - [ - param( - dt.string, - id="string", - ), - param( - "ANY TYPE", - id="string", - ), - ], -) -def test_udf_sql(con, argument_type): - format_t = udf.sql( - "format_t", - params={"input": argument_type}, - output_type=dt.string, - sql_expression="FORMAT('%T', input)", - ) - - s = ibis.literal("abcd") - expr = format_t(s) - - con.execute(expr) - - @pytest.mark.parametrize( ("value", "expected"), [ diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/filter-approx_median/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/filter-approx_median/out.sql index 7000a628bb25..e86bffa88a0b 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/filter-approx_median/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/filter-approx_median/out.sql @@ -1,3 +1,3 @@ SELECT - APPROX_QUANTILES(IF(t0.`month` > 0, t0.`double_col`, NULL), 2)[OFFSET(1)] AS `ApproxMedian_double_col_ Greater_month_ 0` + approx_quantiles(IF(t0.month > 0, t0.double_col, NULL), IF(t0.month > 0, 2, NULL))[offset(1)] AS `ApproxMedian_double_col_ Greater_month_ 0` FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/filter-approx_nunique/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/filter-approx_nunique/out.sql index 4e0b6bfd607c..967e47b3904f 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/filter-approx_nunique/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/filter-approx_nunique/out.sql @@ -1,3 +1,3 @@ SELECT - APPROX_COUNT_DISTINCT(IF(t0.`month` > 0, t0.`double_col`, NULL)) AS `ApproxCountDistinct_double_col_ Greater_month_ 0` + APPROX_COUNT_DISTINCT(IF(t0.month > 0, t0.double_col, NULL)) AS `ApproxCountDistinct_double_col_ Greater_month_ 0` FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/no_filter-approx_median/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/no_filter-approx_median/out.sql index 06569c747fe0..8a4438fd4e83 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/no_filter-approx_median/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/no_filter-approx_median/out.sql @@ -1,3 +1,3 @@ SELECT - APPROX_QUANTILES(t0.`double_col`, 2)[OFFSET(1)] AS `ApproxMedian_double_col` + approx_quantiles(t0.double_col, 2)[offset(1)] AS ApproxMedian_double_col FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/no_filter-approx_nunique/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/no_filter-approx_nunique/out.sql index 69b0211b7995..7ee240de9fd9 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/no_filter-approx_nunique/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_approx/no_filter-approx_nunique/out.sql @@ -1,3 +1,3 @@ SELECT - APPROX_COUNT_DISTINCT(t0.`double_col`) AS `ApproxCountDistinct_double_col` + APPROX_COUNT_DISTINCT(t0.double_col) AS ApproxCountDistinct_double_col FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_binary/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_binary/out.sql index 29bb2f587d13..c9b5a33bdd2a 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_binary/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_binary/out.sql @@ -1,3 +1,3 @@ SELECT - CAST(t0.`value` AS BYTES) AS `Cast_value_ binary` + CAST(t0.value AS BYTES) AS `Cast_value_ binary` FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/filter-bit_and/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/filter-bit_and/out.sql index a3eda8e81031..9086990603db 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/filter-bit_and/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/filter-bit_and/out.sql @@ -1,3 +1,3 @@ SELECT - BIT_AND(IF(t0.`bigint_col` > 0, t0.`int_col`, NULL)) AS `BitAnd_int_col_ Greater_bigint_col_ 0` + bit_and(IF(t0.bigint_col > 0, t0.int_col, NULL)) AS `BitAnd_int_col_ Greater_bigint_col_ 0` FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/filter-bit_or/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/filter-bit_or/out.sql index 335de82862ea..ec9ed6c6b66b 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/filter-bit_or/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/filter-bit_or/out.sql @@ -1,3 +1,3 @@ SELECT - BIT_OR(IF(t0.`bigint_col` > 0, t0.`int_col`, NULL)) AS `BitOr_int_col_ Greater_bigint_col_ 0` + bit_or(IF(t0.bigint_col > 0, t0.int_col, NULL)) AS `BitOr_int_col_ Greater_bigint_col_ 0` FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/filter-bit_xor/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/filter-bit_xor/out.sql index 2905a7ad8912..7997e495ef8f 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/filter-bit_xor/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/filter-bit_xor/out.sql @@ -1,3 +1,3 @@ SELECT - BIT_XOR(IF(t0.`bigint_col` > 0, t0.`int_col`, NULL)) AS `BitXor_int_col_ Greater_bigint_col_ 0` + bit_xor(IF(t0.bigint_col > 0, t0.int_col, NULL)) AS `BitXor_int_col_ Greater_bigint_col_ 0` FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/no_filter-bit_and/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/no_filter-bit_and/out.sql index 622f27b2071d..fcafb18759da 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/no_filter-bit_and/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/no_filter-bit_and/out.sql @@ -1,3 +1,3 @@ SELECT - BIT_AND(t0.`int_col`) AS `BitAnd_int_col` + bit_and(t0.int_col) AS BitAnd_int_col FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/no_filter-bit_or/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/no_filter-bit_or/out.sql index 6fed6bb96de3..22f8eb5534f9 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/no_filter-bit_or/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/no_filter-bit_or/out.sql @@ -1,3 +1,3 @@ SELECT - BIT_OR(t0.`int_col`) AS `BitOr_int_col` + bit_or(t0.int_col) AS BitOr_int_col FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/no_filter-bit_xor/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/no_filter-bit_xor/out.sql index 91808632fe0c..de39538dbbd7 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/no_filter-bit_xor/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bit/no_filter-bit_xor/out.sql @@ -1,3 +1,3 @@ SELECT - BIT_XOR(t0.`int_col`) AS `BitXor_int_col` + bit_xor(t0.int_col) AS BitXor_int_col FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers/mean/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers/mean/out.sql index 41815bb5d337..1345addaed7e 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers/mean/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers/mean/out.sql @@ -1,3 +1,3 @@ SELECT - avg(CAST(t0.`bool_col` AS INT64)) AS `Mean_bool_col` + AVG(CAST(t0.bool_col AS INT64)) AS Mean_bool_col FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers/sum/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers/sum/out.sql index 731b35d720f3..52a1683495d8 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers/sum/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers/sum/out.sql @@ -1,3 +1,3 @@ SELECT - sum(CAST(t0.`bool_col` AS INT64)) AS `Sum_bool_col` + SUM(CAST(t0.bool_col AS INT64)) AS Sum_bool_col FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers_where_conj/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers_where_conj/out.sql index 5361ef28236f..686b89ebd74f 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers_where_conj/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers_where_conj/out.sql @@ -1,9 +1,7 @@ SELECT - sum( - IF(( - t0.`month` > 6 - ) AND ( - t0.`month` < 10 - ), CAST(t0.`bool_col` AS INT64), NULL) - ) AS `Sum_bool_col_ And_Greater_month_ 6_ Less_month_ 10` + SUM(IF(( + t0.month > 6 + ) AND ( + t0.month < 10 + ), CAST(t0.bool_col AS INT64), NULL)) AS `Sum_bool_col_ And_Greater_month_ 6_ Less_month_ 10` FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers_where_simple/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers_where_simple/out.sql index 5f8d980b673f..3e3b21ef17e2 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers_where_simple/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bool_reducers_where_simple/out.sql @@ -1,3 +1,3 @@ SELECT - avg(IF(t0.`month` > 6, CAST(t0.`bool_col` AS INT64), NULL)) AS `Mean_bool_col_ Greater_month_ 6` + AVG(IF(t0.month > 6, CAST(t0.bool_col AS INT64), NULL)) AS `Mean_bool_col_ Greater_month_ 6` FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bucket/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bucket/out.sql index 4c4e2a32e2ea..7aa64367725a 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bucket/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_bucket/out.sql @@ -1,17 +1,17 @@ SELECT CASE WHEN ( - 0 <= t0.`value` + 0 <= t0.value ) AND ( - t0.`value` < 1 + t0.value < 1 ) THEN 0 WHEN ( - 1 <= t0.`value` + 1 <= t0.value ) AND ( - t0.`value` <= 3 + t0.value <= 3 ) THEN 1 ELSE CAST(NULL AS INT64) - END AS `tmp` + END AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_cast_float_to_int/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_cast_float_to_int/out.sql index acc0555d337d..d81e2d9cbdcf 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_cast_float_to_int/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_cast_float_to_int/out.sql @@ -1,3 +1,3 @@ SELECT - CAST(TRUNC(t0.`double_col`) AS INT64) AS `Cast_double_col_ int64` + CAST(trunc(t0.double_col) AS INT64) AS `Cast_double_col_ int64` FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_compile_toplevel/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_compile_toplevel/out.sql index 119f5daa423e..1b1cef1e6341 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_compile_toplevel/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_compile_toplevel/out.sql @@ -1,3 +1,3 @@ SELECT - sum(t0.`foo`) AS `Sum_foo` -FROM t0 \ No newline at end of file + SUM(t0.foo) AS Sum_foo +FROM t0 AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_cov/pop/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_cov/pop/out.sql index 6f2e6ac5f580..16c39cbe3843 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_cov/pop/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_cov/pop/out.sql @@ -1,3 +1,3 @@ SELECT - COVAR_POP(t0.`double_col`, t0.`double_col`) AS `Covariance_double_col_ double_col` + COVAR_POP(t0.double_col, t0.double_col) AS `Covariance_double_col_ double_col` FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_cov/sample/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_cov/sample/out.sql index 7d097f565cdd..cd9d190ae5e5 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_cov/sample/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_cov/sample/out.sql @@ -1,3 +1,3 @@ SELECT - COVAR_SAMP(t0.`double_col`, t0.`double_col`) AS `Covariance_double_col_ double_col` + COVAR_SAMP(t0.double_col, t0.double_col) AS `Covariance_double_col_ double_col` FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/date/index.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/date/index.sql index fe81b1746772..f95467b3223b 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/date/index.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/date/index.sql @@ -1,2 +1,2 @@ SELECT - MOD(EXTRACT(DAYOFWEEK FROM CAST('2017-01-01' AS DATE)) + 5, 7) AS `DayOfWeekIndex_datetime_date_2017_ 1_ 1` \ No newline at end of file + mod(EXTRACT(dayofweek FROM DATE(2017, 1, 1)) + 5, 7) AS `DayOfWeekIndex_datetime_date_2017_ 1_ 1` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/date/name.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/date/name.sql index da4b415da6de..f207aa60fec7 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/date/name.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/date/name.sql @@ -1,2 +1,2 @@ SELECT - INITCAP(CAST(CAST('2017-01-01' AS DATE) AS STRING FORMAT 'DAY')) AS `DayOfWeekName_datetime_date_2017_ 1_ 1` \ No newline at end of file + INITCAP(CAST(DATE(2017, 1, 1) AS STRING FORMAT 'DAY')) AS `DayOfWeekName_datetime_date_2017_ 1_ 1` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/datetime/index.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/datetime/index.sql index 2304d7a2866b..7d9d59767bc6 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/datetime/index.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/datetime/index.sql @@ -1,2 +1,2 @@ SELECT - MOD(EXTRACT(DAYOFWEEK FROM CAST('2017-01-01 04:55:59' AS DATETIME)) + 5, 7) AS `DayOfWeekIndex_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file + mod(EXTRACT(dayofweek FROM datetime('2017-01-01T04:55:59')) + 5, 7) AS `DayOfWeekIndex_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/datetime/name.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/datetime/name.sql index 79ebfd44155a..fed76c0c815d 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/datetime/name.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/datetime/name.sql @@ -1,2 +1,2 @@ SELECT - INITCAP(CAST(CAST('2017-01-01 04:55:59' AS DATETIME) AS STRING FORMAT 'DAY')) AS `DayOfWeekName_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file + INITCAP(CAST(datetime('2017-01-01T04:55:59') AS STRING FORMAT 'DAY')) AS `DayOfWeekName_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_date/index.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_date/index.sql index fe81b1746772..f95467b3223b 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_date/index.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_date/index.sql @@ -1,2 +1,2 @@ SELECT - MOD(EXTRACT(DAYOFWEEK FROM CAST('2017-01-01' AS DATE)) + 5, 7) AS `DayOfWeekIndex_datetime_date_2017_ 1_ 1` \ No newline at end of file + mod(EXTRACT(dayofweek FROM DATE(2017, 1, 1)) + 5, 7) AS `DayOfWeekIndex_datetime_date_2017_ 1_ 1` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_date/name.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_date/name.sql index da4b415da6de..f207aa60fec7 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_date/name.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_date/name.sql @@ -1,2 +1,2 @@ SELECT - INITCAP(CAST(CAST('2017-01-01' AS DATE) AS STRING FORMAT 'DAY')) AS `DayOfWeekName_datetime_date_2017_ 1_ 1` \ No newline at end of file + INITCAP(CAST(DATE(2017, 1, 1) AS STRING FORMAT 'DAY')) AS `DayOfWeekName_datetime_date_2017_ 1_ 1` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_timestamp/index.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_timestamp/index.sql index 2304d7a2866b..7d9d59767bc6 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_timestamp/index.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_timestamp/index.sql @@ -1,2 +1,2 @@ SELECT - MOD(EXTRACT(DAYOFWEEK FROM CAST('2017-01-01 04:55:59' AS DATETIME)) + 5, 7) AS `DayOfWeekIndex_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file + mod(EXTRACT(dayofweek FROM datetime('2017-01-01T04:55:59')) + 5, 7) AS `DayOfWeekIndex_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_timestamp/name.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_timestamp/name.sql index 79ebfd44155a..fed76c0c815d 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_timestamp/name.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/string_timestamp/name.sql @@ -1,2 +1,2 @@ SELECT - INITCAP(CAST(CAST('2017-01-01 04:55:59' AS DATETIME) AS STRING FORMAT 'DAY')) AS `DayOfWeekName_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file + INITCAP(CAST(datetime('2017-01-01T04:55:59') AS STRING FORMAT 'DAY')) AS `DayOfWeekName_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp/index.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp/index.sql index 2304d7a2866b..7d9d59767bc6 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp/index.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp/index.sql @@ -1,2 +1,2 @@ SELECT - MOD(EXTRACT(DAYOFWEEK FROM CAST('2017-01-01 04:55:59' AS DATETIME)) + 5, 7) AS `DayOfWeekIndex_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file + mod(EXTRACT(dayofweek FROM datetime('2017-01-01T04:55:59')) + 5, 7) AS `DayOfWeekIndex_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp/name.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp/name.sql index 79ebfd44155a..fed76c0c815d 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp/name.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp/name.sql @@ -1,2 +1,2 @@ SELECT - INITCAP(CAST(CAST('2017-01-01 04:55:59' AS DATETIME) AS STRING FORMAT 'DAY')) AS `DayOfWeekName_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file + INITCAP(CAST(datetime('2017-01-01T04:55:59') AS STRING FORMAT 'DAY')) AS `DayOfWeekName_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp_date/index.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp_date/index.sql index fe81b1746772..f95467b3223b 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp_date/index.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp_date/index.sql @@ -1,2 +1,2 @@ SELECT - MOD(EXTRACT(DAYOFWEEK FROM CAST('2017-01-01' AS DATE)) + 5, 7) AS `DayOfWeekIndex_datetime_date_2017_ 1_ 1` \ No newline at end of file + mod(EXTRACT(dayofweek FROM DATE(2017, 1, 1)) + 5, 7) AS `DayOfWeekIndex_datetime_date_2017_ 1_ 1` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp_date/name.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp_date/name.sql index da4b415da6de..f207aa60fec7 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp_date/name.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_day_of_week/timestamp_date/name.sql @@ -1,2 +1,2 @@ SELECT - INITCAP(CAST(CAST('2017-01-01' AS DATE) AS STRING FORMAT 'DAY')) AS `DayOfWeekName_datetime_date_2017_ 1_ 1` \ No newline at end of file + INITCAP(CAST(DATE(2017, 1, 1) AS STRING FORMAT 'DAY')) AS `DayOfWeekName_datetime_date_2017_ 1_ 1` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_divide_by_zero/floordiv/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_divide_by_zero/floordiv/out.sql index 23ca5617a3c5..021c289c519a 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_divide_by_zero/floordiv/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_divide_by_zero/floordiv/out.sql @@ -1,3 +1,3 @@ SELECT - CAST(FLOOR(IEEE_DIVIDE(t0.`double_col`, 0)) AS INT64) AS `FloorDivide_double_col_ 0` + CAST(FLOOR(ieee_divide(t0.double_col, 0)) AS INT64) AS `FloorDivide_double_col_ 0` FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_divide_by_zero/truediv/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_divide_by_zero/truediv/out.sql index bf88197b4112..2f451f747c62 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_divide_by_zero/truediv/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_divide_by_zero/truediv/out.sql @@ -1,3 +1,3 @@ SELECT - IEEE_DIVIDE(t0.`double_col`, 0) AS `Divide_double_col_ 0` + ieee_divide(t0.double_col, 0) AS `Divide_double_col_ 0` FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_extract_temporal_from_timestamp/date/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_extract_temporal_from_timestamp/date/out.sql index 9098cd2126ba..bc3328e12ee8 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_extract_temporal_from_timestamp/date/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_extract_temporal_from_timestamp/date/out.sql @@ -1,3 +1,3 @@ SELECT - DATE(t0.`ts`) AS `tmp` + DATE(t0.ts) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_extract_temporal_from_timestamp/time/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_extract_temporal_from_timestamp/time/out.sql index 4847f7b392f3..c35dfe0331a2 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_extract_temporal_from_timestamp/time/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_extract_temporal_from_timestamp/time/out.sql @@ -1,3 +1,3 @@ SELECT - TIME(t0.`ts`) AS `tmp` + time(t0.ts) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_azimuth/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_azimuth/out.sql index 05e9145e119e..c1326749682c 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_azimuth/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_azimuth/out.sql @@ -1,3 +1,3 @@ SELECT - ST_AZIMUTH(t0.`p0`, t0.`p1`) AS `tmp` + st_azimuth(t0.p0, t0.p1) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/contains/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/contains/out.sql index 1b1a82779e31..41db52f14ef0 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/contains/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/contains/out.sql @@ -1,3 +1,3 @@ SELECT - ST_CONTAINS(t0.`geog0`, t0.`geog1`) AS `tmp` + st_contains(t0.geog0, t0.geog1) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/covered_by/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/covered_by/out.sql index 35876973cb31..45073f686366 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/covered_by/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/covered_by/out.sql @@ -1,3 +1,3 @@ SELECT - ST_COVEREDBY(t0.`geog0`, t0.`geog1`) AS `tmp` + st_coveredby(t0.geog0, t0.geog1) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/covers/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/covers/out.sql index 1e347e6b1ae1..316f696e43df 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/covers/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/covers/out.sql @@ -1,3 +1,3 @@ SELECT - ST_COVERS(t0.`geog0`, t0.`geog1`) AS `tmp` + st_covers(t0.geog0, t0.geog1) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/d_within/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/d_within/out.sql index 436a09b96fb3..95b081d2f324 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/d_within/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/d_within/out.sql @@ -1,3 +1,3 @@ SELECT - ST_DWITHIN(t0.`geog0`, t0.`geog1`, 5.2) AS `tmp` + st_dwithin(t0.geog0, t0.geog1, 5.2) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/difference/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/difference/out.sql index 28a0a1ca0241..3680bf6dc8cf 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/difference/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/difference/out.sql @@ -1,3 +1,3 @@ SELECT - ST_DIFFERENCE(t0.`geog0`, t0.`geog1`) AS `tmp` + st_difference(t0.geog0, t0.geog1) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/disjoint/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/disjoint/out.sql index 03e7ddd2697f..47f67918fdaf 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/disjoint/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/disjoint/out.sql @@ -1,3 +1,3 @@ SELECT - ST_DISJOINT(t0.`geog0`, t0.`geog1`) AS `tmp` + st_disjoint(t0.geog0, t0.geog1) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/distance/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/distance/out.sql index 8039f2517172..dd498ab034f6 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/distance/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/distance/out.sql @@ -1,3 +1,3 @@ SELECT - ST_DISTANCE(t0.`geog0`, t0.`geog1`) AS `tmp` + st_distance(t0.geog0, t0.geog1) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/geo_equals/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/geo_equals/out.sql index 0d50e35c1aae..525bdd5a98c5 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/geo_equals/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/geo_equals/out.sql @@ -1,3 +1,3 @@ SELECT - ST_EQUALS(t0.`geog0`, t0.`geog1`) AS `tmp` + st_equals(t0.geog0, t0.geog1) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/intersection/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/intersection/out.sql index 66b95db7fc95..881691e93e02 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/intersection/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/intersection/out.sql @@ -1,3 +1,3 @@ SELECT - ST_INTERSECTION(t0.`geog0`, t0.`geog1`) AS `tmp` + st_intersection(t0.geog0, t0.geog1) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/intersects/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/intersects/out.sql index 7b1912963355..9768493f1bbc 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/intersects/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/intersects/out.sql @@ -1,3 +1,3 @@ SELECT - ST_INTERSECTS(t0.`geog0`, t0.`geog1`) AS `tmp` + st_intersects(t0.geog0, t0.geog1) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/max_distance/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/max_distance/out.sql index fe6ed8cc970c..bebf364b68f8 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/max_distance/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/max_distance/out.sql @@ -1,3 +1,3 @@ SELECT - ST_MAXDISTANCE(t0.`geog0`, t0.`geog1`) AS `tmp` + st_maxdistance(t0.geog0, t0.geog1) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/touches/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/touches/out.sql index d6e4482d3630..3d50dc04d227 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/touches/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/touches/out.sql @@ -1,3 +1,3 @@ SELECT - ST_TOUCHES(t0.`geog0`, t0.`geog1`) AS `tmp` + st_touches(t0.geog0, t0.geog1) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/union/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/union/out.sql index ba3b55982ea3..8f5d1d5a8f42 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/union/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/union/out.sql @@ -1,3 +1,3 @@ SELECT - ST_UNION(t0.`geog0`, t0.`geog1`) AS `tmp` + st_union(t0.geog0, t0.geog1) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/within/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/within/out.sql index 5008c4637d85..9ac98c0a3feb 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/within/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_binary/within/out.sql @@ -1,3 +1,3 @@ SELECT - ST_WITHIN(t0.`geog0`, t0.`geog1`) AS `tmp` + st_within(t0.geog0, t0.geog1) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/x_max/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/x_max/out.sql index 5a1e009d3368..3acaebab60b6 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/x_max/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/x_max/out.sql @@ -1,3 +1,3 @@ SELECT - ST_BOUNDINGBOX(t0.`geog`).xmax AS `tmp` + st_boundingbox(t0.geog).xmax AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/x_min/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/x_min/out.sql index 8b3e8d89f450..3e46dda22606 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/x_min/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/x_min/out.sql @@ -1,3 +1,3 @@ SELECT - ST_BOUNDINGBOX(t0.`geog`).xmin AS `tmp` + st_boundingbox(t0.geog).xmin AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/y_max/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/y_max/out.sql index a1fdaad96165..f2163789f6e8 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/y_max/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/y_max/out.sql @@ -1,3 +1,3 @@ SELECT - ST_BOUNDINGBOX(t0.`geog`).ymax AS `tmp` + st_boundingbox(t0.geog).ymax AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/y_min/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/y_min/out.sql index e3efd9949a19..dc148200be48 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/y_min/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_minmax/y_min/out.sql @@ -1,3 +1,3 @@ SELECT - ST_BOUNDINGBOX(t0.`geog`).ymin AS `tmp` + st_boundingbox(t0.geog).ymin AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_point/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_point/out.sql index 89c468270545..a57d8a6952f5 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_point/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_point/out.sql @@ -1,3 +1,3 @@ SELECT - ST_GEOGPOINT(t0.`lon`, t0.`lat`) AS `tmp` + st_geogpoint(t0.lon, t0.lat) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_simplify/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_simplify/out.sql index 3d194f1e0368..0f9f1b6cc556 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_simplify/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_simplify/out.sql @@ -1,3 +1,3 @@ SELECT - ST_SIMPLIFY(t0.`geog`, 5.2) AS `tmp` + st_simplify(t0.geog, 5.2) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/aread/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/aread/out.sql index b16445decdb4..6b56cb853600 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/aread/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/aread/out.sql @@ -1,3 +1,3 @@ SELECT - ST_AREA(t0.`geog`) AS `tmp` + st_area(t0.geog) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/as_binary/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/as_binary/out.sql index f9875a1abe00..d08e811dab1a 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/as_binary/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/as_binary/out.sql @@ -1,3 +1,3 @@ SELECT - ST_ASBINARY(t0.`geog`) AS `tmp` + st_asbinary(t0.geog) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/as_text/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/as_text/out.sql index a42c9d599cc1..5a15cdcf347a 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/as_text/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/as_text/out.sql @@ -1,3 +1,3 @@ SELECT - ST_ASTEXT(t0.`geog`) AS `tmp` + st_astext(t0.geog) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/buffer/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/buffer/out.sql index e5990a079594..026f5a13ab9c 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/buffer/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/buffer/out.sql @@ -1,3 +1,3 @@ SELECT - ST_BUFFER(t0.`geog`, 5.2) AS `tmp` + st_buffer(t0.geog, 5.2) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/centroid/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/centroid/out.sql index 7b9e4235e496..c486a43d1f54 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/centroid/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/centroid/out.sql @@ -1,3 +1,3 @@ SELECT - ST_CENTROID(t0.`geog`) AS `tmp` + st_centroid(t0.geog) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/end_point/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/end_point/out.sql index 3512019b95a3..0f4f517bc5d7 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/end_point/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/end_point/out.sql @@ -1,3 +1,3 @@ SELECT - ST_ENDPOINT(t0.`geog`) AS `tmp` + st_endpoint(t0.geog) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/geometry_type/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/geometry_type/out.sql index 152acfd628b3..6b87c638e82d 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/geometry_type/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/geometry_type/out.sql @@ -1,3 +1,3 @@ SELECT - ST_GEOMETRYTYPE(t0.`geog`) AS `tmp` + st_geometrytype(t0.geog) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/length/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/length/out.sql index 5c604906d5a7..cc0c00c15a40 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/length/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/length/out.sql @@ -1,3 +1,3 @@ SELECT - ST_LENGTH(t0.`geog`) AS `tmp` + st_length(t0.geog) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/npoints/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/npoints/out.sql index 1678db820185..ce6c5c6a9988 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/npoints/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/npoints/out.sql @@ -1,3 +1,3 @@ SELECT - ST_NUMPOINTS(t0.`geog`) AS `tmp` + st_numpoints(t0.geog) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/perimeter/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/perimeter/out.sql index 1cc8832884ea..62b3f322440b 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/perimeter/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/perimeter/out.sql @@ -1,3 +1,3 @@ SELECT - ST_PERIMETER(t0.`geog`) AS `tmp` + st_perimeter(t0.geog) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/point_n/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/point_n/out.sql index 12471430c8c7..0b03f583db71 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/point_n/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/point_n/out.sql @@ -1,3 +1,3 @@ SELECT - ST_POINTN(t0.`geog`, 3) AS `tmp` + st_pointn(t0.geog, 3) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/start_point/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/start_point/out.sql index e75c80c0c11e..9c77d20eaae8 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/start_point/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary/start_point/out.sql @@ -1,3 +1,3 @@ SELECT - ST_STARTPOINT(t0.`geog`) AS `tmp` + st_startpoint(t0.geog) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary_union/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary_union/out.sql index 884ee4e6d94e..112d191a4eae 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary_union/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_unary_union/out.sql @@ -1,3 +1,3 @@ SELECT - ST_UNION_AGG(t0.`geog`) AS `tmp` + st_union_agg(t0.geog) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_xy/x/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_xy/x/out.sql index 1afb858c9035..76b1ac85d03e 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_xy/x/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_xy/x/out.sql @@ -1,3 +1,3 @@ SELECT - ST_X(t0.`pt`) AS `tmp` + st_x(t0.pt) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_xy/y/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_xy/y/out.sql index b89b748ddc35..472068d4b795 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_xy/y/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_geospatial_xy/y/out.sql @@ -1,3 +1,3 @@ SELECT - ST_Y(t0.`pt`) AS `tmp` + st_y(t0.pt) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hash/binary/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hash/binary/out.sql index af390b72dfb2..16b493bec461 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hash/binary/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hash/binary/out.sql @@ -1,2 +1,2 @@ SELECT - farm_fingerprint(b'test of hash') AS `Hash_b'test of hash'` \ No newline at end of file + farm_fingerprint(CAST('74657374206f662068617368' AS BYTES FORMAT 'HEX')) AS `Hash_b'test of hash'` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/md5-test-binary/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/md5-test-binary/out.sql index 097f00501480..06e3d4e02c04 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/md5-test-binary/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/md5-test-binary/out.sql @@ -1,2 +1,2 @@ SELECT - MD5(b'test') AS `tmp` \ No newline at end of file + TO_HEX(MD5(CAST('74657374' AS BYTES FORMAT 'HEX'))) AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/md5-test-string/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/md5-test-string/out.sql index 8f15582c94e8..a9b91d0a8fd8 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/md5-test-string/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/md5-test-string/out.sql @@ -1,2 +1,2 @@ SELECT - MD5('test') AS `tmp` \ No newline at end of file + TO_HEX(MD5('test')) AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha1-test-binary/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha1-test-binary/out.sql index a93af9c0c71d..b47ac212bf11 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha1-test-binary/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha1-test-binary/out.sql @@ -1,2 +1,2 @@ SELECT - sha1(b'test') AS `tmp` \ No newline at end of file + SHA(CAST('74657374' AS BYTES FORMAT 'HEX')) AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha1-test-string/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha1-test-string/out.sql index 04a6f16e94cb..006923cc428f 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha1-test-string/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha1-test-string/out.sql @@ -1,2 +1,2 @@ SELECT - sha1('test') AS `tmp` \ No newline at end of file + SHA('test') AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha256-test-binary/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha256-test-binary/out.sql index de7f002062c7..66952364a23a 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha256-test-binary/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha256-test-binary/out.sql @@ -1,2 +1,2 @@ SELECT - SHA256(b'test') AS `tmp` \ No newline at end of file + sha256(CAST('74657374' AS BYTES FORMAT 'HEX')) AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha256-test-string/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha256-test-string/out.sql index da8766656e8f..eee9f3b6f5c6 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha256-test-string/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha256-test-string/out.sql @@ -1,2 +1,2 @@ SELECT - SHA256('test') AS `tmp` \ No newline at end of file + sha256('test') AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha512-test-binary/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha512-test-binary/out.sql index fe8843530cd5..0a34496a1b95 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha512-test-binary/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha512-test-binary/out.sql @@ -1,2 +1,2 @@ SELECT - SHA512(b'test') AS `tmp` \ No newline at end of file + sha512(CAST('74657374' AS BYTES FORMAT 'HEX')) AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha512-test-string/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha512-test-string/out.sql index 6097e6df15c1..ba16c3204a26 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha512-test-string/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_hashbytes/sha512-test-string/out.sql @@ -1,2 +1,2 @@ SELECT - SHA512('test') AS `tmp` \ No newline at end of file + sha512('test') AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_identical_to/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_identical_to/out.sql index a23fe427cf02..dbd6d0bc38db 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_identical_to/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_identical_to/out.sql @@ -1,10 +1,18 @@ SELECT - t0.* + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month FROM functional_alltypes AS t0 WHERE - ( - t0.`string_col` IS NOT DISTINCT FROM 'a' - ) - AND ( - t0.`date_string_col` IS NOT DISTINCT FROM 'b' - ) \ No newline at end of file + t0.string_col IS NOT DISTINCT FROM 'a' + AND t0.date_string_col IS NOT DISTINCT FROM 'b' \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/ms/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/ms/out.sql index 52c8b7dd7a1f..108f5bdd6655 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/ms/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/ms/out.sql @@ -1,2 +1,2 @@ SELECT - TIMESTAMP_MILLIS(-123456789) AS `tmp` \ No newline at end of file + timestamp_millis(-123456789) AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/ns/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/ns/out.sql index f38a5acbc58a..aaa93e0528b8 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/ns/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/ns/out.sql @@ -1,2 +1,2 @@ SELECT - TIMESTAMP_MICROS(CAST(ROUND(1234567891011 / 1000) AS INT64)) AS `tmp` \ No newline at end of file + timestamp_micros(CAST(ROUND(1234567891011 / 1000) AS INT64)) AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/s/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/s/out.sql index 14033e028843..a470e62fd7a5 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/s/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/s/out.sql @@ -1,2 +1,2 @@ SELECT - TIMESTAMP_SECONDS(123456789) AS `tmp` \ No newline at end of file + timestamp_seconds(123456789) AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/us/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/us/out.sql index e77ca976a4ae..9c2e88bc505f 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/us/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_integer_to_timestamp/us/out.sql @@ -1,2 +1,2 @@ SELECT - TIMESTAMP_MICROS(123456789) AS `tmp` \ No newline at end of file + timestamp_micros(123456789) AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/datetime/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/datetime/out.sql index 742c13e36aa5..4a5e85730e47 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/datetime/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/datetime/out.sql @@ -1,2 +1,2 @@ SELECT - EXTRACT(hour FROM CAST('2017-01-01 04:55:59' AS DATETIME)) AS `tmp` \ No newline at end of file + EXTRACT(HOUR FROM datetime('2017-01-01T04:55:59')) AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/string_time/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/string_time/out.sql index a227037389f3..b0b094d49530 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/string_time/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/string_time/out.sql @@ -1,2 +1,2 @@ SELECT - EXTRACT(hour FROM CAST('04:55:59' AS TIME)) AS `tmp` \ No newline at end of file + EXTRACT(HOUR FROM time(4, 55, 59)) AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/string_timestamp/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/string_timestamp/out.sql index 742c13e36aa5..4a5e85730e47 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/string_timestamp/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/string_timestamp/out.sql @@ -1,2 +1,2 @@ SELECT - EXTRACT(hour FROM CAST('2017-01-01 04:55:59' AS DATETIME)) AS `tmp` \ No newline at end of file + EXTRACT(HOUR FROM datetime('2017-01-01T04:55:59')) AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/time/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/time/out.sql index a227037389f3..b0b094d49530 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/time/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/time/out.sql @@ -1,2 +1,2 @@ SELECT - EXTRACT(hour FROM CAST('04:55:59' AS TIME)) AS `tmp` \ No newline at end of file + EXTRACT(HOUR FROM time(4, 55, 59)) AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/timestamp/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/timestamp/out.sql index 742c13e36aa5..4a5e85730e47 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/timestamp/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_timestamp_or_time/timestamp/out.sql @@ -1,2 +1,2 @@ SELECT - EXTRACT(hour FROM CAST('2017-01-01 04:55:59' AS DATETIME)) AS `tmp` \ No newline at end of file + EXTRACT(HOUR FROM datetime('2017-01-01T04:55:59')) AS tmp \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/date/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/date/out.sql index c01e22721bb5..c4e62dab9bdf 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/date/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/date/out.sql @@ -1,2 +1,2 @@ SELECT - EXTRACT(year FROM CAST('2017-01-01' AS DATE)) AS `ExtractYear_datetime_date_2017_ 1_ 1` \ No newline at end of file + EXTRACT(YEAR FROM DATE(2017, 1, 1)) AS `ExtractYear_datetime_date_2017_ 1_ 1` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/datetime/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/datetime/out.sql index e3919b046dfa..c1f82282802c 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/datetime/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/datetime/out.sql @@ -1,2 +1,2 @@ SELECT - EXTRACT(year FROM CAST('2017-01-01 04:55:59' AS DATETIME)) AS `ExtractYear_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file + EXTRACT(YEAR FROM datetime('2017-01-01T04:55:59')) AS `ExtractYear_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/string_date/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/string_date/out.sql index c01e22721bb5..c4e62dab9bdf 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/string_date/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/string_date/out.sql @@ -1,2 +1,2 @@ SELECT - EXTRACT(year FROM CAST('2017-01-01' AS DATE)) AS `ExtractYear_datetime_date_2017_ 1_ 1` \ No newline at end of file + EXTRACT(YEAR FROM DATE(2017, 1, 1)) AS `ExtractYear_datetime_date_2017_ 1_ 1` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/string_timestamp/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/string_timestamp/out.sql index e3919b046dfa..c1f82282802c 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/string_timestamp/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/string_timestamp/out.sql @@ -1,2 +1,2 @@ SELECT - EXTRACT(year FROM CAST('2017-01-01 04:55:59' AS DATETIME)) AS `ExtractYear_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file + EXTRACT(YEAR FROM datetime('2017-01-01T04:55:59')) AS `ExtractYear_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/timestamp/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/timestamp/out.sql index e3919b046dfa..c1f82282802c 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/timestamp/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/timestamp/out.sql @@ -1,2 +1,2 @@ SELECT - EXTRACT(year FROM CAST('2017-01-01 04:55:59' AS DATETIME)) AS `ExtractYear_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file + EXTRACT(YEAR FROM datetime('2017-01-01T04:55:59')) AS `ExtractYear_datetime_datetime_2017_ 1_ 1_ 4_ 55_ 59` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/timestamp_date/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/timestamp_date/out.sql index c01e22721bb5..c4e62dab9bdf 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/timestamp_date/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_literal_year/timestamp_date/out.sql @@ -1,2 +1,2 @@ SELECT - EXTRACT(year FROM CAST('2017-01-01' AS DATE)) AS `ExtractYear_datetime_date_2017_ 1_ 1` \ No newline at end of file + EXTRACT(YEAR FROM DATE(2017, 1, 1)) AS `ExtractYear_datetime_date_2017_ 1_ 1` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_now/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_now/out.sql index 3eecc7336d34..c86165f516de 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_now/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_now/out.sql @@ -1,2 +1,2 @@ SELECT - CURRENT_TIMESTAMP() AS `TimestampNow` \ No newline at end of file + CURRENT_TIMESTAMP() AS TimestampNow \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_projection_fusion_only_peeks_at_immediate_parent/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_projection_fusion_only_peeks_at_immediate_parent/out.sql index a064f3cabc1e..c64aff514d1f 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_projection_fusion_only_peeks_at_immediate_parent/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_projection_fusion_only_peeks_at_immediate_parent/out.sql @@ -1,28 +1,26 @@ -WITH t0 AS ( +SELECT + t2.file_date, + t2.PARTITIONTIME, + t2.val, + t2.XYZ +FROM ( SELECT - t5.* - FROM unbound_table AS t5 + CAST(t0.file_date AS DATE) AS file_date, + t0.PARTITIONTIME, + t0.val, + t0.val * 2 AS XYZ + FROM unbound_table AS t0 WHERE - t5.`PARTITIONTIME` < CAST('2017-01-01' AS DATE) -), t1 AS ( - SELECT - CAST(t0.`file_date` AS DATE) AS `file_date`, - t0.`PARTITIONTIME`, - t0.`val` - FROM t0 -), t2 AS ( + t0.PARTITIONTIME < DATE(2017, 1, 1) AND CAST(t0.file_date AS DATE) < DATE(2017, 1, 1) +) AS t2 +INNER JOIN ( SELECT - t1.* - FROM t1 + CAST(t0.file_date AS DATE) AS file_date, + t0.PARTITIONTIME, + t0.val, + t0.val * 2 AS XYZ + FROM unbound_table AS t0 WHERE - t1.`file_date` < CAST('2017-01-01' AS DATE) -), t3 AS ( - SELECT - t2.*, - t2.`val` * 2 AS `XYZ` - FROM t2 -) -SELECT - t3.* -FROM t3 -INNER JOIN t3 AS t4 \ No newline at end of file + t0.PARTITIONTIME < DATE(2017, 1, 1) AND CAST(t0.file_date AS DATE) < DATE(2017, 1, 1) +) AS t4 + ON TRUE \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_range_window_function/prec_foll/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_range_window_function/prec_foll/out.sql index f128f6c60f1a..b429e0b40a4b 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_range_window_function/prec_foll/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_range_window_function/prec_foll/out.sql @@ -1,4 +1,16 @@ SELECT - t0.*, - avg(t0.`float_col`) OVER (PARTITION BY t0.`year` ORDER BY t0.`month` ASC RANGE BETWEEN 1 PRECEDING AND CURRENT ROW) AS `two_month_avg` + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month, + AVG(t0.float_col) OVER (PARTITION BY t0.year ORDER BY t0.month ASC RANGE BETWEEN 1 preceding AND CURRENT ROW) AS two_month_avg FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_range_window_function/prec_prec/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_range_window_function/prec_prec/out.sql index c3432e74aa9d..28f748f8d387 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_range_window_function/prec_prec/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_range_window_function/prec_prec/out.sql @@ -1,4 +1,16 @@ SELECT - t0.*, - avg(t0.`float_col`) OVER (PARTITION BY t0.`year` ORDER BY UNIX_MICROS(t0.`timestamp_col`) ASC RANGE BETWEEN 4 PRECEDING AND 2 PRECEDING) AS `two_month_avg` + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month, + AVG(t0.float_col) OVER (PARTITION BY t0.year ORDER BY t0.timestamp_col ASC RANGE BETWEEN 4 preceding AND 2 preceding) AS two_month_avg FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/difference/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/difference/out.sql index 957f0bc1ed5b..1fba54566ec5 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/difference/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/difference/out.sql @@ -1,11 +1,11 @@ SELECT - t0.`a` + t2.a FROM ( SELECT - t1.* - FROM t0 AS t1 + * + FROM t0 AS t0 EXCEPT DISTINCT SELECT - t1.* - FROM t1 -) AS t0 \ No newline at end of file + * + FROM t1 AS t1 +) AS t2 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/intersect/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/intersect/out.sql index f93956e195ff..0300759946af 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/intersect/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/intersect/out.sql @@ -1,11 +1,11 @@ SELECT - t0.`a` + t2.a FROM ( SELECT - t1.* - FROM t0 AS t1 + * + FROM t0 AS t0 INTERSECT DISTINCT SELECT - t1.* - FROM t1 -) AS t0 \ No newline at end of file + * + FROM t1 AS t1 +) AS t2 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/union_all/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/union_all/out.sql index 0d385460634b..65ca427a3cc1 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/union_all/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/union_all/out.sql @@ -1,11 +1,11 @@ SELECT - t0.`a` + t2.a FROM ( SELECT - t1.* - FROM t0 AS t1 + * + FROM t0 AS t0 UNION ALL SELECT - t1.* - FROM t1 -) AS t0 \ No newline at end of file + * + FROM t1 AS t1 +) AS t2 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/union_distinct/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/union_distinct/out.sql index 08eb83fc95fd..f060886f0ca7 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/union_distinct/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_set_operation/union_distinct/out.sql @@ -1,11 +1,11 @@ SELECT - t0.`a` + t2.a FROM ( SELECT - t1.* - FROM t0 AS t1 + * + FROM t0 AS t0 UNION DISTINCT SELECT - t1.* - FROM t1 -) AS t0 \ No newline at end of file + * + FROM t1 AS t1 +) AS t2 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_substring/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_substring/out.sql index 5aa66a18ed01..dde97310a2d4 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_substring/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_substring/out.sql @@ -1,7 +1,3 @@ SELECT - IF( - 3 >= 0, - SUBSTR(t0.`value`, 3 + 1, 1), - SUBSTR(t0.`value`, LENGTH(t0.`value`) + 3 + 1, 1) - ) AS `tmp` + IF(3 >= 0, substr(t0.value, 3 + 1, 1), substr(t0.value, LENGTH(t0.value) + 3 + 1, 1)) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/day-date/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/day-date/out.sql index 9c2606fa97ea..cfda92082d60 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/day-date/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/day-date/out.sql @@ -1,3 +1,3 @@ SELECT - DATE_TRUNC(t0.`a`, DAY) AS `tmp` + DATE_TRUNC(t0.a, DAY) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/day-timestamp/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/day-timestamp/out.sql index 123365031a50..5914e551cdfa 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/day-timestamp/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/day-timestamp/out.sql @@ -1,3 +1,3 @@ SELECT - TIMESTAMP_TRUNC(t0.`a`, DAY) AS `tmp` + TIMESTAMP_TRUNC(t0.a, DAY) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/hour-time/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/hour-time/out.sql index 0dab09b39086..627fd52607ea 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/hour-time/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/hour-time/out.sql @@ -1,3 +1,3 @@ SELECT - TIME_TRUNC(t0.`a`, HOUR) AS `tmp` + TIME_TRUNC(t0.a, HOUR) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/hour-timestamp/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/hour-timestamp/out.sql index b2cb572f2089..2eb17fc72aa7 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/hour-timestamp/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/hour-timestamp/out.sql @@ -1,3 +1,3 @@ SELECT - TIMESTAMP_TRUNC(t0.`a`, HOUR) AS `tmp` + TIMESTAMP_TRUNC(t0.a, HOUR) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/micros-time/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/micros-time/out.sql index 2c51c76585e8..bc83a4627907 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/micros-time/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/micros-time/out.sql @@ -1,3 +1,3 @@ SELECT - TIME_TRUNC(t0.`a`, MICROSECOND) AS `tmp` + TIME_TRUNC(t0.a, MICROSECOND) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/micros-timestamp/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/micros-timestamp/out.sql index 294a6422566b..85129ae2ca98 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/micros-timestamp/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/micros-timestamp/out.sql @@ -1,3 +1,3 @@ SELECT - TIMESTAMP_TRUNC(t0.`a`, MICROSECOND) AS `tmp` + TIMESTAMP_TRUNC(t0.a, MICROSECOND) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/millis-time/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/millis-time/out.sql index f985ffa2058e..d568ac473f24 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/millis-time/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/millis-time/out.sql @@ -1,3 +1,3 @@ SELECT - TIME_TRUNC(t0.`a`, MILLISECOND) AS `tmp` + TIME_TRUNC(t0.a, MILLISECOND) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/millis-timestamp/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/millis-timestamp/out.sql index 3f2f5970047f..eeb61582028e 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/millis-timestamp/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/millis-timestamp/out.sql @@ -1,3 +1,3 @@ SELECT - TIMESTAMP_TRUNC(t0.`a`, MILLISECOND) AS `tmp` + TIMESTAMP_TRUNC(t0.a, MILLISECOND) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/minute-time/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/minute-time/out.sql index e10273b41268..53a26cff7227 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/minute-time/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/minute-time/out.sql @@ -1,3 +1,3 @@ SELECT - TIME_TRUNC(t0.`a`, MINUTE) AS `tmp` + TIME_TRUNC(t0.a, MINUTE) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/minute-timestamp/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/minute-timestamp/out.sql index aadc0830deef..75ed2f48e4f3 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/minute-timestamp/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/minute-timestamp/out.sql @@ -1,3 +1,3 @@ SELECT - TIMESTAMP_TRUNC(t0.`a`, MINUTE) AS `tmp` + TIMESTAMP_TRUNC(t0.a, MINUTE) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/month-date/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/month-date/out.sql index 02a34e41ad78..08aea93a5d16 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/month-date/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/month-date/out.sql @@ -1,3 +1,3 @@ SELECT - DATE_TRUNC(t0.`a`, MONTH) AS `tmp` + DATE_TRUNC(t0.a, MONTH) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/month-timestamp/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/month-timestamp/out.sql index ae1748be8c8f..fb5755607afc 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/month-timestamp/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/month-timestamp/out.sql @@ -1,3 +1,3 @@ SELECT - TIMESTAMP_TRUNC(t0.`a`, MONTH) AS `tmp` + TIMESTAMP_TRUNC(t0.a, MONTH) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/quarter-date/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/quarter-date/out.sql index 9954515fe93f..69fb0c1c0073 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/quarter-date/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/quarter-date/out.sql @@ -1,3 +1,3 @@ SELECT - DATE_TRUNC(t0.`a`, QUARTER) AS `tmp` + DATE_TRUNC(t0.a, QUARTER) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/quarter-timestamp/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/quarter-timestamp/out.sql index f3e39becf528..6ae384abfe45 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/quarter-timestamp/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/quarter-timestamp/out.sql @@ -1,3 +1,3 @@ SELECT - TIMESTAMP_TRUNC(t0.`a`, QUARTER) AS `tmp` + TIMESTAMP_TRUNC(t0.a, QUARTER) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/second-time/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/second-time/out.sql index 4ca550c5abea..ed1a4f61a766 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/second-time/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/second-time/out.sql @@ -1,3 +1,3 @@ SELECT - TIME_TRUNC(t0.`a`, SECOND) AS `tmp` + TIME_TRUNC(t0.a, SECOND) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/second-timestamp/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/second-timestamp/out.sql index 31ec37d86b72..ca6b3eea53d1 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/second-timestamp/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/second-timestamp/out.sql @@ -1,3 +1,3 @@ SELECT - TIMESTAMP_TRUNC(t0.`a`, SECOND) AS `tmp` + TIMESTAMP_TRUNC(t0.a, SECOND) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/week-date/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/week-date/out.sql index 024ff840348f..5fa3caed910e 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/week-date/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/week-date/out.sql @@ -1,3 +1,3 @@ SELECT - DATE_TRUNC(t0.`a`, WEEK(MONDAY)) AS `tmp` + DATE_TRUNC(t0.a, WEEK(MONDAY)) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/week-timestamp/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/week-timestamp/out.sql index 461c643074de..116adb3c510c 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/week-timestamp/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/week-timestamp/out.sql @@ -1,3 +1,3 @@ SELECT - TIMESTAMP_TRUNC(t0.`a`, WEEK(MONDAY)) AS `tmp` + TIMESTAMP_TRUNC(t0.a, WEEK(MONDAY)) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/year-date/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/year-date/out.sql index f3d4c321fb3b..6c6515bd7737 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/year-date/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/year-date/out.sql @@ -1,3 +1,3 @@ SELECT - DATE_TRUNC(t0.`a`, YEAR) AS `tmp` + DATE_TRUNC(t0.a, YEAR) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/year-timestamp/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/year-timestamp/out.sql index 20551a51be41..9b639983ccda 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/year-timestamp/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_temporal_truncate/year-timestamp/out.sql @@ -1,3 +1,3 @@ SELECT - TIMESTAMP_TRUNC(t0.`a`, YEAR) AS `tmp` + TIMESTAMP_TRUNC(t0.a, YEAR) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_to_timestamp_no_timezone/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_to_timestamp_no_timezone/out.sql index ca0788063c21..13bb7ff2b42b 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_to_timestamp_no_timezone/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_to_timestamp_no_timezone/out.sql @@ -1,3 +1,3 @@ SELECT - PARSE_TIMESTAMP('%F', t0.`date_string_col`) AS `StringToTimestamp_date_string_col_ '%F'` + parse_timestamp('%F', t0.date_string_col, 'UTC') AS `StringToTimestamp_date_string_col_ '%F'` FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_to_timestamp_timezone/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_to_timestamp_timezone/out.sql index 67f139bb19b6..9e192cd1a351 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_to_timestamp_timezone/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_to_timestamp_timezone/out.sql @@ -1,3 +1,3 @@ SELECT - PARSE_TIMESTAMP('%F %Z', CONCAT(t0.`date_string_col`, ' America/New_York')) AS `StringToTimestamp_StringConcat_ '%F %Z'` + parse_timestamp('%F %Z', CONCAT(t0.date_string_col, ' America/New_York'), 'UTC') AS `StringToTimestamp_StringConcat_ '%F %Z'` FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/days/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/days/out.sql index 15bf5ccf4711..45c49c5adc20 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/days/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/days/out.sql @@ -1,4 +1,16 @@ SELECT - t0.*, - avg(t0.`float_col`) OVER (ORDER BY UNIX_MICROS(t0.`timestamp_col`) ASC RANGE BETWEEN 86400000000 PRECEDING AND EXTRACT(DAY FROM INTERVAL '0' DAY) * 86400000000 FOLLOWING) AS `win_avg` + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month, + AVG(t0.float_col) OVER (ORDER BY t0.timestamp_col ASC RANGE BETWEEN INTERVAL '1' DAY preceding AND INTERVAL 0 DAY following) AS win_avg FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/five/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/five/out.sql index f92b75d70240..7483e117fe59 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/five/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/five/out.sql @@ -1,4 +1,16 @@ SELECT - t0.*, - avg(t0.`float_col`) OVER (ORDER BY UNIX_MICROS(t0.`timestamp_col`) ASC RANGE BETWEEN 5 PRECEDING AND CURRENT ROW) AS `win_avg` + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month, + AVG(t0.float_col) OVER (ORDER BY t0.timestamp_col ASC RANGE BETWEEN 5 preceding AND CURRENT ROW) AS win_avg FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/hours/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/hours/out.sql index c96fcf3d161f..497fe55410fe 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/hours/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/hours/out.sql @@ -1,4 +1,16 @@ SELECT - t0.*, - avg(t0.`float_col`) OVER (ORDER BY UNIX_MICROS(t0.`timestamp_col`) ASC RANGE BETWEEN 3600000000 PRECEDING AND EXTRACT(HOUR FROM INTERVAL '0' HOUR) * 3600000000 FOLLOWING) AS `win_avg` + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month, + AVG(t0.float_col) OVER (ORDER BY t0.timestamp_col ASC RANGE BETWEEN INTERVAL '1' HOUR preceding AND INTERVAL 0 HOUR following) AS win_avg FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/micros/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/micros/out.sql index 87ff9a7e26be..37c7b9452f03 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/micros/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/micros/out.sql @@ -1,4 +1,16 @@ SELECT - t0.*, - avg(t0.`float_col`) OVER (ORDER BY UNIX_MICROS(t0.`timestamp_col`) ASC RANGE BETWEEN 1 PRECEDING AND EXTRACT(MICROSECOND FROM INTERVAL '0' MICROSECOND) * 1 FOLLOWING) AS `win_avg` + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month, + AVG(t0.float_col) OVER (ORDER BY t0.timestamp_col ASC RANGE BETWEEN INTERVAL '1' MICROSECOND preceding AND INTERVAL 0 MICROSECOND following) AS win_avg FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/minutes/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/minutes/out.sql index fa4f2044265d..ab8f45c80717 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/minutes/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/minutes/out.sql @@ -1,4 +1,16 @@ SELECT - t0.*, - avg(t0.`float_col`) OVER (ORDER BY UNIX_MICROS(t0.`timestamp_col`) ASC RANGE BETWEEN 60000000 PRECEDING AND EXTRACT(MINUTE FROM INTERVAL '0' MINUTE) * 60000000 FOLLOWING) AS `win_avg` + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month, + AVG(t0.float_col) OVER (ORDER BY t0.timestamp_col ASC RANGE BETWEEN INTERVAL '1' MINUTE preceding AND INTERVAL 0 MINUTE following) AS win_avg FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/nanos/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/nanos/out.sql new file mode 100644 index 000000000000..6a2f458acc07 --- /dev/null +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/nanos/out.sql @@ -0,0 +1,16 @@ +SELECT + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month, + AVG(t0.float_col) OVER (ORDER BY t0.timestamp_col ASC NULLS LAST RANGE BETWEEN INTERVAL '1' NANOSECOND preceding AND CAST(0 AS INTERVAL NANOSECOND) following) AS win_avg +FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/seconds/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/seconds/out.sql index 81bbc1eab462..88e0656e843b 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/seconds/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/seconds/out.sql @@ -1,4 +1,16 @@ SELECT - t0.*, - avg(t0.`float_col`) OVER (ORDER BY UNIX_MICROS(t0.`timestamp_col`) ASC RANGE BETWEEN 1000000 PRECEDING AND EXTRACT(SECOND FROM INTERVAL '0' SECOND) * 1000000 FOLLOWING) AS `win_avg` + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month, + AVG(t0.float_col) OVER (ORDER BY t0.timestamp_col ASC RANGE BETWEEN INTERVAL '1' SECOND preceding AND INTERVAL 0 SECOND following) AS win_avg FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/two_days/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/two_days/out.sql index 0b58d5414c0b..b2e631f727c3 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/two_days/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/two_days/out.sql @@ -1,4 +1,16 @@ SELECT - t0.*, - avg(t0.`float_col`) OVER (ORDER BY UNIX_MICROS(t0.`timestamp_col`) ASC RANGE BETWEEN EXTRACT(DAY FROM INTERVAL '2' DAY) * 86400000000 PRECEDING AND EXTRACT(DAY FROM INTERVAL '0' DAY) * 86400000000 FOLLOWING) AS `win_avg` + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month, + AVG(t0.float_col) OVER (ORDER BY t0.timestamp_col ASC RANGE BETWEEN INTERVAL (EXTRACT(DAY FROM INTERVAL '1' DAY) * 2) DAY preceding AND INTERVAL 0 DAY following) AS win_avg FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/week/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/week/out.sql index ac28f7967fad..db904f4be055 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/week/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_trailing_range_window/week/out.sql @@ -1,3 +1,16 @@ -SELECT t0.*, - avg(t0.`float_col`) OVER (ORDER BY UNIX_MICROS(t0.`timestamp_col`) ASC RANGE BETWEEN 604800000000 PRECEDING AND EXTRACT(WEEK from INTERVAL 0 WEEK) * 604800000000 FOLLOWING) AS `win_avg` -FROM functional_alltypes t0 \ No newline at end of file +SELECT + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month, + AVG(t0.float_col) OVER (ORDER BY t0.timestamp_col ASC RANGE BETWEEN INTERVAL '1' WEEK preceding AND INTERVAL 0 WEEK following) AS win_avg +FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union/False/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union/False/out.sql index 64d48db414a7..de91c71e2d5d 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union/False/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union/False/out.sql @@ -1,23 +1,23 @@ SELECT - t0.`id`, - t0.`bool_col`, - t0.`tinyint_col`, - t0.`smallint_col`, - t0.`int_col`, - t0.`bigint_col`, - t0.`float_col`, - t0.`double_col`, - t0.`date_string_col`, - t0.`string_col`, - t0.`timestamp_col`, - t0.`year`, - t0.`month` + t1.id, + t1.bool_col, + t1.tinyint_col, + t1.smallint_col, + t1.int_col, + t1.bigint_col, + t1.float_col, + t1.double_col, + t1.date_string_col, + t1.string_col, + t1.timestamp_col, + t1.year, + t1.month FROM ( SELECT - t1.* - FROM functional_alltypes AS t1 + * + FROM functional_alltypes AS t0 UNION ALL SELECT - t1.* - FROM functional_alltypes AS t1 -) AS t0 \ No newline at end of file + * + FROM functional_alltypes AS t0 +) AS t1 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union/True/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union/True/out.sql index 580b185465fb..84d01ac2951a 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union/True/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union/True/out.sql @@ -1,23 +1,23 @@ SELECT - t0.`id`, - t0.`bool_col`, - t0.`tinyint_col`, - t0.`smallint_col`, - t0.`int_col`, - t0.`bigint_col`, - t0.`float_col`, - t0.`double_col`, - t0.`date_string_col`, - t0.`string_col`, - t0.`timestamp_col`, - t0.`year`, - t0.`month` + t1.id, + t1.bool_col, + t1.tinyint_col, + t1.smallint_col, + t1.int_col, + t1.bigint_col, + t1.float_col, + t1.double_col, + t1.date_string_col, + t1.string_col, + t1.timestamp_col, + t1.year, + t1.month FROM ( SELECT - t1.* - FROM functional_alltypes AS t1 + * + FROM functional_alltypes AS t0 UNION DISTINCT SELECT - t1.* - FROM functional_alltypes AS t1 -) AS t0 \ No newline at end of file + * + FROM functional_alltypes AS t0 +) AS t1 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/False-False/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/False-False/out.sql index c2ee631db698..061bd3df1a64 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/False-False/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/False-False/out.sql @@ -1,55 +1,30 @@ -WITH t0 AS ( - SELECT - t2.`string_col`, - sum(t2.`double_col`) AS `metric` - FROM functional_alltypes AS t2 - GROUP BY - 1 -) SELECT - t1.`string_col`, - t1.`metric` + t6.string_col, + t6.metric FROM ( - WITH t0 AS ( + SELECT + t4.string_col, + t4.metric + FROM ( SELECT - t2.`string_col`, - sum(t2.`double_col`) AS `metric` - FROM functional_alltypes AS t2 + t0.string_col, + SUM(t0.double_col) AS metric + FROM functional_alltypes AS t0 GROUP BY 1 - ), t2 AS ( + UNION ALL SELECT - t3.`string_col`, - t3.`metric` - FROM ( - WITH t0 AS ( - SELECT - t2.`string_col`, - sum(t2.`double_col`) AS `metric` - FROM functional_alltypes AS t2 - GROUP BY - 1 - ) - SELECT - * - FROM t0 - UNION ALL - SELECT - t4.`string_col`, - sum(t4.`double_col`) AS `metric` - FROM functional_alltypes AS t4 - GROUP BY - 1 - ) AS t3 - ) - SELECT - * - FROM t2 + t0.string_col, + SUM(t0.double_col) AS metric + FROM functional_alltypes AS t0 + GROUP BY + 1 + ) AS t4 UNION ALL SELECT - t4.`string_col`, - sum(t4.`double_col`) AS `metric` - FROM functional_alltypes AS t4 + t0.string_col, + SUM(t0.double_col) AS metric + FROM functional_alltypes AS t0 GROUP BY 1 -) AS t1 \ No newline at end of file +) AS t6 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/False-True/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/False-True/out.sql index 1ee77c4af309..ef59312a1de9 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/False-True/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/False-True/out.sql @@ -1,55 +1,30 @@ -WITH t0 AS ( - SELECT - t2.`string_col`, - sum(t2.`double_col`) AS `metric` - FROM functional_alltypes AS t2 - GROUP BY - 1 -) SELECT - t1.`string_col`, - t1.`metric` + t6.string_col, + t6.metric FROM ( - WITH t0 AS ( + SELECT + t4.string_col, + t4.metric + FROM ( SELECT - t2.`string_col`, - sum(t2.`double_col`) AS `metric` - FROM functional_alltypes AS t2 + t0.string_col, + SUM(t0.double_col) AS metric + FROM functional_alltypes AS t0 GROUP BY 1 - ), t2 AS ( + UNION DISTINCT SELECT - t3.`string_col`, - t3.`metric` - FROM ( - WITH t0 AS ( - SELECT - t2.`string_col`, - sum(t2.`double_col`) AS `metric` - FROM functional_alltypes AS t2 - GROUP BY - 1 - ) - SELECT - * - FROM t0 - UNION DISTINCT - SELECT - t4.`string_col`, - sum(t4.`double_col`) AS `metric` - FROM functional_alltypes AS t4 - GROUP BY - 1 - ) AS t3 - ) - SELECT - * - FROM t2 + t0.string_col, + SUM(t0.double_col) AS metric + FROM functional_alltypes AS t0 + GROUP BY + 1 + ) AS t4 UNION ALL SELECT - t4.`string_col`, - sum(t4.`double_col`) AS `metric` - FROM functional_alltypes AS t4 + t0.string_col, + SUM(t0.double_col) AS metric + FROM functional_alltypes AS t0 GROUP BY 1 -) AS t1 \ No newline at end of file +) AS t6 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/True-False/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/True-False/out.sql index 5ed562f29914..36dde7359805 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/True-False/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/True-False/out.sql @@ -1,55 +1,30 @@ -WITH t0 AS ( - SELECT - t2.`string_col`, - sum(t2.`double_col`) AS `metric` - FROM functional_alltypes AS t2 - GROUP BY - 1 -) SELECT - t1.`string_col`, - t1.`metric` + t6.string_col, + t6.metric FROM ( - WITH t0 AS ( + SELECT + t4.string_col, + t4.metric + FROM ( SELECT - t2.`string_col`, - sum(t2.`double_col`) AS `metric` - FROM functional_alltypes AS t2 + t0.string_col, + SUM(t0.double_col) AS metric + FROM functional_alltypes AS t0 GROUP BY 1 - ), t2 AS ( + UNION ALL SELECT - t3.`string_col`, - t3.`metric` - FROM ( - WITH t0 AS ( - SELECT - t2.`string_col`, - sum(t2.`double_col`) AS `metric` - FROM functional_alltypes AS t2 - GROUP BY - 1 - ) - SELECT - * - FROM t0 - UNION ALL - SELECT - t4.`string_col`, - sum(t4.`double_col`) AS `metric` - FROM functional_alltypes AS t4 - GROUP BY - 1 - ) AS t3 - ) - SELECT - * - FROM t2 + t0.string_col, + SUM(t0.double_col) AS metric + FROM functional_alltypes AS t0 + GROUP BY + 1 + ) AS t4 UNION DISTINCT SELECT - t4.`string_col`, - sum(t4.`double_col`) AS `metric` - FROM functional_alltypes AS t4 + t0.string_col, + SUM(t0.double_col) AS metric + FROM functional_alltypes AS t0 GROUP BY 1 -) AS t1 \ No newline at end of file +) AS t6 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/True-True/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/True-True/out.sql index bc9dc55839a3..1ba202a0f834 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/True-True/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_union_cte/True-True/out.sql @@ -1,55 +1,30 @@ -WITH t0 AS ( - SELECT - t2.`string_col`, - sum(t2.`double_col`) AS `metric` - FROM functional_alltypes AS t2 - GROUP BY - 1 -) SELECT - t1.`string_col`, - t1.`metric` + t6.string_col, + t6.metric FROM ( - WITH t0 AS ( + SELECT + t4.string_col, + t4.metric + FROM ( SELECT - t2.`string_col`, - sum(t2.`double_col`) AS `metric` - FROM functional_alltypes AS t2 + t0.string_col, + SUM(t0.double_col) AS metric + FROM functional_alltypes AS t0 GROUP BY 1 - ), t2 AS ( + UNION DISTINCT SELECT - t3.`string_col`, - t3.`metric` - FROM ( - WITH t0 AS ( - SELECT - t2.`string_col`, - sum(t2.`double_col`) AS `metric` - FROM functional_alltypes AS t2 - GROUP BY - 1 - ) - SELECT - * - FROM t0 - UNION DISTINCT - SELECT - t4.`string_col`, - sum(t4.`double_col`) AS `metric` - FROM functional_alltypes AS t4 - GROUP BY - 1 - ) AS t3 - ) - SELECT - * - FROM t2 + t0.string_col, + SUM(t0.double_col) AS metric + FROM functional_alltypes AS t0 + GROUP BY + 1 + ) AS t4 UNION DISTINCT SELECT - t4.`string_col`, - sum(t4.`double_col`) AS `metric` - FROM functional_alltypes AS t4 + t0.string_col, + SUM(t0.double_col) AS metric + FROM functional_alltypes AS t0 GROUP BY 1 -) AS t1 \ No newline at end of file +) AS t6 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_unnest/out_one_unnest.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_unnest/out_one_unnest.sql index 702add1dcd83..1efaf2f26d3a 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_unnest/out_one_unnest.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_unnest/out_one_unnest.sql @@ -1,16 +1,16 @@ SELECT - t0.`rowindex`, - IF(pos = pos_2, `repeated_struct_col`, NULL) AS `repeated_struct_col` + t0.rowindex, + IF(pos = pos_2, repeated_struct_col, NULL) AS repeated_struct_col FROM array_test AS t0 -CROSS JOIN UNNEST(GENERATE_ARRAY(0, GREATEST(ARRAY_LENGTH(t0.`repeated_struct_col`)) - 1)) AS pos -CROSS JOIN UNNEST(t0.`repeated_struct_col`) AS `repeated_struct_col` WITH OFFSET AS pos_2 +CROSS JOIN UNNEST(GENERATE_ARRAY(0, GREATEST(ARRAY_LENGTH(t0.repeated_struct_col)) - 1)) AS pos +CROSS JOIN UNNEST(t0.repeated_struct_col) AS repeated_struct_col WITH OFFSET AS pos_2 WHERE pos = pos_2 OR ( pos > ( - ARRAY_LENGTH(t0.`repeated_struct_col`) - 1 + ARRAY_LENGTH(t0.repeated_struct_col) - 1 ) AND pos_2 = ( - ARRAY_LENGTH(t0.`repeated_struct_col`) - 1 + ARRAY_LENGTH(t0.repeated_struct_col) - 1 ) ) \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_unnest/out_two_unnests.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_unnest/out_two_unnests.sql index 57ec0a5f4378..febc60d6c4df 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_unnest/out_two_unnests.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_unnest/out_two_unnests.sql @@ -1,32 +1,32 @@ SELECT - IF(pos = pos_2, `level_two`, NULL) AS `level_two` + IF(pos = pos_2, level_two, NULL) AS level_two FROM ( SELECT - t1.`rowindex`, - IF(pos = pos_2, `level_one`, NULL).`nested_struct_col` AS `level_one` - FROM array_test AS t1 - CROSS JOIN UNNEST(GENERATE_ARRAY(0, GREATEST(ARRAY_LENGTH(t1.`repeated_struct_col`)) - 1)) AS pos - CROSS JOIN UNNEST(t1.`repeated_struct_col`) AS `level_one` WITH OFFSET AS pos_2 + t0.rowindex, + IF(pos = pos_2, level_one, NULL).nested_struct_col AS level_one + FROM array_test AS t0 + CROSS JOIN UNNEST(GENERATE_ARRAY(0, GREATEST(ARRAY_LENGTH(t0.repeated_struct_col)) - 1)) AS pos + CROSS JOIN UNNEST(t0.repeated_struct_col) AS level_one WITH OFFSET AS pos_2 WHERE pos = pos_2 OR ( pos > ( - ARRAY_LENGTH(t1.`repeated_struct_col`) - 1 + ARRAY_LENGTH(t0.repeated_struct_col) - 1 ) AND pos_2 = ( - ARRAY_LENGTH(t1.`repeated_struct_col`) - 1 + ARRAY_LENGTH(t0.repeated_struct_col) - 1 ) ) -) AS t0 -CROSS JOIN UNNEST(GENERATE_ARRAY(0, GREATEST(ARRAY_LENGTH(t0.`level_one`)) - 1)) AS pos -CROSS JOIN UNNEST(t0.`level_one`) AS `level_two` WITH OFFSET AS pos_2 +) AS t1 +CROSS JOIN UNNEST(GENERATE_ARRAY(0, GREATEST(ARRAY_LENGTH(t1.level_one)) - 1)) AS pos +CROSS JOIN UNNEST(t1.level_one) AS level_two WITH OFFSET AS pos_2 WHERE pos = pos_2 OR ( pos > ( - ARRAY_LENGTH(t0.`level_one`) - 1 + ARRAY_LENGTH(t1.level_one) - 1 ) AND pos_2 = ( - ARRAY_LENGTH(t0.`level_one`) - 1 + ARRAY_LENGTH(t1.level_one) - 1 ) ) \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_function/current_foll/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_function/current_foll/out.sql index 48fa0b1e700d..f7af87cc3889 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_function/current_foll/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_function/current_foll/out.sql @@ -1,4 +1,16 @@ SELECT - t0.*, - avg(t0.`float_col`) OVER (PARTITION BY t0.`year` ORDER BY t0.`timestamp_col` ASC ROWS BETWEEN CURRENT ROW AND 2 FOLLOWING) AS `win_avg` + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month, + AVG(t0.float_col) OVER (PARTITION BY t0.year ORDER BY t0.timestamp_col ASC ROWS BETWEEN CURRENT ROW AND 2 following) AS win_avg FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_function/prec_current/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_function/prec_current/out.sql index c7a654e6c4af..812d5c8e17fc 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_function/prec_current/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_function/prec_current/out.sql @@ -1,4 +1,16 @@ SELECT - t0.*, - avg(t0.`float_col`) OVER (PARTITION BY t0.`year` ORDER BY t0.`timestamp_col` ASC ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) AS `win_avg` + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month, + AVG(t0.float_col) OVER (PARTITION BY t0.year ORDER BY t0.timestamp_col ASC ROWS BETWEEN 1 preceding AND CURRENT ROW) AS win_avg FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_function/prec_prec/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_function/prec_prec/out.sql index 8b6db281ec69..dc3996ac2b7d 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_function/prec_prec/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_function/prec_prec/out.sql @@ -1,4 +1,16 @@ SELECT - t0.*, - avg(t0.`float_col`) OVER (PARTITION BY t0.`year` ORDER BY t0.`timestamp_col` ASC ROWS BETWEEN 4 PRECEDING AND 2 PRECEDING) AS `win_avg` + t0.id, + t0.bool_col, + t0.tinyint_col, + t0.smallint_col, + t0.int_col, + t0.bigint_col, + t0.float_col, + t0.double_col, + t0.date_string_col, + t0.string_col, + t0.timestamp_col, + t0.year, + t0.month, + AVG(t0.float_col) OVER (PARTITION BY t0.year ORDER BY t0.timestamp_col ASC ROWS BETWEEN 4 preceding AND 2 preceding) AS win_avg FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_unbounded/following/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_unbounded/following/out.sql index 0a9989863d4f..4efa722dccbc 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_unbounded/following/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_unbounded/following/out.sql @@ -1,3 +1,3 @@ SELECT - sum(t0.`a`) OVER (ROWS BETWEEN 1 FOLLOWING AND UNBOUNDED FOLLOWING) AS `tmp` + SUM(t0.a) OVER (ROWS BETWEEN 1 following AND UNBOUNDED FOLLOWING) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_unbounded/preceding/out.sql b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_unbounded/preceding/out.sql index cfdb5364e102..fe5c57096cc2 100644 --- a/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_unbounded/preceding/out.sql +++ b/ibis/backends/bigquery/tests/unit/snapshots/test_compiler/test_window_unbounded/preceding/out.sql @@ -1,3 +1,3 @@ SELECT - sum(t0.`a`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND 1 PRECEDING) AS `tmp` + SUM(t0.a) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND 1 preceding) AS tmp FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/test_compiler.py b/ibis/backends/bigquery/tests/unit/test_compiler.py index fafeddbdaec2..0d95438df2c6 100644 --- a/ibis/backends/bigquery/tests/unit/test_compiler.py +++ b/ibis/backends/bigquery/tests/unit/test_compiler.py @@ -1,7 +1,6 @@ from __future__ import annotations import datetime -import re import time from operator import floordiv, methodcaller, truediv @@ -10,6 +9,7 @@ from pytest import param import ibis +import ibis.common.exceptions as com import ibis.expr.datatypes as dt import ibis.expr.operations as ops from ibis import _ @@ -233,7 +233,7 @@ def test_substring_neg_length(): t = ibis.table([("value", "string")], name="t") expr = t["value"].substr(3, -1).name("tmp") with pytest.raises( - Exception, match=r"Length parameter must be a non-negative value\." + Exception, match=r"Length parameter must be a non-negative value; got -1" ): to_sql(expr) @@ -387,10 +387,10 @@ def test_geospatial_simplify(snapshot): def test_geospatial_simplify_error(): t = ibis.table([("geog", "geography")], name="t") expr = t.geog.simplify(5.2, preserve_collapsed=True).name("tmp") - with pytest.raises(Exception) as exception_info: + with pytest.raises( + Exception, match="simplify does not support preserving collapsed geometries" + ): to_sql(expr) - expected = "BigQuery simplify does not support preserving collapsed geometries, must pass preserve_collapsed=False" - assert str(exception_info.value) == expected def test_timestamp_accepts_date_literals(alltypes): @@ -399,7 +399,7 @@ def test_timestamp_accepts_date_literals(alltypes): expr = alltypes.mutate(param=p) params = {p: date_string} result = to_sql(expr, params=params) - assert re.search(r"@param_\d+ AS `param`", result) is not None + assert "2009-03-01T00:00:00" in result @pytest.mark.parametrize("distinct", [True, False]) @@ -483,14 +483,18 @@ def test_range_window_function(alltypes, window, snapshot): "preceding", [ param(5, id="five"), - param(ibis.interval(nanoseconds=1), id="nanos", marks=pytest.mark.xfail), + param( + ibis.interval(nanoseconds=1), + id="nanos", + marks=pytest.mark.xfail(raises=com.UnsupportedOperationError), + ), param(ibis.interval(microseconds=1), id="micros"), param(ibis.interval(seconds=1), id="seconds"), param(ibis.interval(minutes=1), id="minutes"), param(ibis.interval(hours=1), id="hours"), param(ibis.interval(days=1), id="days"), param(2 * ibis.interval(days=1), id="two_days"), - param(ibis.interval(weeks=1), id="week", marks=pytest.mark.xfail), + param(ibis.interval(weeks=1), id="week"), ], ) def test_trailing_range_window(alltypes, preceding, snapshot): @@ -584,7 +588,7 @@ def test_scalar_param_scope(alltypes): t = alltypes param = ibis.param("timestamp") result = to_sql(t.mutate(param=param), params={param: "2017-01-01"}) - assert re.search(r"@param_\d+ AS `param`", result) is not None + assert "2017-01-01T00:00:00" in result def test_cast_float_to_int(alltypes, snapshot): diff --git a/ibis/backends/bigquery/tests/unit/udf/snapshots/test_builtin/test_bqutil_fn_from_hex/out.sql b/ibis/backends/bigquery/tests/unit/udf/snapshots/test_builtin/test_bqutil_fn_from_hex/out.sql index 2cabc41aa447..3308b79b3ad0 100644 --- a/ibis/backends/bigquery/tests/unit/udf/snapshots/test_builtin/test_bqutil_fn_from_hex/out.sql +++ b/ibis/backends/bigquery/tests/unit/udf/snapshots/test_builtin/test_bqutil_fn_from_hex/out.sql @@ -1,2 +1,2 @@ SELECT - `bqutil`.`fn`.from_hex('face') AS `from_hex_'face'` \ No newline at end of file + bqutil.fn.from_hex('face') AS `from_hex_0_'face'` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/udf/snapshots/test_builtin/test_farm_fingerprint/out.sql b/ibis/backends/bigquery/tests/unit/udf/snapshots/test_builtin/test_farm_fingerprint/out.sql index 9128e636a8a6..49c49900b198 100644 --- a/ibis/backends/bigquery/tests/unit/udf/snapshots/test_builtin/test_farm_fingerprint/out.sql +++ b/ibis/backends/bigquery/tests/unit/udf/snapshots/test_builtin/test_farm_fingerprint/out.sql @@ -1,2 +1,2 @@ SELECT - farm_fingerprint(b'Hello, World!') AS `farm_fingerprint_b'Hello_ World_'` \ No newline at end of file + farm_fingerprint(CAST('48656c6c6f2c20576f726c6421' AS BYTES FORMAT 'HEX')) AS `farm_fingerprint_0_b'Hello_ World_'` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_multiple_calls_redefinition/out.sql b/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_multiple_calls_redefinition/out.sql deleted file mode 100644 index bca06e837674..000000000000 --- a/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_multiple_calls_redefinition/out.sql +++ /dev/null @@ -1,18 +0,0 @@ -CREATE TEMPORARY FUNCTION my_len_0( - s STRING -) -RETURNS FLOAT64 -LANGUAGE js AS -'\n\'use strict\';\nfunction my_len(s) {\n return s.length;\n}\nreturn my_len(s);\n'; - -CREATE TEMPORARY FUNCTION my_len_1( - s STRING -) -RETURNS FLOAT64 -LANGUAGE js AS -'\n\'use strict\';\nfunction my_len(s) {\n return (s.length + 1);\n}\nreturn my_len(s);\n'; - -SELECT - ( - my_len_0('abcd') + my_len_0('abcd') - ) + my_len_1('abcd') AS `Add_Add_my_len_0_'abcd'_ my_len_0_'abcd'_ my_len_1_'abcd'` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_udf_determinism/False/out.sql b/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_udf_determinism/False/out.sql deleted file mode 100644 index 4b7713dd8904..000000000000 --- a/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_udf_determinism/False/out.sql +++ /dev/null @@ -1,10 +0,0 @@ -CREATE TEMPORARY FUNCTION my_len_0( - s STRING -) -RETURNS FLOAT64 -NOT DETERMINISTIC -LANGUAGE js AS -'\n\'use strict\';\nfunction my_len(s) {\n return s.length;\n}\nreturn my_len(s);\n'; - -SELECT - my_len_0('abcd') AS `my_len_0_'abcd'` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_udf_determinism/None/out.sql b/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_udf_determinism/None/out.sql deleted file mode 100644 index 0c86cd6c558b..000000000000 --- a/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_udf_determinism/None/out.sql +++ /dev/null @@ -1,9 +0,0 @@ -CREATE TEMPORARY FUNCTION my_len_0( - s STRING -) -RETURNS FLOAT64 -LANGUAGE js AS -'\n\'use strict\';\nfunction my_len(s) {\n return s.length;\n}\nreturn my_len(s);\n'; - -SELECT - my_len_0('abcd') AS `my_len_0_'abcd'` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_udf_determinism/True/out.sql b/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_udf_determinism/True/out.sql deleted file mode 100644 index 0ace954587a6..000000000000 --- a/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_udf_determinism/True/out.sql +++ /dev/null @@ -1,10 +0,0 @@ -CREATE TEMPORARY FUNCTION my_len_0( - s STRING -) -RETURNS FLOAT64 -DETERMINISTIC -LANGUAGE js AS -'\n\'use strict\';\nfunction my_len(s) {\n return s.length;\n}\nreturn my_len(s);\n'; - -SELECT - my_len_0('abcd') AS `my_len_0_'abcd'` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_udf_sql/out.sql b/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_udf_sql/out.sql deleted file mode 100644 index 70812f001fb9..000000000000 --- a/ibis/backends/bigquery/tests/unit/udf/snapshots/test_usage/test_udf_sql/out.sql +++ /dev/null @@ -1,10 +0,0 @@ -CREATE TEMPORARY FUNCTION format_t_0( - input STRING -) -RETURNS FLOAT64 AS -( - FORMAT('%T', input) -); - -SELECT - format_t_0('abcd') AS `format_t_0_'abcd'` \ No newline at end of file diff --git a/ibis/backends/bigquery/tests/unit/udf/test_builtin.py b/ibis/backends/bigquery/tests/unit/udf/test_builtin.py index 2d877eba9f4e..14d2373a45a2 100644 --- a/ibis/backends/bigquery/tests/unit/udf/test_builtin.py +++ b/ibis/backends/bigquery/tests/unit/udf/test_builtin.py @@ -10,7 +10,7 @@ def farm_fingerprint(value: bytes) -> int: ... -@ibis.udf.scalar.builtin(schema="bqutil.fn") +@ibis.udf.scalar.builtin(schema="fn", database="bqutil") def from_hex(value: str) -> int: """Community function to convert from hex string to integer. diff --git a/ibis/backends/bigquery/tests/unit/udf/test_usage.py b/ibis/backends/bigquery/tests/unit/udf/test_usage.py index ae0135187cfb..4f8dd4218d36 100644 --- a/ibis/backends/bigquery/tests/unit/udf/test_usage.py +++ b/ibis/backends/bigquery/tests/unit/udf/test_usage.py @@ -1,120 +1,74 @@ from __future__ import annotations +import re + import pytest -from pytest import param import ibis +import ibis.common.exceptions as com import ibis.expr.datatypes as dt -from ibis.backends.bigquery import udf -from ibis.backends.bigquery.udf import _udf_name_cache - +from ibis import udf -def test_multiple_calls_redefinition(snapshot): - _udf_name_cache.clear() - @udf.python([dt.string], dt.double) - def my_len(s): +def test_multiple_calls_redefinition(): + @udf.scalar.python + def my_len(s: str) -> float: return s.length s = ibis.literal("abcd") expr = my_len(s) + my_len(s) - @udf.python([dt.string], dt.double) - def my_len(s): + @udf.scalar.python + def my_len(s: str) -> float: return s.length + 1 expr = expr + my_len(s) sql = ibis.bigquery.compile(expr) - snapshot.assert_match(sql, "out.sql") + assert len(set(re.findall(r"my_len_(\d+)", sql))) == 2 -@pytest.mark.parametrize( - ("determinism",), - [ - param(True), - param(False), - param(None), - ], -) -def test_udf_determinism(snapshot, determinism): - _udf_name_cache.clear() - - @udf.python([dt.string], dt.double, determinism=determinism) - def my_len(s): +@pytest.mark.parametrize("determinism", [True, False, None]) +def test_udf_determinism(determinism): + @udf.scalar.python(determinism=determinism) + def my_len(s: str) -> float: return s.length s = ibis.literal("abcd") expr = my_len(s) sql = ibis.bigquery.compile(expr) - snapshot.assert_match(sql, "out.sql") - -def test_udf_sql(snapshot): - _udf_name_cache.clear() - - format_t = udf.sql( - "format_t", - params={"input": dt.string}, - output_type=dt.double, - sql_expression="FORMAT('%T', input)", - ) - - s = ibis.literal("abcd") - expr = format_t(s) - - sql = ibis.bigquery.compile(expr) - snapshot.assert_match(sql, "out.sql") + if not determinism: + assert "NOT DETERMINISTIC" in sql + else: + assert "DETERMINISTIC" in sql and "NOT DETERMINISTIC" not in sql @pytest.mark.parametrize( ("argument_type", "return_type"), [ - param( - dt.int64, - dt.float64, - marks=pytest.mark.xfail(raises=TypeError), - id="int_float", - ), - param( - dt.float64, - dt.int64, - marks=pytest.mark.xfail(raises=TypeError), - id="float_int", - ), + # invalid input type + (dt.int64, dt.float64), + # invalid return type + (dt.float64, dt.int64), # complex argument type, valid return type - param( - dt.Array(dt.int64), - dt.float64, - marks=pytest.mark.xfail(raises=TypeError), - id="array_int_float", - ), + (dt.Array(dt.int64), dt.float64), # valid argument type, complex invalid return type - param( - dt.float64, - dt.Array(dt.int64), - marks=pytest.mark.xfail(raises=TypeError), - id="float_array_int", - ), + (dt.float64, dt.Array(dt.int64)), # both invalid - param( - dt.Array(dt.Array(dt.int64)), - dt.int64, - marks=pytest.mark.xfail(raises=TypeError), - id="array_array_int_int", - ), + (dt.Array(dt.Array(dt.int64)), dt.int64), # struct type with nested integer, valid return type - param( - dt.Struct.from_tuples([("x", dt.Array(dt.int64))]), - dt.float64, - marks=pytest.mark.xfail(raises=TypeError), - id="struct", - ), + (dt.Struct({"x": dt.Array(dt.int64)}), dt.float64), ], + ids=str, ) def test_udf_int64(argument_type, return_type): # invalid argument type, valid return type - @udf.python([argument_type], return_type) - def my_int64_add(x): - return 1.0 + @udf.scalar.python(signature=((argument_type,), return_type)) + def my_func(x): + return 1 + + expr = my_func(None) + with pytest.raises(com.UnsupportedBackendType): + ibis.bigquery.compile(expr) diff --git a/ibis/backends/bigquery/udf/__init__.py b/ibis/backends/bigquery/udf/__init__.py index 0e84b972c438..e69de29bb2d1 100644 --- a/ibis/backends/bigquery/udf/__init__.py +++ b/ibis/backends/bigquery/udf/__init__.py @@ -1,396 +0,0 @@ -from __future__ import annotations - -import collections -import inspect -import itertools -from typing import TYPE_CHECKING, Callable, Literal - -import ibis.expr.datatypes as dt -import ibis.expr.rules as rlz -from ibis.backends.bigquery.datatypes import BigQueryType, spread_type -from ibis.backends.bigquery.operations import BigQueryUDFNode -from ibis.backends.bigquery.udf.core import PythonToJavaScriptTranslator -from ibis.legacy.udf.validate import validate_output_type - -if TYPE_CHECKING: - from collections.abc import Iterable, Mapping - -__all__ = ("udf",) - -_udf_name_cache: dict[str, Iterable[int]] = collections.defaultdict(itertools.count) - - -def _make_udf_name(name): - definition = next(_udf_name_cache[name]) - external_name = f"{name}_{definition:d}" - return external_name - - -class _BigQueryUDF: - def __call__(self, *args, **kwargs): - return self.python(*args, **kwargs) - - def python( - self, - input_type: Iterable[dt.DataType], - output_type: dt.DataType, - strict: bool = True, - libraries: Iterable[str] | None = None, - determinism: bool | None = None, - ) -> Callable: - '''Define a UDF for BigQuery. - - The function is transpiled to JS. - - `INT64` is not supported as an argument type or a return type, as per - [the BigQuery documentation](https://cloud.google.com/bigquery/docs/reference/standard-sql/user-defined-functions#sql-type-encodings-in-javascript). - - Parameters - ---------- - input_type - Iterable of types, one per argument. - output_type - Return type of the UDF. - strict - Whether or not to put a ``'use strict';`` string at the beginning of - the UDF. Setting to ``False`` is probably a bad idea. - libraries - An iterable of Google Cloud Storage URIs containing to JavaScript source - code. Note that any symbols (functions, classes, variables, etc.) that - are exposed in these JavaScript files will be visible inside the UDF. - determinism - Provides a hint to BigQuery as to whether the query result can be cached. - - Returns - ------- - Callable - The wrapped user-defined function. - - Examples - -------- - >>> from ibis.backends.bigquery import udf - >>> import ibis.expr.datatypes as dt - >>> @udf.python(input_type=[dt.double], output_type=dt.double) - ... def add_one(x): - ... return x + 1 - >>> print(add_one.sql) - CREATE TEMPORARY FUNCTION add_one_0(x FLOAT64) - RETURNS FLOAT64 - LANGUAGE js AS """ - 'use strict'; - function add_one(x) { - return (x + 1); - } - return add_one(x); - """; - >>> @udf.python(input_type=[dt.double, dt.double], output_type=dt.Array(dt.double)) - ... def my_range(start, stop): - ... def gen(start, stop): - ... curr = start - ... while curr < stop: - ... yield curr - ... curr += 1 - ... - ... result = [] - ... for value in gen(start, stop): - ... result.append(value) - ... return result - >>> print(my_range.sql) - CREATE TEMPORARY FUNCTION my_range_0(start FLOAT64, stop FLOAT64) - RETURNS ARRAY - LANGUAGE js AS """ - 'use strict'; - function my_range(start, stop) { - function* gen(start, stop) { - let curr = start; - while ((curr < stop)) { - yield curr; - curr += 1; - } - } - let result = []; - for (let value of gen(start, stop)) { - result.push(value); - } - return result; - } - return my_range(start, stop); - """; - >>> @udf.python( - ... input_type=[dt.double, dt.double], - ... output_type=dt.Struct.from_tuples([("width", "double"), ("height", "double")]), - ... ) - ... def my_rectangle(width, height): - ... class Rectangle: - ... def __init__(self, width, height): - ... self.width = width - ... self.height = height - ... - ... @property - ... def area(self): - ... return self.width * self.height - ... - ... def perimeter(self): - ... return 2 * (self.width + self.height) - ... - ... return Rectangle(width, height) - >>> print(my_rectangle.sql) - CREATE TEMPORARY FUNCTION my_rectangle_0(width FLOAT64, height FLOAT64) - RETURNS STRUCT - LANGUAGE js AS """ - 'use strict'; - function my_rectangle(width, height) { - class Rectangle { - constructor(width, height) { - this.width = width; - this.height = height; - } - get area() { - return (this.width * this.height); - } - perimeter() { - return (2 * (this.width + this.height)); - } - } - return (new Rectangle(width, height)); - } - return my_rectangle(width, height); - """; - ''' - validate_output_type(output_type) - - if libraries is None: - libraries = [] - - def wrapper(f): - if not callable(f): - raise TypeError(f"f must be callable, got {f}") - - signature = inspect.signature(f) - parameter_names = signature.parameters.keys() - source = PythonToJavaScriptTranslator(f).compile() - args = ", ".join(parameter_names) - strict_str = repr("use strict") + ";\n" if strict else "" - function_body = f"""\ -{strict_str}{source} -return {f.__name__}({args});\ -""" - - return self.js( - name=f.__name__, - params=(dict(zip(parameter_names, input_type))), - output_type=output_type, - body=function_body, - libraries=libraries, - determinism=determinism, - ) - - return wrapper - - @staticmethod - def js( - name: str, - params: Mapping[str, dt.DataType], - output_type: dt.DataType, - body: str, - libraries: Iterable[str] | None = None, - determinism: bool | None = None, - ) -> Callable: - '''Define a Javascript UDF for BigQuery. - - `INT64` is not supported as an argument type or a return type, as per - [the BigQuery documentation](https://cloud.google.com/bigquery/docs/reference/standard-sql/user-defined-functions#sql-type-encodings-in-javascript). - - Parameters - ---------- - name: - The name of the function. - params - Mapping of names and types of parameters - output_type - Return type of the UDF. - body: - The code of the function. - libraries - An iterable of Google Cloud Storage URIs containing to JavaScript source - code. Note that any symbols (functions, classes, variables, etc.) that - are exposed in these JavaScript files will be visible inside the UDF. - determinism - Provides a hint to BigQuery as to whether the query result can be cached. - - Returns - ------- - Callable - The user-defined function. - - Examples - -------- - >>> from ibis.backends.bigquery import udf - >>> import ibis.expr.datatypes as dt - >>> add_one = udf.js( - ... name="add_one", - ... params={"a": dt.double}, - ... output_type=dt.double, - ... body="return x + 1", - ... ) - >>> print(add_one.sql) - CREATE TEMPORARY FUNCTION add_one_0(x FLOAT64) - RETURNS FLOAT64 - LANGUAGE js AS """ - return x + 1 - """; - ''' - validate_output_type(output_type) - if any( - type_ == dt.int64 - for param_type in params.values() - for type_ in spread_type(param_type) - ) or any(type_ == dt.int64 for type_ in spread_type(output_type)): - raise TypeError( - "BigQuery does not support INT64 as an argument type or a return type " - "for UDFs. Replace INT64 with FLOAT64 in your UDF signature and " - "cast all INT64 inputs to FLOAT64." - ) - - if libraries is None: - libraries = [] - - bigquery_signature = ", ".join( - f"{name} {BigQueryType.to_string(dt.dtype(type_))}" - for name, type_ in params.items() - ) - return_type = BigQueryType.to_string(dt.dtype(output_type)) - libraries_opts = ( - f"\nOPTIONS (\n library={list(libraries)!r}\n)" if libraries else "" - ) - determinism_formatted = { - True: "DETERMINISTIC\n", - False: "NOT DETERMINISTIC\n", - None: "", - }.get(determinism) - - name = _make_udf_name(name) - sql_code = f'''\ -CREATE TEMPORARY FUNCTION {name}({bigquery_signature}) -RETURNS {return_type} -{determinism_formatted}LANGUAGE js AS """ -{body} -"""{libraries_opts};''' - - udf_node_fields = { - name: rlz.ValueOf(None if type_ == "ANY TYPE" else type_) - for name, type_ in params.items() - } - - udf_node_fields["dtype"] = output_type - udf_node_fields["shape"] = rlz.shape_like("args") - udf_node_fields["sql"] = sql_code - - udf_node = type(name, (BigQueryUDFNode,), udf_node_fields) - - from ibis.backends.bigquery.compiler import compiles - - @compiles(udf_node) - def compiles_udf_node(t, op): - args = ", ".join(map(t.translate, op.args)) - return f"{udf_node.__name__}({args})" - - def wrapped(*args, **kwargs): - node = udf_node(*args, **kwargs) - return node.to_expr() - - wrapped.__signature__ = inspect.Signature( - parameters=[ - inspect.Parameter( - name=param, kind=inspect.Parameter.POSITIONAL_OR_KEYWORD - ) - for param in params.keys() - ] - ) - wrapped.__name__ = name - wrapped.sql = sql_code - return wrapped - - @staticmethod - def sql( - name: str, - params: Mapping[str, dt.DataType | Literal["ANY TYPE"]], - output_type: dt.DataType, - sql_expression: str, - ) -> Callable: - """Define a SQL UDF for BigQuery. - - Parameters - ---------- - name: - The name of the function. - params - Mapping of names and types of parameters - output_type - Return type of the UDF. - sql_expression - The SQL expression that defines the function. - - Returns - ------- - Callable - The wrapped user-defined function. - - Examples - -------- - >>> from ibis.backends.bigquery import udf - >>> import ibis.expr.datatypes as dt - >>> add_one = udf.sql( - ... name="add_one", - ... params={"x": dt.double}, - ... output_type=dt.double, - ... sql_expression="x + 1", - ... ) - >>> print(add_one.sql) - CREATE TEMPORARY FUNCTION add_one_0(x FLOAT64) - RETURNS FLOAT64 - AS (x + 1) - """ - validate_output_type(output_type) - udf_node_fields = { - name: rlz.ValueOf(None if type_ == "ANY TYPE" else type_) - for name, type_ in params.items() - } - return_type = BigQueryType.to_string(dt.dtype(output_type)) - - bigquery_signature = ", ".join( - "{name} {type}".format( - name=name, - type="ANY TYPE" - if type_ == "ANY TYPE" - else BigQueryType.to_string(dt.dtype(type_)), - ) - for name, type_ in params.items() - ) - name = _make_udf_name(name) - sql_code = f"""\ -CREATE TEMPORARY FUNCTION {name}({bigquery_signature}) -RETURNS {return_type} -AS ({sql_expression});""" - - udf_node_fields["dtype"] = output_type - udf_node_fields["shape"] = rlz.shape_like("args") - udf_node_fields["sql"] = sql_code - - udf_node = type(name, (BigQueryUDFNode,), udf_node_fields) - - from ibis.backends.bigquery.compiler import compiles - - @compiles(udf_node) - def compiles_udf_node(t, op): - args = ", ".join(map(t.translate, op.args)) - return f"{udf_node.__name__}({args})" - - def wrapper(*args, **kwargs): - node = udf_node(*args, **kwargs) - return node.to_expr() - - return wrapper - - -udf = _BigQueryUDF() diff --git a/ibis/backends/bigquery/udf/core.py b/ibis/backends/bigquery/udf/core.py index 00a4bf3c6235..58351841e7f1 100644 --- a/ibis/backends/bigquery/udf/core.py +++ b/ibis/backends/bigquery/udf/core.py @@ -10,7 +10,6 @@ from collections import ChainMap from typing import Callable -import ibis.expr.datatypes as dt from ibis.backends.bigquery.udf.find import find_names from ibis.backends.bigquery.udf.rewrite import rewrite @@ -514,14 +513,11 @@ def visit_Delete(self, node): if __name__ == "__main__": - from ibis.backends.bigquery.udf import udf + import ibis + from ibis import udf - @udf( - input_type=[dt.double, dt.double, dt.int64], - output_type=dt.Array(dt.double), - strict=False, - ) - def my_func(a, b, n): + @udf.scalar.python(strict=False) + def my_func(a: float, b: float, n: float) -> list[float]: class Rectangle: def __init__(self, width, height): self.width = width @@ -598,4 +594,4 @@ def range(n): nnn = len(values) return [sum(values) - a + b * y**-x, z, foo.width, nnn] - print(my_func.sql) # noqa: T201 + print(ibis.bigquery.compile(my_func(42.7, 13.2, 1))) # noqa: T201 diff --git a/ibis/backends/duckdb/__init__.py b/ibis/backends/duckdb/__init__.py index d7111410cb28..f13b8b8127cd 100644 --- a/ibis/backends/duckdb/__init__.py +++ b/ibis/backends/duckdb/__init__.py @@ -1434,12 +1434,13 @@ def _register_udfs(self, expr: ir.Expr) -> None: def _compile_udf(self, udf_node: ops.ScalarUDF) -> None: func = udf_node.__func__ - name = func.__name__ + name = type(udf_node).__name__ + type_mapper = self.compiler.type_mapper input_types = [ - self.compiler.type_mapper.to_string(param.annotation.pattern.dtype) + type_mapper.to_string(param.annotation.pattern.dtype) for param in udf_node.__signature__.parameters.values() ] - output_type = self.compiler.type_mapper.to_string(udf_node.dtype) + output_type = type_mapper.to_string(udf_node.dtype) def register_udf(con): return con.create_function( diff --git a/ibis/backends/impala/tests/snapshots/test_udf/test_sql_generation/out.sql b/ibis/backends/impala/tests/snapshots/test_udf/test_sql_generation/out.sql index 6ae4e58e0380..1aa828dc1fc5 100644 --- a/ibis/backends/impala/tests/snapshots/test_udf/test_sql_generation/out.sql +++ b/ibis/backends/impala/tests/snapshots/test_udf/test_sql_generation/out.sql @@ -1,2 +1,2 @@ SELECT - UDF_TESTING.IDENTITY('hello world') AS `identity('hello world')` \ No newline at end of file + UDF_TESTING.IDENTITY('hello world') AS `identity_0('hello world')` \ No newline at end of file diff --git a/ibis/backends/postgres/__init__.py b/ibis/backends/postgres/__init__.py index 7e25f97c9432..f177d38bf126 100644 --- a/ibis/backends/postgres/__init__.py +++ b/ibis/backends/postgres/__init__.py @@ -433,7 +433,7 @@ def _get_udf_source(self, udf_node: ops.ScalarUDF): type_mapper = self.compiler.type_mapper argnames = udf_node.argnames return dict( - name=udf_node.__func_name__, + name=type(udf_node).__name__, ident=self.compiler.__sql_name__(udf_node), signature=", ".join( f"{argname} {type_mapper.to_string(arg.dtype)}" diff --git a/ibis/backends/tests/snapshots/test_interactive/test_default_limit/bigquery/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_default_limit/bigquery/out.sql new file mode 100644 index 000000000000..09e7f95d3580 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_default_limit/bigquery/out.sql @@ -0,0 +1,5 @@ +SELECT + t0.id, + t0.bool_col +FROM `ibis-gbq`.ibis_gbq_testing.functional_alltypes AS t0 +LIMIT 11 \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_interactive/test_disable_query_limit/bigquery/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_disable_query_limit/bigquery/out.sql new file mode 100644 index 000000000000..09e7f95d3580 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_disable_query_limit/bigquery/out.sql @@ -0,0 +1,5 @@ +SELECT + t0.id, + t0.bool_col +FROM `ibis-gbq`.ibis_gbq_testing.functional_alltypes AS t0 +LIMIT 11 \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_interactive/test_interactive_execute_on_repr/bigquery/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_interactive_execute_on_repr/bigquery/out.sql new file mode 100644 index 000000000000..7601954ae202 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_interactive_execute_on_repr/bigquery/out.sql @@ -0,0 +1,3 @@ +SELECT + SUM(t0.bigint_col) AS Sum_bigint_col +FROM `ibis-gbq`.ibis_gbq_testing.functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_interactive/test_respect_set_limit/bigquery/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_respect_set_limit/bigquery/out.sql new file mode 100644 index 000000000000..1f1906f84e03 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_respect_set_limit/bigquery/out.sql @@ -0,0 +1,10 @@ +SELECT + * +FROM ( + SELECT + t0.id, + t0.bool_col + FROM `ibis-gbq`.ibis_gbq_testing.functional_alltypes AS t0 + LIMIT 10 +) AS t2 +LIMIT 11 \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/bigquery/out.sql b/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/bigquery/out.sql index 96780ab79914..fc16f2428d16 100644 --- a/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/bigquery/out.sql +++ b/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/bigquery/out.sql @@ -1,5 +1,5 @@ SELECT - CASE t0.`continent` + CASE t0.continent WHEN 'NA' THEN 'North America' WHEN 'SA' @@ -15,8 +15,8 @@ SELECT WHEN 'AN' THEN 'Antarctica' ELSE 'Unknown continent' - END AS `cont`, - sum(t0.`population`) AS `total_pop` + END AS cont, + SUM(t0.population) AS total_pop FROM countries AS t0 GROUP BY 1 \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/pyspark/out.sql b/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/pyspark/out.sql new file mode 100644 index 000000000000..ac006b1d5f25 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/pyspark/out.sql @@ -0,0 +1,22 @@ +SELECT + CASE `t0`.`continent` + WHEN 'NA' + THEN 'North America' + WHEN 'SA' + THEN 'South America' + WHEN 'EU' + THEN 'Europe' + WHEN 'AF' + THEN 'Africa' + WHEN 'AS' + THEN 'Asia' + WHEN 'OC' + THEN 'Oceania' + WHEN 'AN' + THEN 'Antarctica' + ELSE 'Unknown continent' + END AS `cont`, + SUM(`t0`.`population`) AS `total_pop` +FROM `countries` AS `t0` +GROUP BY + 1 \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_sql/test_isin_bug/bigquery/out.sql b/ibis/backends/tests/snapshots/test_sql/test_isin_bug/bigquery/out.sql index 95548f1704ec..9e6bcbdd13af 100644 --- a/ibis/backends/tests/snapshots/test_sql/test_isin_bug/bigquery/out.sql +++ b/ibis/backends/tests/snapshots/test_sql/test_isin_bug/bigquery/out.sql @@ -1,13 +1,9 @@ SELECT - t0.`x` IN ( + t0.x IN ( SELECT - t1.`x` - FROM ( - SELECT - t0.* - FROM t AS t0 - WHERE - t0.`x` > 2 - ) AS t1 - ) AS `InColumn_x_ x` + t0.x + FROM t AS t0 + WHERE + t0.x > 2 + ) AS InSubquery_x FROM t AS t0 \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_sql/test_isin_bug/pyspark/out.sql b/ibis/backends/tests/snapshots/test_sql/test_isin_bug/pyspark/out.sql new file mode 100644 index 000000000000..db5ddb124e86 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_sql/test_isin_bug/pyspark/out.sql @@ -0,0 +1,9 @@ +SELECT + `t0`.`x` IN ( + SELECT + `t0`.`x` + FROM `t` AS `t0` + WHERE + `t0`.`x` > 2 + ) AS `InSubquery(x)` +FROM `t` AS `t0` \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_sql/test_union_aliasing/bigquery/out.sql b/ibis/backends/tests/snapshots/test_sql/test_union_aliasing/bigquery/out.sql index 58b8e22977be..a53e4550c0dd 100644 --- a/ibis/backends/tests/snapshots/test_sql/test_union_aliasing/bigquery/out.sql +++ b/ibis/backends/tests/snapshots/test_sql/test_union_aliasing/bigquery/out.sql @@ -1,156 +1,138 @@ -WITH t0 AS ( - SELECT - t7.`field_of_study`, - IF(pos = pos_2, `__pivoted__`, NULL) AS `__pivoted__` - FROM humanities AS t7 - CROSS JOIN UNNEST(GENERATE_ARRAY( - 0, - GREATEST( - ARRAY_LENGTH( - [STRUCT('1970-71' AS years, t7.`1970-71` AS degrees), STRUCT('1975-76' AS years, t7.`1975-76` AS degrees), STRUCT('1980-81' AS years, t7.`1980-81` AS degrees), STRUCT('1985-86' AS years, t7.`1985-86` AS degrees), STRUCT('1990-91' AS years, t7.`1990-91` AS degrees), STRUCT('1995-96' AS years, t7.`1995-96` AS degrees), STRUCT('2000-01' AS years, t7.`2000-01` AS degrees), STRUCT('2005-06' AS years, t7.`2005-06` AS degrees), STRUCT('2010-11' AS years, t7.`2010-11` AS degrees), STRUCT('2011-12' AS years, t7.`2011-12` AS degrees), STRUCT('2012-13' AS years, t7.`2012-13` AS degrees), STRUCT('2013-14' AS years, t7.`2013-14` AS degrees), STRUCT('2014-15' AS years, t7.`2014-15` AS degrees), STRUCT('2015-16' AS years, t7.`2015-16` AS degrees), STRUCT('2016-17' AS years, t7.`2016-17` AS degrees), STRUCT('2017-18' AS years, t7.`2017-18` AS degrees), STRUCT('2018-19' AS years, t7.`2018-19` AS degrees), STRUCT('2019-20' AS years, t7.`2019-20` AS degrees)] - ) - ) - 1 - )) AS pos - CROSS JOIN UNNEST([STRUCT('1970-71' AS years, t7.`1970-71` AS degrees), STRUCT('1975-76' AS years, t7.`1975-76` AS degrees), STRUCT('1980-81' AS years, t7.`1980-81` AS degrees), STRUCT('1985-86' AS years, t7.`1985-86` AS degrees), STRUCT('1990-91' AS years, t7.`1990-91` AS degrees), STRUCT('1995-96' AS years, t7.`1995-96` AS degrees), STRUCT('2000-01' AS years, t7.`2000-01` AS degrees), STRUCT('2005-06' AS years, t7.`2005-06` AS degrees), STRUCT('2010-11' AS years, t7.`2010-11` AS degrees), STRUCT('2011-12' AS years, t7.`2011-12` AS degrees), STRUCT('2012-13' AS years, t7.`2012-13` AS degrees), STRUCT('2013-14' AS years, t7.`2013-14` AS degrees), STRUCT('2014-15' AS years, t7.`2014-15` AS degrees), STRUCT('2015-16' AS years, t7.`2015-16` AS degrees), STRUCT('2016-17' AS years, t7.`2016-17` AS degrees), STRUCT('2017-18' AS years, t7.`2017-18` AS degrees), STRUCT('2018-19' AS years, t7.`2018-19` AS degrees), STRUCT('2019-20' AS years, t7.`2019-20` AS degrees)]) AS `__pivoted__` WITH OFFSET AS pos_2 - WHERE - pos = pos_2 - OR ( - pos > ( - ARRAY_LENGTH( - [STRUCT('1970-71' AS years, t7.`1970-71` AS degrees), STRUCT('1975-76' AS years, t7.`1975-76` AS degrees), STRUCT('1980-81' AS years, t7.`1980-81` AS degrees), STRUCT('1985-86' AS years, t7.`1985-86` AS degrees), STRUCT('1990-91' AS years, t7.`1990-91` AS degrees), STRUCT('1995-96' AS years, t7.`1995-96` AS degrees), STRUCT('2000-01' AS years, t7.`2000-01` AS degrees), STRUCT('2005-06' AS years, t7.`2005-06` AS degrees), STRUCT('2010-11' AS years, t7.`2010-11` AS degrees), STRUCT('2011-12' AS years, t7.`2011-12` AS degrees), STRUCT('2012-13' AS years, t7.`2012-13` AS degrees), STRUCT('2013-14' AS years, t7.`2013-14` AS degrees), STRUCT('2014-15' AS years, t7.`2014-15` AS degrees), STRUCT('2015-16' AS years, t7.`2015-16` AS degrees), STRUCT('2016-17' AS years, t7.`2016-17` AS degrees), STRUCT('2017-18' AS years, t7.`2017-18` AS degrees), STRUCT('2018-19' AS years, t7.`2018-19` AS degrees), STRUCT('2019-20' AS years, t7.`2019-20` AS degrees)] - ) - 1 - ) - AND pos_2 = ( - ARRAY_LENGTH( - [STRUCT('1970-71' AS years, t7.`1970-71` AS degrees), STRUCT('1975-76' AS years, t7.`1975-76` AS degrees), STRUCT('1980-81' AS years, t7.`1980-81` AS degrees), STRUCT('1985-86' AS years, t7.`1985-86` AS degrees), STRUCT('1990-91' AS years, t7.`1990-91` AS degrees), STRUCT('1995-96' AS years, t7.`1995-96` AS degrees), STRUCT('2000-01' AS years, t7.`2000-01` AS degrees), STRUCT('2005-06' AS years, t7.`2005-06` AS degrees), STRUCT('2010-11' AS years, t7.`2010-11` AS degrees), STRUCT('2011-12' AS years, t7.`2011-12` AS degrees), STRUCT('2012-13' AS years, t7.`2012-13` AS degrees), STRUCT('2013-14' AS years, t7.`2013-14` AS degrees), STRUCT('2014-15' AS years, t7.`2014-15` AS degrees), STRUCT('2015-16' AS years, t7.`2015-16` AS degrees), STRUCT('2016-17' AS years, t7.`2016-17` AS degrees), STRUCT('2017-18' AS years, t7.`2017-18` AS degrees), STRUCT('2018-19' AS years, t7.`2018-19` AS degrees), STRUCT('2019-20' AS years, t7.`2019-20` AS degrees)] - ) - 1 - ) - ) -), t1 AS ( - SELECT - t0.`field_of_study`, - t0.`__pivoted__`.`years` AS `years`, - t0.`__pivoted__`.`degrees` AS `degrees` - FROM t0 -), t2 AS ( - SELECT - t1.*, - first_value(t1.`degrees`) OVER (PARTITION BY t1.`field_of_study` ORDER BY t1.`years` ASC) AS `earliest_degrees`, - last_value(t1.`degrees`) OVER (PARTITION BY t1.`field_of_study` ORDER BY t1.`years` ASC) AS `latest_degrees` - FROM t1 -), t3 AS ( - SELECT - t2.*, - t2.`latest_degrees` - t2.`earliest_degrees` AS `diff` - FROM t2 -), t4 AS ( - SELECT - t3.`field_of_study`, - ANY_VALUE(t3.`diff`) AS `diff` - FROM t3 - GROUP BY - 1 -), t5 AS ( - SELECT - t4.* - FROM t4 - WHERE - t4.`diff` < 0 -) SELECT - t6.`field_of_study`, - t6.`diff` + t10.field_of_study, + t10.diff FROM ( - WITH t0 AS ( - SELECT - t7.`field_of_study`, - IF(pos = pos_2, `__pivoted__`, NULL) AS `__pivoted__` - FROM humanities AS t7 - CROSS JOIN UNNEST(GENERATE_ARRAY( - 0, - GREATEST( - ARRAY_LENGTH( - [STRUCT('1970-71' AS years, t7.`1970-71` AS degrees), STRUCT('1975-76' AS years, t7.`1975-76` AS degrees), STRUCT('1980-81' AS years, t7.`1980-81` AS degrees), STRUCT('1985-86' AS years, t7.`1985-86` AS degrees), STRUCT('1990-91' AS years, t7.`1990-91` AS degrees), STRUCT('1995-96' AS years, t7.`1995-96` AS degrees), STRUCT('2000-01' AS years, t7.`2000-01` AS degrees), STRUCT('2005-06' AS years, t7.`2005-06` AS degrees), STRUCT('2010-11' AS years, t7.`2010-11` AS degrees), STRUCT('2011-12' AS years, t7.`2011-12` AS degrees), STRUCT('2012-13' AS years, t7.`2012-13` AS degrees), STRUCT('2013-14' AS years, t7.`2013-14` AS degrees), STRUCT('2014-15' AS years, t7.`2014-15` AS degrees), STRUCT('2015-16' AS years, t7.`2015-16` AS degrees), STRUCT('2016-17' AS years, t7.`2016-17` AS degrees), STRUCT('2017-18' AS years, t7.`2017-18` AS degrees), STRUCT('2018-19' AS years, t7.`2018-19` AS degrees), STRUCT('2019-20' AS years, t7.`2019-20` AS degrees)] - ) - ) - 1 - )) AS pos - CROSS JOIN UNNEST([STRUCT('1970-71' AS years, t7.`1970-71` AS degrees), STRUCT('1975-76' AS years, t7.`1975-76` AS degrees), STRUCT('1980-81' AS years, t7.`1980-81` AS degrees), STRUCT('1985-86' AS years, t7.`1985-86` AS degrees), STRUCT('1990-91' AS years, t7.`1990-91` AS degrees), STRUCT('1995-96' AS years, t7.`1995-96` AS degrees), STRUCT('2000-01' AS years, t7.`2000-01` AS degrees), STRUCT('2005-06' AS years, t7.`2005-06` AS degrees), STRUCT('2010-11' AS years, t7.`2010-11` AS degrees), STRUCT('2011-12' AS years, t7.`2011-12` AS degrees), STRUCT('2012-13' AS years, t7.`2012-13` AS degrees), STRUCT('2013-14' AS years, t7.`2013-14` AS degrees), STRUCT('2014-15' AS years, t7.`2014-15` AS degrees), STRUCT('2015-16' AS years, t7.`2015-16` AS degrees), STRUCT('2016-17' AS years, t7.`2016-17` AS degrees), STRUCT('2017-18' AS years, t7.`2017-18` AS degrees), STRUCT('2018-19' AS years, t7.`2018-19` AS degrees), STRUCT('2019-20' AS years, t7.`2019-20` AS degrees)]) AS `__pivoted__` WITH OFFSET AS pos_2 - WHERE - pos = pos_2 - OR ( - pos > ( - ARRAY_LENGTH( - [STRUCT('1970-71' AS years, t7.`1970-71` AS degrees), STRUCT('1975-76' AS years, t7.`1975-76` AS degrees), STRUCT('1980-81' AS years, t7.`1980-81` AS degrees), STRUCT('1985-86' AS years, t7.`1985-86` AS degrees), STRUCT('1990-91' AS years, t7.`1990-91` AS degrees), STRUCT('1995-96' AS years, t7.`1995-96` AS degrees), STRUCT('2000-01' AS years, t7.`2000-01` AS degrees), STRUCT('2005-06' AS years, t7.`2005-06` AS degrees), STRUCT('2010-11' AS years, t7.`2010-11` AS degrees), STRUCT('2011-12' AS years, t7.`2011-12` AS degrees), STRUCT('2012-13' AS years, t7.`2012-13` AS degrees), STRUCT('2013-14' AS years, t7.`2013-14` AS degrees), STRUCT('2014-15' AS years, t7.`2014-15` AS degrees), STRUCT('2015-16' AS years, t7.`2015-16` AS degrees), STRUCT('2016-17' AS years, t7.`2016-17` AS degrees), STRUCT('2017-18' AS years, t7.`2017-18` AS degrees), STRUCT('2018-19' AS years, t7.`2018-19` AS degrees), STRUCT('2019-20' AS years, t7.`2019-20` AS degrees)] - ) - 1 - ) - AND pos_2 = ( - ARRAY_LENGTH( - [STRUCT('1970-71' AS years, t7.`1970-71` AS degrees), STRUCT('1975-76' AS years, t7.`1975-76` AS degrees), STRUCT('1980-81' AS years, t7.`1980-81` AS degrees), STRUCT('1985-86' AS years, t7.`1985-86` AS degrees), STRUCT('1990-91' AS years, t7.`1990-91` AS degrees), STRUCT('1995-96' AS years, t7.`1995-96` AS degrees), STRUCT('2000-01' AS years, t7.`2000-01` AS degrees), STRUCT('2005-06' AS years, t7.`2005-06` AS degrees), STRUCT('2010-11' AS years, t7.`2010-11` AS degrees), STRUCT('2011-12' AS years, t7.`2011-12` AS degrees), STRUCT('2012-13' AS years, t7.`2012-13` AS degrees), STRUCT('2013-14' AS years, t7.`2013-14` AS degrees), STRUCT('2014-15' AS years, t7.`2014-15` AS degrees), STRUCT('2015-16' AS years, t7.`2015-16` AS degrees), STRUCT('2016-17' AS years, t7.`2016-17` AS degrees), STRUCT('2017-18' AS years, t7.`2017-18` AS degrees), STRUCT('2018-19' AS years, t7.`2018-19` AS degrees), STRUCT('2019-20' AS years, t7.`2019-20` AS degrees)] - ) - 1 - ) - ) - ), t1 AS ( - SELECT - t0.`field_of_study`, - t0.`__pivoted__`.`years` AS `years`, - t0.`__pivoted__`.`degrees` AS `degrees` - FROM t0 - ), t2 AS ( - SELECT - t1.*, - first_value(t1.`degrees`) OVER (PARTITION BY t1.`field_of_study` ORDER BY t1.`years` ASC) AS `earliest_degrees`, - last_value(t1.`degrees`) OVER (PARTITION BY t1.`field_of_study` ORDER BY t1.`years` ASC) AS `latest_degrees` - FROM t1 - ), t3 AS ( - SELECT - t2.*, - t2.`latest_degrees` - t2.`earliest_degrees` AS `diff` - FROM t2 - ), t4 AS ( + SELECT + t5.field_of_study, + t5.diff + FROM ( SELECT - t3.`field_of_study`, - ANY_VALUE(t3.`diff`) AS `diff` - FROM t3 + t4.field_of_study, + ANY_VALUE(t4.diff) AS diff + FROM ( + SELECT + t3.field_of_study, + t3.years, + t3.degrees, + t3.earliest_degrees, + t3.latest_degrees, + t3.latest_degrees - t3.earliest_degrees AS diff + FROM ( + SELECT + t2.field_of_study, + t2.years, + t2.degrees, + first_value(t2.degrees) OVER (PARTITION BY t2.field_of_study ORDER BY t2.years ASC ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS earliest_degrees, + last_value(t2.degrees) OVER (PARTITION BY t2.field_of_study ORDER BY t2.years ASC ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS latest_degrees + FROM ( + SELECT + t1.field_of_study, + t1.__pivoted__.years AS years, + t1.__pivoted__.degrees AS degrees + FROM ( + SELECT + t0.field_of_study, + IF(pos = pos_2, __pivoted__, NULL) AS __pivoted__ + FROM humanities AS t0 + CROSS JOIN UNNEST(GENERATE_ARRAY( + 0, + GREATEST( + ARRAY_LENGTH( + [STRUCT('1970-71' AS years, t0.`1970-71` AS degrees), STRUCT('1975-76' AS years, t0.`1975-76` AS degrees), STRUCT('1980-81' AS years, t0.`1980-81` AS degrees), STRUCT('1985-86' AS years, t0.`1985-86` AS degrees), STRUCT('1990-91' AS years, t0.`1990-91` AS degrees), STRUCT('1995-96' AS years, t0.`1995-96` AS degrees), STRUCT('2000-01' AS years, t0.`2000-01` AS degrees), STRUCT('2005-06' AS years, t0.`2005-06` AS degrees), STRUCT('2010-11' AS years, t0.`2010-11` AS degrees), STRUCT('2011-12' AS years, t0.`2011-12` AS degrees), STRUCT('2012-13' AS years, t0.`2012-13` AS degrees), STRUCT('2013-14' AS years, t0.`2013-14` AS degrees), STRUCT('2014-15' AS years, t0.`2014-15` AS degrees), STRUCT('2015-16' AS years, t0.`2015-16` AS degrees), STRUCT('2016-17' AS years, t0.`2016-17` AS degrees), STRUCT('2017-18' AS years, t0.`2017-18` AS degrees), STRUCT('2018-19' AS years, t0.`2018-19` AS degrees), STRUCT('2019-20' AS years, t0.`2019-20` AS degrees)] + ) + ) - 1 + )) AS pos + CROSS JOIN UNNEST([STRUCT('1970-71' AS years, t0.`1970-71` AS degrees), STRUCT('1975-76' AS years, t0.`1975-76` AS degrees), STRUCT('1980-81' AS years, t0.`1980-81` AS degrees), STRUCT('1985-86' AS years, t0.`1985-86` AS degrees), STRUCT('1990-91' AS years, t0.`1990-91` AS degrees), STRUCT('1995-96' AS years, t0.`1995-96` AS degrees), STRUCT('2000-01' AS years, t0.`2000-01` AS degrees), STRUCT('2005-06' AS years, t0.`2005-06` AS degrees), STRUCT('2010-11' AS years, t0.`2010-11` AS degrees), STRUCT('2011-12' AS years, t0.`2011-12` AS degrees), STRUCT('2012-13' AS years, t0.`2012-13` AS degrees), STRUCT('2013-14' AS years, t0.`2013-14` AS degrees), STRUCT('2014-15' AS years, t0.`2014-15` AS degrees), STRUCT('2015-16' AS years, t0.`2015-16` AS degrees), STRUCT('2016-17' AS years, t0.`2016-17` AS degrees), STRUCT('2017-18' AS years, t0.`2017-18` AS degrees), STRUCT('2018-19' AS years, t0.`2018-19` AS degrees), STRUCT('2019-20' AS years, t0.`2019-20` AS degrees)]) AS __pivoted__ WITH OFFSET AS pos_2 + WHERE + pos = pos_2 + OR ( + pos > ( + ARRAY_LENGTH( + [STRUCT('1970-71' AS years, t0.`1970-71` AS degrees), STRUCT('1975-76' AS years, t0.`1975-76` AS degrees), STRUCT('1980-81' AS years, t0.`1980-81` AS degrees), STRUCT('1985-86' AS years, t0.`1985-86` AS degrees), STRUCT('1990-91' AS years, t0.`1990-91` AS degrees), STRUCT('1995-96' AS years, t0.`1995-96` AS degrees), STRUCT('2000-01' AS years, t0.`2000-01` AS degrees), STRUCT('2005-06' AS years, t0.`2005-06` AS degrees), STRUCT('2010-11' AS years, t0.`2010-11` AS degrees), STRUCT('2011-12' AS years, t0.`2011-12` AS degrees), STRUCT('2012-13' AS years, t0.`2012-13` AS degrees), STRUCT('2013-14' AS years, t0.`2013-14` AS degrees), STRUCT('2014-15' AS years, t0.`2014-15` AS degrees), STRUCT('2015-16' AS years, t0.`2015-16` AS degrees), STRUCT('2016-17' AS years, t0.`2016-17` AS degrees), STRUCT('2017-18' AS years, t0.`2017-18` AS degrees), STRUCT('2018-19' AS years, t0.`2018-19` AS degrees), STRUCT('2019-20' AS years, t0.`2019-20` AS degrees)] + ) - 1 + ) + AND pos_2 = ( + ARRAY_LENGTH( + [STRUCT('1970-71' AS years, t0.`1970-71` AS degrees), STRUCT('1975-76' AS years, t0.`1975-76` AS degrees), STRUCT('1980-81' AS years, t0.`1980-81` AS degrees), STRUCT('1985-86' AS years, t0.`1985-86` AS degrees), STRUCT('1990-91' AS years, t0.`1990-91` AS degrees), STRUCT('1995-96' AS years, t0.`1995-96` AS degrees), STRUCT('2000-01' AS years, t0.`2000-01` AS degrees), STRUCT('2005-06' AS years, t0.`2005-06` AS degrees), STRUCT('2010-11' AS years, t0.`2010-11` AS degrees), STRUCT('2011-12' AS years, t0.`2011-12` AS degrees), STRUCT('2012-13' AS years, t0.`2012-13` AS degrees), STRUCT('2013-14' AS years, t0.`2013-14` AS degrees), STRUCT('2014-15' AS years, t0.`2014-15` AS degrees), STRUCT('2015-16' AS years, t0.`2015-16` AS degrees), STRUCT('2016-17' AS years, t0.`2016-17` AS degrees), STRUCT('2017-18' AS years, t0.`2017-18` AS degrees), STRUCT('2018-19' AS years, t0.`2018-19` AS degrees), STRUCT('2019-20' AS years, t0.`2019-20` AS degrees)] + ) - 1 + ) + ) + ) AS t1 + ) AS t2 + ) AS t3 + ) AS t4 GROUP BY 1 - ), t5 AS ( - SELECT - t4.* - FROM t4 - WHERE - t4.`diff` < 0 - ), t7 AS ( - SELECT - t5.* - FROM t5 - ORDER BY - t5.`diff` ASC - ), t8 AS ( - SELECT - t4.* - FROM t4 - ORDER BY - t4.`diff` DESC - ), t9 AS ( - SELECT - t5.* - FROM t5 - ORDER BY - t5.`diff` ASC - LIMIT 10 - ), t10 AS ( - SELECT - t4.* - FROM t4 - ORDER BY - t4.`diff` DESC - LIMIT 10 - ) - SELECT - * - FROM t10 + ) AS t5 + ORDER BY + t5.diff DESC + LIMIT 10 UNION ALL SELECT - * - FROM t9 -) AS t6 \ No newline at end of file + t5.field_of_study, + t5.diff + FROM ( + SELECT + t4.field_of_study, + ANY_VALUE(t4.diff) AS diff + FROM ( + SELECT + t3.field_of_study, + t3.years, + t3.degrees, + t3.earliest_degrees, + t3.latest_degrees, + t3.latest_degrees - t3.earliest_degrees AS diff + FROM ( + SELECT + t2.field_of_study, + t2.years, + t2.degrees, + first_value(t2.degrees) OVER (PARTITION BY t2.field_of_study ORDER BY t2.years ASC ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS earliest_degrees, + last_value(t2.degrees) OVER (PARTITION BY t2.field_of_study ORDER BY t2.years ASC ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS latest_degrees + FROM ( + SELECT + t1.field_of_study, + t1.__pivoted__.years AS years, + t1.__pivoted__.degrees AS degrees + FROM ( + SELECT + t0.field_of_study, + IF(pos = pos_2, __pivoted__, NULL) AS __pivoted__ + FROM humanities AS t0 + CROSS JOIN UNNEST(GENERATE_ARRAY( + 0, + GREATEST( + ARRAY_LENGTH( + [STRUCT('1970-71' AS years, t0.`1970-71` AS degrees), STRUCT('1975-76' AS years, t0.`1975-76` AS degrees), STRUCT('1980-81' AS years, t0.`1980-81` AS degrees), STRUCT('1985-86' AS years, t0.`1985-86` AS degrees), STRUCT('1990-91' AS years, t0.`1990-91` AS degrees), STRUCT('1995-96' AS years, t0.`1995-96` AS degrees), STRUCT('2000-01' AS years, t0.`2000-01` AS degrees), STRUCT('2005-06' AS years, t0.`2005-06` AS degrees), STRUCT('2010-11' AS years, t0.`2010-11` AS degrees), STRUCT('2011-12' AS years, t0.`2011-12` AS degrees), STRUCT('2012-13' AS years, t0.`2012-13` AS degrees), STRUCT('2013-14' AS years, t0.`2013-14` AS degrees), STRUCT('2014-15' AS years, t0.`2014-15` AS degrees), STRUCT('2015-16' AS years, t0.`2015-16` AS degrees), STRUCT('2016-17' AS years, t0.`2016-17` AS degrees), STRUCT('2017-18' AS years, t0.`2017-18` AS degrees), STRUCT('2018-19' AS years, t0.`2018-19` AS degrees), STRUCT('2019-20' AS years, t0.`2019-20` AS degrees)] + ) + ) - 1 + )) AS pos + CROSS JOIN UNNEST([STRUCT('1970-71' AS years, t0.`1970-71` AS degrees), STRUCT('1975-76' AS years, t0.`1975-76` AS degrees), STRUCT('1980-81' AS years, t0.`1980-81` AS degrees), STRUCT('1985-86' AS years, t0.`1985-86` AS degrees), STRUCT('1990-91' AS years, t0.`1990-91` AS degrees), STRUCT('1995-96' AS years, t0.`1995-96` AS degrees), STRUCT('2000-01' AS years, t0.`2000-01` AS degrees), STRUCT('2005-06' AS years, t0.`2005-06` AS degrees), STRUCT('2010-11' AS years, t0.`2010-11` AS degrees), STRUCT('2011-12' AS years, t0.`2011-12` AS degrees), STRUCT('2012-13' AS years, t0.`2012-13` AS degrees), STRUCT('2013-14' AS years, t0.`2013-14` AS degrees), STRUCT('2014-15' AS years, t0.`2014-15` AS degrees), STRUCT('2015-16' AS years, t0.`2015-16` AS degrees), STRUCT('2016-17' AS years, t0.`2016-17` AS degrees), STRUCT('2017-18' AS years, t0.`2017-18` AS degrees), STRUCT('2018-19' AS years, t0.`2018-19` AS degrees), STRUCT('2019-20' AS years, t0.`2019-20` AS degrees)]) AS __pivoted__ WITH OFFSET AS pos_2 + WHERE + pos = pos_2 + OR ( + pos > ( + ARRAY_LENGTH( + [STRUCT('1970-71' AS years, t0.`1970-71` AS degrees), STRUCT('1975-76' AS years, t0.`1975-76` AS degrees), STRUCT('1980-81' AS years, t0.`1980-81` AS degrees), STRUCT('1985-86' AS years, t0.`1985-86` AS degrees), STRUCT('1990-91' AS years, t0.`1990-91` AS degrees), STRUCT('1995-96' AS years, t0.`1995-96` AS degrees), STRUCT('2000-01' AS years, t0.`2000-01` AS degrees), STRUCT('2005-06' AS years, t0.`2005-06` AS degrees), STRUCT('2010-11' AS years, t0.`2010-11` AS degrees), STRUCT('2011-12' AS years, t0.`2011-12` AS degrees), STRUCT('2012-13' AS years, t0.`2012-13` AS degrees), STRUCT('2013-14' AS years, t0.`2013-14` AS degrees), STRUCT('2014-15' AS years, t0.`2014-15` AS degrees), STRUCT('2015-16' AS years, t0.`2015-16` AS degrees), STRUCT('2016-17' AS years, t0.`2016-17` AS degrees), STRUCT('2017-18' AS years, t0.`2017-18` AS degrees), STRUCT('2018-19' AS years, t0.`2018-19` AS degrees), STRUCT('2019-20' AS years, t0.`2019-20` AS degrees)] + ) - 1 + ) + AND pos_2 = ( + ARRAY_LENGTH( + [STRUCT('1970-71' AS years, t0.`1970-71` AS degrees), STRUCT('1975-76' AS years, t0.`1975-76` AS degrees), STRUCT('1980-81' AS years, t0.`1980-81` AS degrees), STRUCT('1985-86' AS years, t0.`1985-86` AS degrees), STRUCT('1990-91' AS years, t0.`1990-91` AS degrees), STRUCT('1995-96' AS years, t0.`1995-96` AS degrees), STRUCT('2000-01' AS years, t0.`2000-01` AS degrees), STRUCT('2005-06' AS years, t0.`2005-06` AS degrees), STRUCT('2010-11' AS years, t0.`2010-11` AS degrees), STRUCT('2011-12' AS years, t0.`2011-12` AS degrees), STRUCT('2012-13' AS years, t0.`2012-13` AS degrees), STRUCT('2013-14' AS years, t0.`2013-14` AS degrees), STRUCT('2014-15' AS years, t0.`2014-15` AS degrees), STRUCT('2015-16' AS years, t0.`2015-16` AS degrees), STRUCT('2016-17' AS years, t0.`2016-17` AS degrees), STRUCT('2017-18' AS years, t0.`2017-18` AS degrees), STRUCT('2018-19' AS years, t0.`2018-19` AS degrees), STRUCT('2019-20' AS years, t0.`2019-20` AS degrees)] + ) - 1 + ) + ) + ) AS t1 + ) AS t2 + ) AS t3 + ) AS t4 + GROUP BY + 1 + ) AS t5 + WHERE + t5.diff < 0 + ORDER BY + t5.diff ASC NULLS LAST + LIMIT 10 +) AS t10 \ No newline at end of file diff --git a/ibis/backends/tests/test_aggregation.py b/ibis/backends/tests/test_aggregation.py index ddbf55d77bbf..52649199d79b 100644 --- a/ibis/backends/tests/test_aggregation.py +++ b/ibis/backends/tests/test_aggregation.py @@ -1374,7 +1374,9 @@ def test_group_concat( .reset_index() ) - backend.assert_frame_equal(result.fillna(pd.NA), expected.fillna(pd.NA)) + backend.assert_frame_equal( + result.replace(np.nan, None), expected.replace(np.nan, None) + ) @pytest.mark.broken( diff --git a/ibis/backends/tests/test_asof_join.py b/ibis/backends/tests/test_asof_join.py index a0a67a36c687..e7563587f7f6 100644 --- a/ibis/backends/tests/test_asof_join.py +++ b/ibis/backends/tests/test_asof_join.py @@ -90,6 +90,7 @@ def time_keyed_right(time_keyed_df2): "pyspark", "druid", "impala", + "bigquery", ] ) def test_asof_join(con, time_left, time_right, time_df1, time_df2, direction, op): @@ -125,6 +126,7 @@ def test_asof_join(con, time_left, time_right, time_df1, time_df2, direction, op "pyspark", "druid", "impala", + "bigquery", ] ) def test_keyed_asof_join_with_tolerance( diff --git a/ibis/backends/tests/test_generic.py b/ibis/backends/tests/test_generic.py index 617abc539b64..c92f65090918 100644 --- a/ibis/backends/tests/test_generic.py +++ b/ibis/backends/tests/test_generic.py @@ -1347,7 +1347,6 @@ def hash_256(col): [ "pandas", "dask", - "bigquery", "mssql", "oracle", "risingwave", @@ -1369,6 +1368,7 @@ def hash_256(col): 1672531200, marks=[ pytest.mark.notyet(["duckdb", "impala"], reason="casts to NULL"), + pytest.mark.notyet(["bigquery"], raises=GoogleBadRequest), pytest.mark.notyet(["trino"], raises=TrinoUserError), pytest.mark.broken( ["druid"], reason="casts to 1672531200000 (millisecond)" @@ -1393,7 +1393,6 @@ def test_try_cast(con, from_val, to_type, expected): @pytest.mark.notimpl( [ - "bigquery", "dask", "datafusion", "druid", @@ -1419,6 +1418,7 @@ def test_try_cast(con, from_val, to_type, expected): pytest.mark.never( ["clickhouse", "pyspark"], reason="casts to 1672531200" ), + pytest.mark.notyet(["bigquery"], raises=GoogleBadRequest), pytest.mark.notyet(["trino"], raises=TrinoUserError), pytest.mark.broken(["polars"], reason="casts to 1672531200000000000"), ], @@ -1434,7 +1434,6 @@ def test_try_cast_null(con, from_val, to_type): [ "pandas", "dask", - "bigquery", "datafusion", "druid", "mssql", @@ -1464,7 +1463,6 @@ def test_try_cast_table(backend, con): [ "pandas", "dask", - "bigquery", "datafusion", "mssql", "mysql", @@ -1490,6 +1488,7 @@ def test_try_cast_table(backend, con): ["clickhouse", "polars", "flink", "pyspark"], reason="casts this to to a number", ), + pytest.mark.notyet(["bigquery"], raises=GoogleBadRequest), pytest.mark.notyet(["trino"], raises=TrinoUserError), ], id="datetime-to-float", @@ -1797,7 +1796,6 @@ def test_dynamic_table_slice_with_computed_offset(backend): @pytest.mark.notimpl( [ - "bigquery", "druid", "flink", "polars", @@ -1827,7 +1825,6 @@ def test_sample(backend): @pytest.mark.notimpl( [ - "bigquery", "druid", "flink", "polars", diff --git a/ibis/backends/tests/test_numeric.py b/ibis/backends/tests/test_numeric.py index 19bf6f15f35e..cc4b54067519 100644 --- a/ibis/backends/tests/test_numeric.py +++ b/ibis/backends/tests/test_numeric.py @@ -395,7 +395,6 @@ def test_numeric_literal(con, backend, expr, expected_types): ibis.literal(decimal.Decimal("Infinity"), type=dt.decimal), # TODO(krzysztof-kwitt): Should we unify it? { - "bigquery": float("inf"), "sqlite": float("inf"), "risingwave": float("nan"), "postgres": decimal.Decimal("Infinity"), @@ -406,7 +405,6 @@ def test_numeric_literal(con, backend, expr, expected_types): "duckdb": float("inf"), }, { - "bigquery": "FLOAT64", "sqlite": "real", "postgres": "numeric", "risingwave": "numeric", @@ -465,6 +463,7 @@ def test_numeric_literal(con, backend, expr, expected_types): "infinity is not allowed as a decimal value", raises=SnowflakeProgrammingError, ), + pytest.mark.notyet(["bigquery"], raises=GoogleBadRequest), ], id="decimal-infinity+", ), @@ -472,7 +471,6 @@ def test_numeric_literal(con, backend, expr, expected_types): ibis.literal(decimal.Decimal("-Infinity"), type=dt.decimal), # TODO(krzysztof-kwitt): Should we unify it? { - "bigquery": float("-inf"), "sqlite": float("-inf"), "risingwave": float("nan"), "postgres": decimal.Decimal("-Infinity"), @@ -483,7 +481,6 @@ def test_numeric_literal(con, backend, expr, expected_types): "duckdb": float("-inf"), }, { - "bigquery": "FLOAT64", "sqlite": "real", "postgres": "numeric", "risingwave": "numeric", @@ -542,6 +539,7 @@ def test_numeric_literal(con, backend, expr, expected_types): raises=TrinoUserError, reason="can't cast infinity to decimal", ), + pytest.mark.notyet(["bigquery"], raises=GoogleBadRequest), ], id="decimal-infinity-", ), @@ -629,6 +627,7 @@ def test_numeric_literal(con, backend, expr, expected_types): raises=TrinoUserError, reason="can't cast nan to decimal", ), + pytest.mark.notyet(["bigquery"], raises=GoogleBadRequest), ], id="decimal-NaN", ), diff --git a/ibis/backends/tests/test_param.py b/ibis/backends/tests/test_param.py index 01dad3aa36de..ed8bb91d466b 100644 --- a/ibis/backends/tests/test_param.py +++ b/ibis/backends/tests/test_param.py @@ -12,7 +12,7 @@ import ibis import ibis.expr.datatypes as dt from ibis import _ -from ibis.backends.tests.errors import GoogleBadRequest, Py4JJavaError +from ibis.backends.tests.errors import Py4JJavaError @pytest.mark.parametrize( @@ -144,42 +144,21 @@ def test_scalar_param_map(con): "timestamp", "timestamp_col", id="string_timestamp", - marks=[ - pytest.mark.notimpl(["druid"]), - pytest.mark.broken( - ["bigquery"], - raises=GoogleBadRequest, - reason="No matching for operator = for argument types: DATETIME, TIMESTAMP", - ), - ], + marks=[pytest.mark.notimpl(["druid"])], ), param( datetime.date(2009, 1, 20), "timestamp", "timestamp_col", id="date_timestamp", - marks=[ - pytest.mark.notimpl(["druid"]), - pytest.mark.broken( - ["bigquery"], - raises=GoogleBadRequest, - reason="No matching for operator = for argument types: DATETIME, TIMESTAMP", - ), - ], + marks=[pytest.mark.notimpl(["druid"])], ), param( datetime.datetime(2009, 1, 20, 1, 2, 3), "timestamp", "timestamp_col", id="datetime_timestamp", - marks=[ - pytest.mark.notimpl(["druid"]), - pytest.mark.broken( - ["bigquery"], - raises=GoogleBadRequest, - reason="No matching for operator = for argument types: DATETIME, TIMESTAMP", - ), - ], + marks=[pytest.mark.notimpl(["druid"])], ), ], ) diff --git a/ibis/backends/tests/test_sql.py b/ibis/backends/tests/test_sql.py index b3033c312359..74db927d928d 100644 --- a/ibis/backends/tests/test_sql.py +++ b/ibis/backends/tests/test_sql.py @@ -61,7 +61,7 @@ def test_literal(backend, expr): assert ibis.to_sql(expr, dialect=backend.name()) -@pytest.mark.never(["pandas", "dask", "polars", "pyspark"], reason="not SQL") +@pytest.mark.never(["pandas", "dask", "polars"], reason="not SQL") @pytest.mark.xfail_version( mssql=["sqlalchemy>=2"], reason="sqlalchemy 2 prefixes literals with `N`" ) @@ -103,7 +103,7 @@ def test_cte_refs_in_topo_order(backend, snapshot): snapshot.assert_match(sql, "out.sql") -@pytest.mark.never(["pandas", "dask", "polars", "pyspark"], reason="not SQL") +@pytest.mark.never(["pandas", "dask", "polars"], reason="not SQL") def test_isin_bug(con, snapshot): t = ibis.table(dict(x="int"), name="t") good = t[t.x > 2].x diff --git a/ibis/backends/tests/test_string.py b/ibis/backends/tests/test_string.py index 5f67dc144cab..5fc5945dc2a3 100644 --- a/ibis/backends/tests/test_string.py +++ b/ibis/backends/tests/test_string.py @@ -1000,7 +1000,6 @@ def test_multiple_subs(con): @pytest.mark.notimpl( [ - "bigquery", "clickhouse", "dask", "datafusion", diff --git a/ibis/backends/tests/test_temporal.py b/ibis/backends/tests/test_temporal.py index 9ed2b1e83855..ab71cefc15e6 100644 --- a/ibis/backends/tests/test_temporal.py +++ b/ibis/backends/tests/test_temporal.py @@ -606,11 +606,6 @@ def test_date_truncate(backend, alltypes, df, unit): pd.offsets.DateOffset, # TODO - DateOffset - #2553 marks=[ - pytest.mark.notimpl( - ["bigquery"], - raises=com.UnsupportedOperationError, - reason="BigQuery does not allow binary operation TIMESTAMP_ADD with INTERVAL offset D", - ), pytest.mark.notimpl( ["polars"], raises=TypeError, @@ -634,11 +629,6 @@ def test_date_truncate(backend, alltypes, df, unit): pd.offsets.DateOffset, # TODO - DateOffset - #2553 marks=[ - pytest.mark.notimpl( - ["bigquery"], - raises=com.UnsupportedOperationError, - reason="BigQuery does not allow binary operation TIMESTAMP_ADD with INTERVAL offset M", - ), pytest.mark.notimpl( ["dask"], raises=ValueError, @@ -661,11 +651,6 @@ def test_date_truncate(backend, alltypes, df, unit): pd.offsets.DateOffset, # TODO - DateOffset - #2553 marks=[ - pytest.mark.notimpl( - ["bigquery"], - raises=com.UnsupportedOperationError, - reason="BigQuery does not allow extracting date part `IntervalUnit.WEEK` from intervals", - ), pytest.mark.notimpl( ["dask"], raises=ValueError, diff --git a/ibis/backends/tests/test_udf.py b/ibis/backends/tests/test_udf.py index 15be61cf1723..2a55a30355b2 100644 --- a/ibis/backends/tests/test_udf.py +++ b/ibis/backends/tests/test_udf.py @@ -37,7 +37,7 @@ def num_vowels(s: str, include_y: bool = False) -> int: batting = batting.limit(100) nvowels = num_vowels(batting.playerID) assert nvowels.op().__module__ == __name__ - assert type(nvowels.op()).__qualname__ == "num_vowels" + assert type(nvowels.op()).__qualname__.startswith("num_vowels") expr = batting.group_by(id_len=nvowels).agg(n=_.count()) result = expr.execute() diff --git a/ibis/backends/tests/test_window.py b/ibis/backends/tests/test_window.py index d48902ead09e..f2fda3721a19 100644 --- a/ibis/backends/tests/test_window.py +++ b/ibis/backends/tests/test_window.py @@ -719,7 +719,7 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): reason="Window operations are unsupported in the dask backend", ), pytest.mark.broken( - ["bigquery", "flink", "impala"], + ["flink", "impala"], reason="default window semantics are different", raises=AssertionError, ), @@ -1306,17 +1306,12 @@ def test_rank_followed_by_over_call_merge_frames(backend, alltypes, df): ) @pytest.mark.notimpl(["polars"], raises=com.OperationNotDefinedError) @pytest.mark.notyet(["flink"], raises=com.UnsupportedOperationError) -@pytest.mark.broken( - ["pandas"], - raises=TypeError, - reason="pandas rank impl cannot handle compound sort keys with null", -) @pytest.mark.notimpl( ["risingwave"], raises=sa.exc.InternalError, reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", ) -def test_ordering_order(con): +def test_windowed_order_by_sequence_is_preserved(con): table = ibis.memtable({"bool_col": [True, False, False, None, True]}) window = ibis.window( order_by=[ diff --git a/ibis/expr/operations/udf.py b/ibis/expr/operations/udf.py index 106fb4a22348..3a469db5b57e 100644 --- a/ibis/expr/operations/udf.py +++ b/ibis/expr/operations/udf.py @@ -1,9 +1,11 @@ from __future__ import annotations import abc +import collections import enum import functools import inspect +import itertools import typing from typing import TYPE_CHECKING, Any, Callable, Optional, TypeVar, overload @@ -20,12 +22,24 @@ from ibis.common.deferred import deferrable if TYPE_CHECKING: + from collections.abc import Iterable, MutableMapping + import ibis.expr.types as ir EMPTY = inspect.Parameter.empty +_udf_name_cache: MutableMapping[ + type[ops.Node], Iterable[int] +] = collections.defaultdict(itertools.count) + + +def _make_udf_name(name: str) -> str: + definition = next(_udf_name_cache[name]) + return f"{name}_{definition:d}" + + @enum.unique class InputType(enum.Enum): BUILTIN = enum.auto() @@ -88,6 +102,7 @@ def _make_node( input_type: InputType, name: str | None = None, schema: str | None = None, + database: str | None = None, signature: tuple[tuple, Any] | None = None, **kwargs, ) -> type[S]: @@ -124,13 +139,13 @@ def _make_node( # method "__func__": property(fget=lambda _, fn=fn: fn), "__config__": FrozenDict(kwargs), - "__udf_namespace__": schema, + "__udf_namespace__": ops.Namespace(schema=schema, database=database), "__module__": fn.__module__, "__func_name__": func_name, } ) - return type(fn.__name__, (cls._base,), fields) + return type(_make_udf_name(fn.__name__), (cls._base,), fields) @classmethod def _make_wrapper( @@ -168,6 +183,7 @@ def builtin( *, name: str | None = None, schema: str | None = None, + database: str | None = None, signature: tuple[tuple[Any, ...], Any] | None = None, **kwargs: Any, ) -> Callable[[Callable], Callable[..., ir.Value]]: @@ -175,7 +191,9 @@ def builtin( @util.experimental @classmethod - def builtin(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): + def builtin( + cls, fn=None, *, name=None, schema=None, database=None, signature=None, **kwargs + ): """Construct a scalar user-defined function that is built-in to the backend. Parameters @@ -186,6 +204,8 @@ def builtin(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): The name of the UDF in the backend if different from the function name. schema The schema in which the builtin function resides. + database + The database in which the builtin function resides. signature If present, a tuple of the form `((arg0type, arg1type, ...), returntype)`. For example, a function taking an int and a float and returning a @@ -215,6 +235,7 @@ def builtin(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): fn, name=name, schema=schema, + database=database, signature=signature, **kwargs, ) @@ -231,6 +252,7 @@ def python( *, name: str | None = None, schema: str | None = None, + database: str | None = None, signature: tuple[tuple[Any, ...], Any] | None = None, **kwargs: Any, ) -> Callable[[Callable], Callable[..., ir.Value]]: @@ -238,7 +260,9 @@ def python( @util.experimental @classmethod - def python(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): + def python( + cls, fn=None, *, name=None, schema=None, database=None, signature=None, **kwargs + ): """Construct a **non-vectorized** scalar user-defined function that accepts Python scalar values as inputs. ::: {.callout-warning collapse="true"} @@ -262,6 +286,8 @@ def python(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): The name of the UDF in the backend if different from the function name. schema The schema in which to create the UDF. + database + The database in which to create the UDF. signature If present, a tuple of the form `((arg0type, arg1type, ...), returntype)`. For example, a function taking an int and a float and returning a @@ -292,6 +318,7 @@ def python(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): fn, name=name, schema=schema, + database=database, signature=signature, **kwargs, ) @@ -308,6 +335,7 @@ def pandas( *, name: str | None = None, schema: str | None = None, + database: str | None = None, signature: tuple[tuple[Any, ...], Any] | None = None, **kwargs: Any, ) -> Callable[[Callable], Callable[..., ir.Value]]: @@ -315,7 +343,9 @@ def pandas( @util.experimental @classmethod - def pandas(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): + def pandas( + cls, fn=None, *, name=None, schema=None, database=None, signature=None, **kwargs + ): """Construct a **vectorized** scalar user-defined function that accepts pandas Series' as inputs. Parameters @@ -326,6 +356,8 @@ def pandas(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): The name of the UDF in the backend if different from the function name. schema The schema in which to create the UDF. + database + The database in which to create the UDF. signature If present, a tuple of the form `((arg0type, arg1type, ...), returntype)`. For example, a function taking an int and a float and returning a @@ -358,6 +390,7 @@ def pandas(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): fn, name=name, schema=schema, + database=database, signature=signature, **kwargs, ) @@ -374,6 +407,7 @@ def pyarrow( *, name: str | None = None, schema: str | None = None, + database: str | None = None, signature: tuple[tuple[Any, ...], Any] | None = None, **kwargs: Any, ) -> Callable[[Callable], Callable[..., ir.Value]]: @@ -381,7 +415,9 @@ def pyarrow( @util.experimental @classmethod - def pyarrow(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): + def pyarrow( + cls, fn=None, *, name=None, schema=None, database=None, signature=None, **kwargs + ): """Construct a **vectorized** scalar user-defined function that accepts PyArrow Arrays as input. Parameters @@ -392,6 +428,8 @@ def pyarrow(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): The name of the UDF in the backend if different from the function name. schema The schema in which to create the UDF. + database + The database in which to create the UDF. signature If present, a tuple of the form `((arg0type, arg1type, ...), returntype)`. For example, a function taking an int and a float and returning a @@ -423,6 +461,7 @@ def pyarrow(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): fn, name=name, schema=schema, + database=database, signature=signature, **kwargs, ) @@ -446,6 +485,7 @@ def builtin( *, name: str | None = None, schema: str | None = None, + database: str | None = None, signature: tuple[tuple[Any, ...], Any] | None = None, **kwargs: Any, ) -> Callable[[Callable], Callable[..., ir.Value]]: @@ -453,7 +493,9 @@ def builtin( @util.experimental @classmethod - def builtin(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): + def builtin( + cls, fn=None, *, name=None, schema=None, database=None, signature=None, **kwargs + ): """Construct an aggregate user-defined function that is built-in to the backend. Parameters @@ -464,6 +506,8 @@ def builtin(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): The name of the UDF in the backend if different from the function name. schema The schema in which the builtin function resides. + database + The database in which the builtin function resides. signature If present, a tuple of the form `((arg0type, arg1type, ...), returntype)`. For example, a function taking an int and a float and returning a @@ -490,6 +534,7 @@ def builtin(cls, fn=None, *, name=None, schema=None, signature=None, **kwargs): fn, name=name, schema=schema, + database=database, signature=signature, **kwargs, ) diff --git a/ibis/expr/rewrites.py b/ibis/expr/rewrites.py index e62a339d5d78..f0352fefa8bd 100644 --- a/ibis/expr/rewrites.py +++ b/ibis/expr/rewrites.py @@ -182,7 +182,7 @@ def window_merge_frames(_, frame): group_by = tuple(toolz.unique(_.frame.group_by + frame.group_by)) order_by = {} - for sort_key in _.frame.order_by + frame.order_by: + for sort_key in frame.order_by + _.frame.order_by: order_by[sort_key.expr] = sort_key.ascending order_by = tuple(ops.SortKey(k, v) for k, v in order_by.items())