diff --git a/.github/workflows/ibis-backends.yml b/.github/workflows/ibis-backends.yml index dc8fd09287ff..6cf7ed40f699 100644 --- a/.github/workflows/ibis-backends.yml +++ b/.github/workflows/ibis-backends.yml @@ -122,6 +122,12 @@ jobs: - postgres sys-deps: - libgeos-dev + - name: risingwave + title: Risingwave + services: + - risingwave + extras: + - risingwave - name: impala title: Impala serial: true @@ -211,6 +217,14 @@ jobs: - postgres sys-deps: - libgeos-dev + - os: windows-latest + backend: + name: risingwave + title: Risingwave + services: + - risingwave + extras: + - risingwave - os: windows-latest backend: name: postgres diff --git a/ci/schema/risingwave.sql b/ci/schema/risingwave.sql new file mode 100644 index 000000000000..251b689ada0d --- /dev/null +++ b/ci/schema/risingwave.sql @@ -0,0 +1,177 @@ +SET RW_IMPLICIT_FLUSH=true; + +DROP TABLE IF EXISTS "diamonds" CASCADE; + +CREATE TABLE "diamonds" ( + "carat" FLOAT, + "cut" TEXT, + "color" TEXT, + "clarity" TEXT, + "depth" FLOAT, + "table" FLOAT, + "price" BIGINT, + "x" FLOAT, + "y" FLOAT, + "z" FLOAT +) WITH ( + connector = 'posix_fs', + match_pattern = 'diamonds.csv', + posix_fs.root = '/data', +) FORMAT PLAIN ENCODE CSV ( without_header = 'false', delimiter = ',' ); + +DROP TABLE IF EXISTS "astronauts" CASCADE; + +CREATE TABLE "astronauts" ( + "id" BIGINT, + "number" BIGINT, + "nationwide_number" BIGINT, + "name" VARCHAR, + "original_name" VARCHAR, + "sex" VARCHAR, + "year_of_birth" BIGINT, + "nationality" VARCHAR, + "military_civilian" VARCHAR, + "selection" VARCHAR, + "year_of_selection" BIGINT, + "mission_number" BIGINT, + "total_number_of_missions" BIGINT, + "occupation" VARCHAR, + "year_of_mission" BIGINT, + "mission_title" VARCHAR, + "ascend_shuttle" VARCHAR, + "in_orbit" VARCHAR, + "descend_shuttle" VARCHAR, + "hours_mission" DOUBLE PRECISION, + "total_hrs_sum" DOUBLE PRECISION, + "field21" BIGINT, + "eva_hrs_mission" DOUBLE PRECISION, + "total_eva_hrs" DOUBLE PRECISION +) WITH ( + connector = 'posix_fs', + match_pattern = 'astronauts.csv', + posix_fs.root = '/data', +) FORMAT PLAIN ENCODE CSV ( without_header = 'false', delimiter = ',' ); + +DROP TABLE IF EXISTS "batting" CASCADE; + +CREATE TABLE "batting" ( + "playerID" TEXT, + "yearID" BIGINT, + "stint" BIGINT, + "teamID" TEXT, + "lgID" TEXT, + "G" BIGINT, + "AB" BIGINT, + "R" BIGINT, + "H" BIGINT, + "X2B" BIGINT, + "X3B" BIGINT, + "HR" BIGINT, + "RBI" BIGINT, + "SB" BIGINT, + "CS" BIGINT, + "BB" BIGINT, + "SO" BIGINT, + "IBB" BIGINT, + "HBP" BIGINT, + "SH" BIGINT, + "SF" BIGINT, + "GIDP" BIGINT +) WITH ( + connector = 'posix_fs', + match_pattern = 'batting.csv', + posix_fs.root = '/data', +) FORMAT PLAIN ENCODE CSV ( without_header = 'false', delimiter = ',' ); + +DROP TABLE IF EXISTS "awards_players" CASCADE; + +CREATE TABLE "awards_players" ( + "playerID" TEXT, + "awardID" TEXT, + "yearID" BIGINT, + "lgID" TEXT, + "tie" TEXT, + "notes" TEXT +) WITH ( + connector = 'posix_fs', + match_pattern = 'awards_players.csv', + posix_fs.root = '/data', +) FORMAT PLAIN ENCODE CSV ( without_header = 'false', delimiter = ',' ); + +DROP TABLE IF EXISTS "functional_alltypes" CASCADE; + +CREATE TABLE "functional_alltypes" ( + "id" INTEGER, + "bool_col" BOOLEAN, + "tinyint_col" SMALLINT, + "smallint_col" SMALLINT, + "int_col" INTEGER, + "bigint_col" BIGINT, + "float_col" REAL, + "double_col" DOUBLE PRECISION, + "date_string_col" TEXT, + "string_col" TEXT, + "timestamp_col" TIMESTAMP WITHOUT TIME ZONE, + "year" INTEGER, + "month" INTEGER +) WITH ( + connector = 'posix_fs', + match_pattern = 'functional_alltypes.csv', + posix_fs.root = '/data', +) FORMAT PLAIN ENCODE CSV ( without_header = 'false', delimiter = ',' ); + +DROP TABLE IF EXISTS "tzone" CASCADE; + +CREATE TABLE "tzone" ( + "ts" TIMESTAMP WITH TIME ZONE, + "key" TEXT, + "value" DOUBLE PRECISION +); + +INSERT INTO "tzone" + SELECT + CAST('2017-05-28 11:01:31.000400' AS TIMESTAMP WITH TIME ZONE) + + t * INTERVAL '1 day 1 second' AS "ts", + CHR(97 + t) AS "key", + t + t / 10.0 AS "value" + FROM generate_series(0, 9) AS "t"; + +DROP TABLE IF EXISTS "array_types" CASCADE; + +CREATE TABLE IF NOT EXISTS "array_types" ( + "x" BIGINT[], + "y" TEXT[], + "z" DOUBLE PRECISION[], + "grouper" TEXT, + "scalar_column" DOUBLE PRECISION, + "multi_dim" BIGINT[][] +); + +INSERT INTO "array_types" VALUES + (ARRAY[1, 2, 3], ARRAY['a', 'b', 'c'], ARRAY[1.0, 2.0, 3.0], 'a', 1.0, ARRAY[ARRAY[NULL::BIGINT, NULL, NULL], ARRAY[1, 2, 3]]), + (ARRAY[4, 5], ARRAY['d', 'e'], ARRAY[4.0, 5.0], 'a', 2.0, ARRAY[]::BIGINT[][]), + (ARRAY[6, NULL], ARRAY['f', NULL], ARRAY[6.0, NULL], 'a', 3.0, ARRAY[NULL, ARRAY[]::BIGINT[], NULL]), + (ARRAY[NULL, 1, NULL], ARRAY[NULL, 'a', NULL], ARRAY[]::DOUBLE PRECISION[], 'b', 4.0, ARRAY[ARRAY[1], ARRAY[2], ARRAY[NULL::BIGINT], ARRAY[3]]), + (ARRAY[2, NULL, 3], ARRAY['b', NULL, 'c'], NULL, 'b', 5.0, NULL), + (ARRAY[4, NULL, NULL, 5], ARRAY['d', NULL, NULL, 'e'], ARRAY[4.0, NULL, NULL, 5.0], 'c', 6.0, ARRAY[ARRAY[1, 2, 3]]); + +DROP TABLE IF EXISTS "json_t" CASCADE; + +CREATE TABLE IF NOT EXISTS "json_t" ("js" JSONB); + +INSERT INTO "json_t" VALUES + ('{"a": [1,2,3,4], "b": 1}'), + ('{"a":null,"b":2}'), + ('{"a":"foo", "c":null}'), + ('null'), + ('[42,47,55]'), + ('[]'); + +DROP TABLE IF EXISTS "win" CASCADE; +CREATE TABLE "win" ("g" TEXT, "x" BIGINT, "y" BIGINT); +INSERT INTO "win" VALUES + ('a', 0, 3), + ('a', 1, 2), + ('a', 2, 0), + ('a', 3, 1), + ('a', 4, 1); diff --git a/ci/schema/trino.sql b/ci/schema/trino.sql index d6281f42ea0d..18477e441595 100644 --- a/ci/schema/trino.sql +++ b/ci/schema/trino.sql @@ -11,7 +11,7 @@ CREATE TABLE hive.default.diamonds ( "y" DOUBLE, "z" DOUBLE ) WITH ( - external_location = 's3a://warehouse/diamonds', + external_location = 's3a://trino/diamonds', format = 'PARQUET' ); @@ -45,7 +45,7 @@ CREATE TABLE hive.default.astronauts ( "eva_hrs_mission" REAL, "total_eva_hrs" REAL ) WITH ( - external_location = 's3a://warehouse/astronauts', + external_location = 's3a://trino/astronauts', format = 'PARQUET' ); @@ -77,7 +77,7 @@ CREATE TABLE hive.default.batting ( "SF" BIGINT, "GIDP" BIGINT ) WITH ( - external_location = 's3a://warehouse/batting', + external_location = 's3a://trino/batting', format = 'PARQUET' ); @@ -93,7 +93,7 @@ CREATE TABLE hive.default.awards_players ( "tie" VARCHAR, "notes" VARCHAR ) WITH ( - external_location = 's3a://warehouse/awards-players', + external_location = 's3a://trino/awards-players', format = 'PARQUET' ); @@ -116,7 +116,7 @@ CREATE TABLE hive.default.functional_alltypes ( "year" INTEGER, "month" INTEGER ) WITH ( - external_location = 's3a://warehouse/functional-alltypes', + external_location = 's3a://trino/functional-alltypes', format = 'PARQUET' ); CREATE OR REPLACE VIEW memory.default.functional_alltypes AS diff --git a/compose.yaml b/compose.yaml index 9d110ccbc7b6..bbddfc7abda5 100644 --- a/compose.yaml +++ b/compose.yaml @@ -94,21 +94,21 @@ services: - trino minio: - image: bitnami/minio:2024.1.16 + image: bitnami/minio:2024.1.18 environment: MINIO_ROOT_USER: accesskey MINIO_ROOT_PASSWORD: secretkey - MINIO_SKIP_CLIENT: yes + MINIO_SKIP_CLIENT: "yes" healthcheck: interval: 1s retries: 20 test: - CMD-SHELL - - mc ping --count 1 trino + - mc ready data && mc mb --ignore-existing data/trino data/risingwave networks: - trino + - risingwave volumes: - - minio:/data - $PWD/docker/minio/config.json:/.mc/config.json:ro hive-metastore: @@ -119,7 +119,7 @@ services: HIVE_METASTORE_JDBC_URL: jdbc:postgresql://hive-metastore-db:23456/metastore HIVE_METASTORE_USER: admin HIVE_METASTORE_PASSWORD: admin - HIVE_METASTORE_WAREHOUSE_DIR: s3://warehouse/ + HIVE_METASTORE_WAREHOUSE_DIR: s3://trino/ HIVE_METASTORE_USERS_IN_ADMIN_ROLE: "admin" S3_ENDPOINT: http://minio:9000 S3_ACCESS_KEY: accesskey @@ -538,6 +538,39 @@ services: networks: - impala + risingwave: + image: ghcr.io/risingwavelabs/risingwave:nightly-20240122 + command: "standalone --meta-opts=\" \ + --advertise-addr 0.0.0.0:5690 \ + --backend mem \ + --state-store hummock+minio://accesskey:secretkey@minio:9000/risingwave \ + --data-directory hummock_001\" \ + --compute-opts=\"--advertise-addr 0.0.0.0:5688 --role both\" \ + --frontend-opts=\"--listen-addr 0.0.0.0:4566 --advertise-addr 0.0.0.0:4566\" \ + --compactor-opts=\"--advertise-addr 0.0.0.0:6660\"" + ports: + - 4566:4566 + depends_on: + minio: + condition: service_healthy + volumes: + - risingwave:/data + environment: + RUST_BACKTRACE: "1" + ENABLE_TELEMETRY: "false" + healthcheck: + test: + - CMD-SHELL + - bash -c 'printf \"GET / HTTP/1.1\n\n\" > /dev/tcp/127.0.0.1/6660; exit $$?;' + - bash -c 'printf \"GET / HTTP/1.1\n\n\" > /dev/tcp/127.0.0.1/5688; exit $$?;' + - bash -c 'printf \"GET / HTTP/1.1\n\n\" > /dev/tcp/127.0.0.1/4566; exit $$?;' + - bash -c 'printf \"GET / HTTP/1.1\n\n\" > /dev/tcp/127.0.0.1/5690; exit $$?;' + interval: 1s + retries: 20 + restart: on-failure + networks: + - risingwave + networks: impala: # docker defaults to naming networks "$PROJECT_$NETWORK" but the Java Hive @@ -554,6 +587,7 @@ networks: oracle: exasol: flink: + risingwave: volumes: broker_var: @@ -569,6 +603,6 @@ volumes: mysql: oracle: postgres: - minio: exasol: impala: + risingwave: diff --git a/docker/minio/config.json b/docker/minio/config.json index df7c82ec0be8..f0064dd72aef 100644 --- a/docker/minio/config.json +++ b/docker/minio/config.json @@ -1,7 +1,7 @@ { "version": "10", "aliases": { - "trino": { + "data": { "url": "http://minio:9000", "accessKey": "accesskey", "secretKey": "secretkey", diff --git a/ibis/backends/base/__init__.py b/ibis/backends/base/__init__.py index 7411077514d1..a4077d8cf02a 100644 --- a/ibis/backends/base/__init__.py +++ b/ibis/backends/base/__init__.py @@ -41,6 +41,7 @@ "datafusion": "postgres", # closest match see https://github.com/ibis-project/ibis/pull/7303#discussion_r1350223901 "exasol": "oracle", + "risingwave": "postgres", } _SQLALCHEMY_TO_SQLGLOT_DIALECT = { diff --git a/ibis/backends/base/sqlglot/datatypes.py b/ibis/backends/base/sqlglot/datatypes.py index dc70f7b6e3ba..8852bdae8837 100644 --- a/ibis/backends/base/sqlglot/datatypes.py +++ b/ibis/backends/base/sqlglot/datatypes.py @@ -394,6 +394,24 @@ def _from_ibis_Map(cls, dtype: dt.Map) -> sge.DataType: return sge.DataType(this=typecode.HSTORE) +class RisingWaveType(PostgresType): + dialect = "risingwave" + + @classmethod + def _from_ibis_Timestamp(cls, dtype: dt.Timestamp) -> sge.DataType: + if dtype.timezone is not None: + return sge.DataType(this=typecode.TIMESTAMPTZ) + return sge.DataType(this=typecode.TIMESTAMP) + + @classmethod + def _from_ibis_Decimal(cls, dtype: dt.Decimal) -> sge.DataType: + return sge.DataType(this=typecode.DECIMAL) + + @classmethod + def _from_ibis_UUID(cls, dtype: dt.UUID) -> sge.DataType: + return sge.DataType(this=typecode.VARCHAR) + + class DataFusionType(PostgresType): unknown_type_strings = { "utf8": dt.string, diff --git a/ibis/backends/clickhouse/compiler.py b/ibis/backends/clickhouse/compiler.py index e4121a2ef9b2..6439fb99a5e2 100644 --- a/ibis/backends/clickhouse/compiler.py +++ b/ibis/backends/clickhouse/compiler.py @@ -209,24 +209,26 @@ def visit_Hash(self, op, *, arg): @visit_node.register(ops.HashBytes) def visit_HashBytes(self, op, *, arg, how): - supported_algorithms = frozenset( - ( - "MD5", - "halfMD5", - "SHA1", - "SHA224", - "SHA256", - "intHash32", - "intHash64", - "cityHash64", - "sipHash64", - "sipHash128", - ) - ) - if how not in supported_algorithms: + supported_algorithms = { + "md5": "MD5", + "MD5": "MD5", + "halfMD5": "halfMD5", + "SHA1": "SHA1", + "sha1": "SHA1", + "SHA224": "SHA224", + "sha224": "SHA224", + "SHA256": "SHA256", + "sha256": "SHA256", + "intHash32": "intHash32", + "intHash64": "intHash64", + "cityHash64": "cityHash64", + "sipHash64": "sipHash64", + "sipHash128": "sipHash128", + } + if (funcname := supported_algorithms.get(how)) is None: raise com.UnsupportedOperationError(f"Unsupported hash algorithm {how}") - return self.f[how](arg) + return self.f[funcname](arg) @visit_node.register(ops.IntervalFromInteger) def visit_IntervalFromInteger(self, op, *, arg, unit): diff --git a/ibis/backends/duckdb/compiler.py b/ibis/backends/duckdb/compiler.py index 4cdc8d59f1bb..71877fb4cff8 100644 --- a/ibis/backends/duckdb/compiler.py +++ b/ibis/backends/duckdb/compiler.py @@ -336,6 +336,13 @@ def visit_Quantile(self, op, *, arg, quantile, where): funcname = f"percentile_{suffix}" return self.agg[funcname](arg, quantile, where=where) + @visit_node.register(ops.HexDigest) + def visit_HexDigest(self, op, *, arg, how): + if how in ("md5", "sha256"): + return getattr(self.f, how)(arg) + else: + raise NotImplementedError(f"No available hashing function for {how}") + _SIMPLE_OPS = { ops.ArrayPosition: "list_indexof", diff --git a/ibis/backends/mssql/compiler.py b/ibis/backends/mssql/compiler.py index 294874e5606f..f45fd6953c65 100644 --- a/ibis/backends/mssql/compiler.py +++ b/ibis/backends/mssql/compiler.py @@ -376,6 +376,36 @@ def visit_Not(self, op, *, arg): return sge.FALSE if arg == sge.TRUE else sge.TRUE return self.if_(arg, 1, 0).eq(0) + @visit_node.register(ops.HashBytes) + def visit_HashBytes(self, op, *, arg, how): + if how in ("md5", "sha1"): + return self.f.hashbytes(how, arg) + elif how == "sha256": + return self.f.hashbytes("sha2_256", arg) + elif how == "sha512": + return self.f.hashbytes("sha2_512", arg) + else: + raise NotImplementedError(how) + + @visit_node.register(ops.HexDigest) + def visit_HexDigest(self, op, *, arg, how): + if how in ("md5", "sha1"): + hashbinary = self.f.hashbytes(how, arg) + elif how == "sha256": + hashbinary = self.f.hashbytes("sha2_256", arg) + elif how == "sha512": + hashbinary = self.f.hashbytes("sha2_512", arg) + else: + raise NotImplementedError(how) + + # mssql uppercases the hexdigest which is inconsistent with several other + # implementations and inconsistent with Python, so lowercase it. + return self.f.lower( + self.f.convert( + sge.Literal(this="VARCHAR(MAX)", is_string=False), hashbinary, 2 + ) + ) + @visit_node.register(ops.Any) @visit_node.register(ops.All) @visit_node.register(ops.ApproxMedian) diff --git a/ibis/backends/postgres/__init__.py b/ibis/backends/postgres/__init__.py index 0f51218c7f7c..36f194d5f258 100644 --- a/ibis/backends/postgres/__init__.py +++ b/ibis/backends/postgres/__init__.py @@ -44,6 +44,7 @@ def _verify_source_line(func_name: str, line: str): class Backend(SQLGlotBackend): name = "postgres" + dialect = "postgres" compiler = PostgresCompiler() supports_python_udfs = True @@ -106,7 +107,7 @@ def _register_in_memory_table(self, op: ops.InMemoryTable) -> None: schema = op.schema if null_columns := [col for col, dtype in schema.items() if dtype.is_null()]: raise exc.IbisTypeError( - "Postgres cannot yet reliably handle `null` typed columns; " + f"{self.name} cannot yet reliably handle `null` typed columns; " f"got null typed columns: {null_columns}" ) @@ -137,18 +138,18 @@ def _register_in_memory_table(self, op: ops.InMemoryTable) -> None: ), properties=sg.exp.Properties(expressions=[sge.TemporaryProperty()]), ) - create_stmt_sql = create_stmt.sql(self.name) + create_stmt_sql = create_stmt.sql(self.dialect) columns = schema.keys() df = op.data.to_frame() data = df.itertuples(index=False) cols = ", ".join( - ident.sql(self.name) + ident.sql(self.dialect) for ident in map(partial(sg.to_identifier, quoted=quoted), columns) ) specs = ", ".join(repeat("%s", len(columns))) table = sg.table(name, quoted=quoted) - sql = f"INSERT INTO {table.sql(self.name)} ({cols}) VALUES ({specs})" + sql = f"INSERT INTO {table.sql(self.dialect)} ({cols}) VALUES ({specs})" with self.begin() as cur: cur.execute(create_stmt_sql) extras.execute_batch(cur, sql, data, 128) @@ -314,7 +315,7 @@ def list_tables( .from_(sg.table("tables", db="information_schema")) .distinct() .where(*conditions) - .sql(self.name) + .sql(self.dialect) ) with self._safe_raw_sql(sql) as cur: @@ -447,10 +448,10 @@ def _compile_builtin_udf(self, udf_node: ops.ScalarUDF) -> None: """No op.""" def _compile_pyarrow_udf(self, udf_node: ops.ScalarUDF) -> None: - raise NotImplementedError("pyarrow UDFs are not supported in Postgres") + raise NotImplementedError(f"pyarrow UDFs are not supported in {self.name}") def _compile_pandas_udf(self, udf_node: ops.ScalarUDF) -> str: - raise NotImplementedError("pandas UDFs are not supported in Postgres") + raise NotImplementedError(f"pandas UDFs are not supported in {self.name}") def _define_udf_translation_rules(self, expr: ir.Expr) -> None: """No-op, these are defined in the compiler.""" @@ -535,11 +536,11 @@ def _metadata(self, query: str) -> Iterable[tuple[str, dt.DataType]]: create_stmt = sge.Create( kind="VIEW", this=sg.table(name), - expression=sg.parse_one(query, read=self.name), + expression=sg.parse_one(query, read=self.dialect), properties=sge.Properties(expressions=[sge.TemporaryProperty()]), ) drop_stmt = sge.Drop(kind="VIEW", this=sg.table(name), exists=True).sql( - self.name + self.dialect ) with self._safe_raw_sql(create_stmt): @@ -555,7 +556,7 @@ def create_schema( ) -> None: if database is not None and database != self.current_database: raise exc.UnsupportedOperationError( - "Postgres does not support creating a schema in a different database" + f"{self.name} does not support creating a schema in a different database" ) sql = sge.Create( kind="SCHEMA", this=sg.table(name, catalog=database), exists=force @@ -572,7 +573,7 @@ def drop_schema( ) -> None: if database is not None and database != self.current_database: raise exc.UnsupportedOperationError( - "Postgres does not support dropping a schema in a different database" + f"{self.name} does not support dropping a schema in a different database" ) sql = sge.Drop( @@ -620,7 +621,7 @@ def create_table( if database is not None and database != self.current_database: raise com.UnsupportedOperationError( - "Creating tables in other databases is not supported by Postgres" + f"Creating tables in other databases is not supported by {self.name}" ) else: database = None @@ -672,15 +673,15 @@ def create_table( this = sg.table(name, catalog=database, quoted=self.compiler.quoted) with self._safe_raw_sql(create_stmt) as cur: if query is not None: - insert_stmt = sge.Insert(this=table, expression=query).sql(self.name) + insert_stmt = sge.Insert(this=table, expression=query).sql(self.dialect) cur.execute(insert_stmt) if overwrite: cur.execute( - sge.Drop(kind="TABLE", this=this, exists=True).sql(self.name) + sge.Drop(kind="TABLE", this=this, exists=True).sql(self.dialect) ) cur.execute( - f"ALTER TABLE IF EXISTS {table.sql(self.name)} RENAME TO {this.sql(self.name)}" + f"ALTER TABLE IF EXISTS {table.sql(self.dialect)} RENAME TO {this.sql(self.dialect)}" ) if schema is None: @@ -700,7 +701,7 @@ def drop_table( ) -> None: if database is not None and database != self.current_database: raise com.UnsupportedOperationError( - "Droppping tables in other databases is not supported by Postgres" + f"Droppping tables in other databases is not supported by {self.name}" ) else: database = None @@ -721,7 +722,7 @@ def _safe_raw_sql(self, *args, **kwargs): def raw_sql(self, query: str | sg.Expression, **kwargs: Any) -> Any: with contextlib.suppress(AttributeError): - query = query.sql(dialect=self.name) + query = query.sql(dialect=self.dialect) con = self.con cursor = con.cursor() @@ -772,6 +773,6 @@ def truncate_table(self, name: str, database: str | None = None) -> None: database Schema name """ - ident = sg.table(name, db=database).sql(self.name) + ident = sg.table(name, db=database).sql(self.dialect) with self._safe_raw_sql(f"TRUNCATE TABLE {ident}"): pass diff --git a/ibis/backends/pyspark/compiler.py b/ibis/backends/pyspark/compiler.py index bc2bbf2b7584..b4e75c959735 100644 --- a/ibis/backends/pyspark/compiler.py +++ b/ibis/backends/pyspark/compiler.py @@ -457,6 +457,17 @@ def visit_JoinLink(self, op, **kwargs): def visit_Undefined(self, op, **_): raise com.OperationNotDefinedError(type(op).__name__) + @visit_node.register(ops.HexDigest) + def visit_HexDigest(self, op, *, arg, how): + if how == "md5": + return self.f.md5(arg) + elif how == "sha1": + return self.f.sha1(arg) + elif how in ("sha256", "sha512"): + return self.f.sha2(arg, int(how[-3:])) + else: + raise NotImplementedError(f"No available hashing function for {how}") + _SIMPLE_OPS = { ops.ArrayDistinct: "array_distinct", diff --git a/ibis/backends/risingwave/__init__.py b/ibis/backends/risingwave/__init__.py new file mode 100644 index 000000000000..4d17b045b1f4 --- /dev/null +++ b/ibis/backends/risingwave/__init__.py @@ -0,0 +1,290 @@ +"""Risingwave backend.""" + +from __future__ import annotations + +import atexit +from functools import partial +from itertools import repeat +from typing import TYPE_CHECKING + +import psycopg2 +import sqlglot as sg +import sqlglot.expressions as sge +from psycopg2 import extras + +import ibis +import ibis.common.exceptions as com +import ibis.expr.operations as ops +import ibis.expr.types as ir +from ibis import util +from ibis.backends.postgres import Backend as PostgresBackend +from ibis.backends.risingwave.compiler import RisingwaveCompiler +from ibis.backends.risingwave.dialect import RisingWave as RisingWaveDialect + +if TYPE_CHECKING: + import pandas as pd + import pyarrow as pa + + +def _verify_source_line(func_name: str, line: str): + if line.startswith("@"): + raise com.InvalidDecoratorError(func_name, line) + return line + + +class Backend(PostgresBackend): + name = "risingwave" + dialect = RisingWaveDialect + compiler = RisingwaveCompiler() + supports_python_udfs = False + + def do_connect( + self, + host: str | None = None, + user: str | None = None, + password: str | None = None, + port: int = 5432, + database: str | None = None, + schema: str | None = None, + ) -> None: + """Create an Ibis client connected to RisingWave database. + + Parameters + ---------- + host + Hostname + user + Username + password + Password + port + Port number + database + Database to connect to + schema + RisingWave schema to use. If `None`, use the default `search_path`. + + Examples + -------- + >>> import os + >>> import getpass + >>> import ibis + >>> host = os.environ.get("IBIS_TEST_POSTGRES_HOST", "localhost") + >>> user = os.environ.get("IBIS_TEST_POSTGRES_USER", getpass.getuser()) + >>> password = os.environ.get("IBIS_TEST_POSTGRES_PASSWORD") + >>> database = os.environ.get("IBIS_TEST_POSTGRES_DATABASE", "ibis_testing") + >>> con = connect(database=database, host=host, user=user, password=password) + >>> con.list_tables() # doctest: +ELLIPSIS + [...] + >>> t = con.table("functional_alltypes") + >>> t + PostgreSQLTable[table] + name: functional_alltypes + schema: + id : int32 + bool_col : boolean + tinyint_col : int16 + smallint_col : int16 + int_col : int32 + bigint_col : int64 + float_col : float32 + double_col : float64 + date_string_col : string + string_col : string + timestamp_col : timestamp + year : int32 + month : int32 + """ + + self.con = psycopg2.connect( + host=host, + port=port, + user=user, + password=password, + database=database, + options=(f"-csearch_path={schema}" * (schema is not None)) or None, + ) + + with self.begin() as cur: + cur.execute("SET TIMEZONE = UTC") + + self._temp_views = set() + + def create_table( + self, + name: str, + obj: pd.DataFrame | pa.Table | ir.Table | None = None, + *, + schema: ibis.Schema | None = None, + database: str | None = None, + temp: bool = False, + overwrite: bool = False, + ): + """Create a table in Risingwave. + + Parameters + ---------- + name + Name of the table to create + obj + The data with which to populate the table; optional, but at least + one of `obj` or `schema` must be specified + schema + The schema of the table to create; optional, but at least one of + `obj` or `schema` must be specified + database + The name of the database in which to create the table; if not + passed, the current database is used. + temp + Create a temporary table + overwrite + If `True`, replace the table if it already exists, otherwise fail + if the table exists + """ + if obj is None and schema is None: + raise ValueError("Either `obj` or `schema` must be specified") + + if database is not None and database != self.current_database: + raise com.UnsupportedOperationError( + f"Creating tables in other databases is not supported by {self.name}" + ) + else: + database = None + + properties = [] + + if temp: + properties.append(sge.TemporaryProperty()) + + if obj is not None: + if not isinstance(obj, ir.Expr): + table = ibis.memtable(obj) + else: + table = obj + + self._run_pre_execute_hooks(table) + + query = self._to_sqlglot(table) + else: + query = None + + column_defs = [ + sge.ColumnDef( + this=sg.to_identifier(colname, quoted=self.compiler.quoted), + kind=self.compiler.type_mapper.from_ibis(typ), + constraints=( + None + if typ.nullable + else [sge.ColumnConstraint(kind=sge.NotNullColumnConstraint())] + ), + ) + for colname, typ in (schema or table.schema()).items() + ] + + if overwrite: + temp_name = util.gen_name(f"{self.name}_table") + else: + temp_name = name + + table = sg.table(temp_name, catalog=database, quoted=self.compiler.quoted) + target = sge.Schema(this=table, expressions=column_defs) + + create_stmt = sge.Create( + kind="TABLE", + this=target, + properties=sge.Properties(expressions=properties), + ) + + this = sg.table(name, catalog=database, quoted=self.compiler.quoted) + with self._safe_raw_sql(create_stmt) as cur: + if query is not None: + insert_stmt = sge.Insert(this=table, expression=query).sql(self.dialect) + cur.execute(insert_stmt) + + if overwrite: + cur.execute( + sge.Drop(kind="TABLE", this=this, exists=True).sql(self.dialect) + ) + cur.execute( + f"ALTER TABLE {table.sql(self.dialect)} RENAME TO {this.sql(self.dialect)}" + ) + + if schema is None: + return self.table(name, schema=database) + + # preserve the input schema if it was provided + return ops.DatabaseTable( + name, schema=schema, source=self, namespace=ops.Namespace(database=database) + ).to_expr() + + def _get_temp_view_definition(self, name: str, definition): + drop = sge.Drop( + kind="VIEW", exists=True, this=sg.table(name), cascade=True + ).sql(self.dialect) + + create = sge.Create( + this=sg.to_identifier(name, quoted=self.compiler.quoted), + kind="VIEW", + expression=definition, + replace=False, + ).sql(self.dialect) + + atexit.register(self._clean_up_tmp_view, name) + return f"{drop}; {create}" + + def _clean_up_tmp_view(self, name: str) -> None: + drop = sge.Drop( + kind="VIEW", exists=True, this=sg.table(name), cascade=True + ).sql(self.dialect) + with self.begin() as bind: + bind.execute(drop) + + def _register_in_memory_table(self, op: ops.InMemoryTable) -> None: + schema = op.schema + if null_columns := [col for col, dtype in schema.items() if dtype.is_null()]: + raise com.IbisTypeError( + f"{self.name} cannot yet reliably handle `null` typed columns; " + f"got null typed columns: {null_columns}" + ) + + # only register if we haven't already done so + if (name := op.name) not in self.list_tables(): + quoted = self.compiler.quoted + column_defs = [ + sg.exp.ColumnDef( + this=sg.to_identifier(colname, quoted=quoted), + kind=self.compiler.type_mapper.from_ibis(typ), + constraints=( + None + if typ.nullable + else [ + sg.exp.ColumnConstraint( + kind=sg.exp.NotNullColumnConstraint() + ) + ] + ), + ) + for colname, typ in schema.items() + ] + + create_stmt = sg.exp.Create( + kind="TABLE", + this=sg.exp.Schema( + this=sg.to_identifier(name, quoted=quoted), expressions=column_defs + ), + ) + create_stmt_sql = create_stmt.sql(self.dialect) + + columns = schema.keys() + df = op.data.to_frame() + data = df.itertuples(index=False) + cols = ", ".join( + ident.sql(self.dialect) + for ident in map(partial(sg.to_identifier, quoted=quoted), columns) + ) + specs = ", ".join(repeat("%s", len(columns))) + table = sg.table(name, quoted=quoted) + sql = f"INSERT INTO {table.sql(self.dialect)} ({cols}) VALUES ({specs})" + with self.begin() as cur: + cur.execute(create_stmt_sql) + extras.execute_batch(cur, sql, data, 128) diff --git a/ibis/backends/risingwave/compiler.py b/ibis/backends/risingwave/compiler.py new file mode 100644 index 000000000000..5bc7bfef2f5b --- /dev/null +++ b/ibis/backends/risingwave/compiler.py @@ -0,0 +1,104 @@ +from __future__ import annotations + +from functools import singledispatchmethod + +import sqlglot.expressions as sge +from public import public + +import ibis.common.exceptions as com +import ibis.expr.datashape as ds +import ibis.expr.datatypes as dt +import ibis.expr.operations as ops +from ibis.backends.base.sqlglot.datatypes import RisingWaveType +from ibis.backends.postgres.compiler import PostgresCompiler +from ibis.backends.risingwave.dialect import RisingWave # noqa: F401 + + +@public +class RisingwaveCompiler(PostgresCompiler): + __slots__ = () + + dialect = "risingwave" + name = "risingwave" + type_mapper = RisingWaveType + + @singledispatchmethod + def visit_node(self, op, **kwargs): + return super().visit_node(op, **kwargs) + + @visit_node.register(ops.Correlation) + def visit_Correlation(self, op, *, left, right, how, where): + if how == "sample": + raise com.UnsupportedOperationError( + f"{self.name} only implements `pop` correlation coefficient" + ) + return super().visit_Correlation( + op, left=left, right=right, how=how, where=where + ) + + @visit_node.register(ops.TimestampTruncate) + @visit_node.register(ops.DateTruncate) + @visit_node.register(ops.TimeTruncate) + def visit_TimestampTruncate(self, op, *, arg, unit): + unit_mapping = { + "Y": "year", + "Q": "quarter", + "M": "month", + "W": "week", + "D": "day", + "h": "hour", + "m": "minute", + "s": "second", + "ms": "milliseconds", + "us": "microseconds", + } + + if (unit := unit_mapping.get(unit.short)) is None: + raise com.UnsupportedOperationError(f"Unsupported truncate unit {unit}") + + return self.f.date_trunc(unit, arg) + + @visit_node.register(ops.IntervalFromInteger) + def visit_IntervalFromInteger(self, op, *, arg, unit): + if op.arg.shape == ds.scalar: + return sge.Interval(this=arg, unit=self.v[unit.name]) + elif op.arg.shape == ds.columnar: + return arg * sge.Interval(this=sge.convert(1), unit=self.v[unit.name]) + else: + raise ValueError("Invalid shape for converting to interval") + + def visit_NonNullLiteral(self, op, *, value, dtype): + if dtype.is_binary(): + return self.cast("".join(map(r"\x{:0>2x}".format, value)), dt.binary) + elif dtype.is_date(): + return self.cast(value.isoformat(), dtype) + elif dtype.is_json(): + return sge.convert(str(value)) + return None + + @visit_node.register(ops.DateFromYMD) + @visit_node.register(ops.Mode) + def visit_Undefined(self, op, **_): + raise com.OperationNotDefinedError(type(op).__name__) + + +_SIMPLE_OPS = { + ops.First: "first_value", + ops.Last: "last_value", +} + +for _op, _name in _SIMPLE_OPS.items(): + assert isinstance(type(_op), type), type(_op) + if issubclass(_op, ops.Reduction): + + @RisingwaveCompiler.visit_node.register(_op) + def _fmt(self, op, *, _name: str = _name, where, **kw): + return self.agg[_name](*kw.values(), where=where) + + else: + + @RisingwaveCompiler.visit_node.register(_op) + def _fmt(self, op, *, _name: str = _name, **kw): + return self.f[_name](*kw.values()) + + setattr(RisingwaveCompiler, f"visit_{_op.__name__}", _fmt) diff --git a/ibis/backends/risingwave/dialect.py b/ibis/backends/risingwave/dialect.py new file mode 100644 index 000000000000..2237c2a4d188 --- /dev/null +++ b/ibis/backends/risingwave/dialect.py @@ -0,0 +1,35 @@ +from __future__ import annotations + +import sqlglot.expressions as sge +from sqlglot import generator +from sqlglot.dialects import Postgres + + +class RisingWave(Postgres): + # Need to disable timestamp precision + # No "or replace" allowed in create statements + # no "not null" clause for column constraints + + class Generator(generator.Generator): + SINGLE_STRING_INTERVAL = True + RENAME_TABLE_WITH_DB = False + LOCKING_READS_SUPPORTED = True + JOIN_HINTS = False + TABLE_HINTS = False + QUERY_HINTS = False + NVL2_SUPPORTED = False + PARAMETER_TOKEN = "$" + TABLESAMPLE_SIZE_IS_ROWS = False + TABLESAMPLE_SEED_KEYWORD = "REPEATABLE" + SUPPORTS_SELECT_INTO = True + JSON_TYPE_REQUIRED_FOR_EXTRACTION = True + SUPPORTS_UNLOGGED_TABLES = True + + TYPE_MAPPING = { + **Postgres.Generator.TYPE_MAPPING, + sge.DataType.Type.TIMESTAMPTZ: "TIMESTAMPTZ", + } + + TRANSFORMS = { + **Postgres.Generator.TRANSFORMS, + } diff --git a/ibis/backends/risingwave/tests/__init__.py b/ibis/backends/risingwave/tests/__init__.py new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/ibis/backends/risingwave/tests/conftest.py b/ibis/backends/risingwave/tests/conftest.py new file mode 100644 index 000000000000..4ffb2ab85722 --- /dev/null +++ b/ibis/backends/risingwave/tests/conftest.py @@ -0,0 +1,87 @@ +from __future__ import annotations + +import os +from typing import TYPE_CHECKING, Any + +import pytest + +import ibis +from ibis.backends.tests.base import ServiceBackendTest + +if TYPE_CHECKING: + from collections.abc import Iterable + from pathlib import Path + +PG_USER = os.environ.get("IBIS_TEST_RISINGWAVE_USER", os.environ.get("PGUSER", "root")) +PG_PASS = os.environ.get( + "IBIS_TEST_RISINGWAVE_PASSWORD", os.environ.get("PGPASSWORD", "") +) +PG_HOST = os.environ.get( + "IBIS_TEST_RISINGWAVE_HOST", os.environ.get("PGHOST", "localhost") +) +PG_PORT = os.environ.get("IBIS_TEST_RISINGWAVE_PORT", os.environ.get("PGPORT", 4566)) +IBIS_TEST_RISINGWAVE_DB = os.environ.get( + "IBIS_TEST_RISINGWAVE_DATABASE", os.environ.get("PGDATABASE", "dev") +) + + +class TestConf(ServiceBackendTest): + # postgres rounds half to even for double precision and half away from zero + # for numeric and decimal + + returned_timestamp_unit = "s" + supports_structs = False + rounding_method = "half_to_even" + service_name = "risingwave" + deps = ("psycopg2",) + + @property + def test_files(self) -> Iterable[Path]: + return self.data_dir.joinpath("csv").glob("*.csv") + + def _load_data(self, **_: Any) -> None: + """Load test data into a PostgreSQL backend instance. + + Parameters + ---------- + data_dir + Location of test data + script_dir + Location of scripts defining schemas + """ + with self.connection._safe_raw_sql(";".join(self.ddl_script)): + pass + + @staticmethod + def connect(*, tmpdir, worker_id, port: int | None = None, **kw): + con = ibis.risingwave.connect( + host=PG_HOST, + port=port or PG_PORT, + user=PG_USER, + password=PG_PASS, + database=IBIS_TEST_RISINGWAVE_DB, + **kw, + ) + cursor = con.raw_sql("SET RW_IMPLICIT_FLUSH TO true;") + cursor.close() + return con + + +@pytest.fixture(scope="session") +def con(tmp_path_factory, data_dir, worker_id): + return TestConf.load_data(data_dir, tmp_path_factory, worker_id).connection + + +@pytest.fixture(scope="module") +def alltypes(con): + return con.tables.functional_alltypes + + +@pytest.fixture(scope="module") +def df(alltypes): + return alltypes.execute() + + +@pytest.fixture(scope="module") +def intervals(con): + return con.table("intervals") diff --git a/ibis/backends/risingwave/tests/snapshots/test_client/test_compile_toplevel/out.sql b/ibis/backends/risingwave/tests/snapshots/test_client/test_compile_toplevel/out.sql new file mode 100644 index 000000000000..c0b4a0b83304 --- /dev/null +++ b/ibis/backends/risingwave/tests/snapshots/test_client/test_compile_toplevel/out.sql @@ -0,0 +1,3 @@ +SELECT + SUM("t0"."foo") AS "Sum(foo)" +FROM "t0" AS "t0" \ No newline at end of file diff --git a/ibis/backends/risingwave/tests/snapshots/test_functions/test_analytic_functions/out.sql b/ibis/backends/risingwave/tests/snapshots/test_functions/test_analytic_functions/out.sql new file mode 100644 index 000000000000..c00dec1bed25 --- /dev/null +++ b/ibis/backends/risingwave/tests/snapshots/test_functions/test_analytic_functions/out.sql @@ -0,0 +1,7 @@ +SELECT + RANK() OVER (ORDER BY t0.double_col ASC) - 1 AS rank, + DENSE_RANK() OVER (ORDER BY t0.double_col ASC) - 1 AS dense_rank, + CUME_DIST() OVER (ORDER BY t0.double_col ASC) AS cume_dist, + NTILE(7) OVER (ORDER BY t0.double_col ASC) - 1 AS ntile, + PERCENT_RANK() OVER (ORDER BY t0.double_col ASC) AS percent_rank +FROM functional_alltypes AS t0 \ No newline at end of file diff --git a/ibis/backends/risingwave/tests/snapshots/test_functions/test_union_cte/False/out.sql b/ibis/backends/risingwave/tests/snapshots/test_functions/test_union_cte/False/out.sql new file mode 100644 index 000000000000..f0366d83444d --- /dev/null +++ b/ibis/backends/risingwave/tests/snapshots/test_functions/test_union_cte/False/out.sql @@ -0,0 +1 @@ +WITH "t1" AS ( SELECT "t0"."string_col", SUM("t0"."double_col") AS "metric" FROM "functional_alltypes" AS "t0" GROUP BY 1 ) SELECT "t7"."string_col", "t7"."metric" FROM ( SELECT "t5"."string_col", "t5"."metric" FROM ( SELECT * FROM "t1" AS "t2" UNION ALL SELECT * FROM "t1" AS "t4" ) AS "t5" UNION ALL SELECT * FROM "t1" AS "t3" ) AS "t7" \ No newline at end of file diff --git a/ibis/backends/risingwave/tests/snapshots/test_functions/test_union_cte/True/out.sql b/ibis/backends/risingwave/tests/snapshots/test_functions/test_union_cte/True/out.sql new file mode 100644 index 000000000000..5a873785e92b --- /dev/null +++ b/ibis/backends/risingwave/tests/snapshots/test_functions/test_union_cte/True/out.sql @@ -0,0 +1 @@ +WITH "t1" AS ( SELECT "t0"."string_col", SUM("t0"."double_col") AS "metric" FROM "functional_alltypes" AS "t0" GROUP BY 1 ) SELECT "t7"."string_col", "t7"."metric" FROM ( SELECT "t5"."string_col", "t5"."metric" FROM ( SELECT * FROM "t1" AS "t2" UNION SELECT * FROM "t1" AS "t4" ) AS "t5" UNION SELECT * FROM "t1" AS "t3" ) AS "t7" \ No newline at end of file diff --git a/ibis/backends/risingwave/tests/test_client.py b/ibis/backends/risingwave/tests/test_client.py new file mode 100644 index 000000000000..918b648b7bc8 --- /dev/null +++ b/ibis/backends/risingwave/tests/test_client.py @@ -0,0 +1,118 @@ +from __future__ import annotations + +import os + +import pandas as pd +import pytest +import sqlglot as sg +from pytest import param + +import ibis +import ibis.expr.datatypes as dt +import ibis.expr.types as ir +from ibis.util import gen_name + +pytest.importorskip("psycopg2") + +RISINGWAVE_TEST_DB = os.environ.get("IBIS_TEST_RISINGWAVE_DATABASE", "dev") +IBIS_RISINGWAVE_HOST = os.environ.get("IBIS_TEST_RISINGWAVE_HOST", "localhost") +IBIS_RISINGWAVE_PORT = os.environ.get("IBIS_TEST_RISINGWAVE_PORT", "4566") +IBIS_RISINGWAVE_USER = os.environ.get("IBIS_TEST_RISINGWAVE_USER", "root") +IBIS_RISINGWAVE_PASS = os.environ.get("IBIS_TEST_RISINGWAVE_PASSWORD", "") + + +def test_table(alltypes): + assert isinstance(alltypes, ir.Table) + + +def test_array_execute(alltypes): + d = alltypes.limit(10).double_col + s = d.execute() + assert isinstance(s, pd.Series) + assert len(s) == 10 + + +def test_literal_execute(con): + expr = ibis.literal("1234") + result = con.execute(expr) + assert result == "1234" + + +def test_simple_aggregate_execute(alltypes): + d = alltypes.double_col.sum() + v = d.execute() + assert isinstance(v, float) + + +def test_list_tables(con): + assert con.list_tables() + assert len(con.list_tables(like="functional")) == 1 + + +def test_compile_toplevel(snapshot): + t = ibis.table([("foo", "double")], name="t0") + + expr = t.foo.sum() + result = ibis.postgres.compile(expr) + snapshot.assert_match(str(result), "out.sql") + + +def test_list_databases(con): + assert RISINGWAVE_TEST_DB is not None + assert RISINGWAVE_TEST_DB in con.list_databases() + + +def test_create_and_drop_table(con, temp_table): + sch = ibis.schema([("first_name", "string")]) + + con.create_table(temp_table, schema=sch) + assert con.table(temp_table) is not None + + con.drop_table(temp_table) + + assert temp_table not in con.list_tables() + + +@pytest.mark.parametrize( + ("pg_type", "expected_type"), + [ + param(pg_type, ibis_type, id=pg_type.lower()) + for (pg_type, ibis_type) in [ + ("boolean", dt.boolean), + ("bytea", dt.binary), + ("bigint", dt.int64), + ("smallint", dt.int16), + ("integer", dt.int32), + ("text", dt.string), + ("real", dt.float32), + ("double precision", dt.float64), + ("character varying", dt.string), + ("date", dt.date), + ("time", dt.time), + ("time without time zone", dt.time), + ("timestamp without time zone", dt.Timestamp(scale=6)), + ("timestamp with time zone", dt.Timestamp("UTC", scale=6)), + ("interval", dt.Interval("s")), + ("numeric", dt.decimal), + ("jsonb", dt.json), + ] + ], +) +def test_get_schema_from_query(con, pg_type, expected_type): + name = sg.table(gen_name("risingwave_temp_table"), quoted=True) + with con.begin() as c: + c.execute(f"CREATE TABLE {name} (x {pg_type}, y {pg_type}[])") + expected_schema = ibis.schema(dict(x=expected_type, y=dt.Array(expected_type))) + result_schema = con._get_schema_using_query(f"SELECT x, y FROM {name}") + assert result_schema == expected_schema + with con.begin() as c: + c.execute(f"DROP TABLE {name}") + + +def test_insert_with_cte(con): + X = con.create_table("X", schema=ibis.schema(dict(id="int")), temp=False) + expr = X.join(X.mutate(a=X["id"] + 1), ["id"]) + Y = con.create_table("Y", expr, temp=False) + assert Y.execute().empty + con.drop_table("Y") + con.drop_table("X") diff --git a/ibis/backends/risingwave/tests/test_functions.py b/ibis/backends/risingwave/tests/test_functions.py new file mode 100644 index 000000000000..d680fb3190f9 --- /dev/null +++ b/ibis/backends/risingwave/tests/test_functions.py @@ -0,0 +1,862 @@ +from __future__ import annotations + +import operator +import warnings +from datetime import datetime + +import numpy as np +import pandas as pd +import pandas.testing as tm +import pytest +from pytest import param + +import ibis +import ibis.expr.datatypes as dt +from ibis import literal as L + +pytest.importorskip("psycopg2") + + +@pytest.mark.parametrize(("value", "expected"), [(0, None), (5.5, 5.5)]) +def test_nullif_zero(con, value, expected): + assert con.execute(L(value).nullif(0)) == expected + + +@pytest.mark.parametrize(("value", "expected"), [("foo_bar", 7), ("", 0)]) +def test_string_length(con, value, expected): + assert con.execute(L(value).length()) == expected + + +@pytest.mark.parametrize( + ("op", "expected"), + [ + param(operator.methodcaller("left", 3), "foo", id="left"), + param(operator.methodcaller("right", 3), "bar", id="right"), + param(operator.methodcaller("substr", 0, 3), "foo", id="substr_0_3"), + param(operator.methodcaller("substr", 4, 3), "bar", id="substr_4, 3"), + param(operator.methodcaller("substr", 1), "oo_bar", id="substr_1"), + ], +) +def test_string_substring(con, op, expected): + value = L("foo_bar") + assert con.execute(op(value)) == expected + + +@pytest.mark.parametrize( + ("opname", "expected"), + [("lstrip", "foo "), ("rstrip", " foo"), ("strip", "foo")], +) +def test_string_strip(con, opname, expected): + op = operator.methodcaller(opname) + value = L(" foo ") + assert con.execute(op(value)) == expected + + +@pytest.mark.parametrize( + ("opname", "count", "char", "expected"), + [("lpad", 6, " ", " foo"), ("rpad", 6, " ", "foo ")], +) +def test_string_pad(con, opname, count, char, expected): + op = operator.methodcaller(opname, count, char) + value = L("foo") + assert con.execute(op(value)) == expected + + +def test_string_reverse(con): + assert con.execute(L("foo").reverse()) == "oof" + + +def test_string_upper(con): + assert con.execute(L("foo").upper()) == "FOO" + + +def test_string_lower(con): + assert con.execute(L("FOO").lower()) == "foo" + + +@pytest.mark.parametrize( + ("haystack", "needle", "expected"), + [ + ("foobar", "bar", True), + ("foobar", "foo", True), + ("foobar", "baz", False), + ("100%", "%", True), + ("a_b_c", "_", True), + ], +) +def test_string_contains(con, haystack, needle, expected): + value = L(haystack) + expr = value.contains(needle) + assert con.execute(expr) == expected + + +@pytest.mark.parametrize( + ("value", "expected"), + [("foo bar foo", "Foo Bar Foo"), ("foobar Foo", "Foobar Foo")], +) +def test_capitalize(con, value, expected): + assert con.execute(L(value).capitalize()) == expected + + +def test_repeat(con): + expr = L("bar ").repeat(3) + assert con.execute(expr) == "bar bar bar " + + +def test_re_replace(con): + expr = L("fudge|||chocolate||candy").re_replace("\\|{2,3}", ", ") + assert con.execute(expr) == "fudge, chocolate, candy" + + +def test_translate(con): + expr = L("faab").translate("a", "b") + assert con.execute(expr) == "fbbb" + + +@pytest.mark.parametrize( + ("raw_value", "expected"), [("a", 0), ("b", 1), ("d", -1), (None, 3)] +) +def test_find_in_set(con, raw_value, expected): + value = L(raw_value, dt.string) + haystack = ["a", "b", "c", None] + expr = value.find_in_set(haystack) + assert con.execute(expr) == expected + + +@pytest.mark.parametrize( + ("raw_value", "opname", "expected"), + [ + (None, "isnull", True), + (1, "isnull", False), + (None, "notnull", False), + (1, "notnull", True), + ], +) +def test_isnull_notnull(con, raw_value, opname, expected): + lit = L(raw_value) + op = operator.methodcaller(opname) + expr = op(lit) + assert con.execute(expr) == expected + + +@pytest.mark.parametrize( + ("expr", "expected"), + [ + param(L("foobar").find("bar"), 3, id="find_pos"), + param(L("foobar").find("baz"), -1, id="find_neg"), + param(L("foobar").like("%bar"), True, id="like_left_pattern"), + param(L("foobar").like("foo%"), True, id="like_right_pattern"), + param(L("foobar").like("%baz%"), False, id="like_both_sides_pattern"), + param(L("foobar").like(["%bar"]), True, id="like_list_left_side"), + param(L("foobar").like(["foo%"]), True, id="like_list_right_side"), + param(L("foobar").like(["%baz%"]), False, id="like_list_both_sides"), + param(L("foobar").like(["%bar", "foo%"]), True, id="like_list_multiple"), + param(L("foobarfoo").replace("foo", "H"), "HbarH", id="replace"), + param(L("a").ascii_str(), ord("a"), id="ascii_str"), + ], +) +def test_string_functions(con, expr, expected): + assert con.execute(expr) == expected + + +@pytest.mark.parametrize( + ("expr", "expected"), + [ + param(L("abcd").re_search("[a-z]"), True, id="re_search_match"), + param(L("abcd").re_search(r"[\d]+"), False, id="re_search_no_match"), + param(L("1222").re_search(r"[\d]+"), True, id="re_search_match_number"), + ], +) +def test_regexp(con, expr, expected): + assert con.execute(expr) == expected + + +@pytest.mark.parametrize( + ("expr", "expected"), + [ + param(ibis.NA.fillna(5), 5, id="filled"), + param(L(5).fillna(10), 5, id="not_filled"), + param(L(5).nullif(5), None, id="nullif_null"), + param(L(10).nullif(5), 10, id="nullif_not_null"), + ], +) +def test_fillna_nullif(con, expr, expected): + assert con.execute(expr) == expected + + +@pytest.mark.parametrize( + ("expr", "expected"), + [ + param(ibis.coalesce(5, None, 4), 5, id="first"), + param(ibis.coalesce(ibis.NA, 4, ibis.NA), 4, id="second"), + param(ibis.coalesce(ibis.NA, ibis.NA, 3.14), 3.14, id="third"), + ], +) +def test_coalesce(con, expr, expected): + assert con.execute(expr) == expected + + +@pytest.mark.parametrize( + ("expr", "expected"), + [ + param(ibis.coalesce(ibis.NA, ibis.NA), None, id="all_null"), + param( + ibis.coalesce( + ibis.NA.cast("int8"), + ibis.NA.cast("int8"), + ibis.NA.cast("int8"), + ), + None, + id="all_nulls_with_all_cast", + ), + ], +) +def test_coalesce_all_na(con, expr, expected): + assert con.execute(expr) is None + + +def test_coalesce_all_na_double(con): + expr = ibis.coalesce(ibis.NA, ibis.NA, ibis.NA.cast("double")) + assert np.isnan(con.execute(expr)) + + +def test_numeric_builtins_work(alltypes, df): + expr = alltypes.double_col.fillna(0) + result = expr.execute() + expected = df.double_col.fillna(0) + expected.name = "Coalesce()" + tm.assert_series_equal(result, expected) + + +@pytest.mark.parametrize( + ("op", "pandas_op"), + [ + param( + lambda t: (t.double_col > 20).ifelse(10, -20), + lambda df: pd.Series(np.where(df.double_col > 20, 10, -20), dtype="int8"), + id="simple", + ), + param( + lambda t: (t.double_col > 20).ifelse(10, -20).abs(), + lambda df: pd.Series( + np.where(df.double_col > 20, 10, -20), dtype="int8" + ).abs(), + id="abs", + ), + ], +) +def test_ifelse(alltypes, df, op, pandas_op): + expr = op(alltypes) + result = expr.execute() + result.name = None + expected = pandas_op(df) + tm.assert_series_equal(result, expected) + + +@pytest.mark.parametrize( + ("func", "pandas_func"), + [ + # tier and histogram + param( + lambda d: d.bucket([0, 10, 25, 50, 100]), + lambda s: pd.cut(s, [0, 10, 25, 50, 100], right=False, labels=False).astype( + "int8" + ), + id="include_over_false", + ), + param( + lambda d: d.bucket([0, 10, 25, 50], include_over=True), + lambda s: pd.cut( + s, [0, 10, 25, 50, np.inf], right=False, labels=False + ).astype("int8"), + id="include_over_true", + ), + param( + lambda d: d.bucket([0, 10, 25, 50], close_extreme=False), + lambda s: pd.cut(s, [0, 10, 25, 50], right=False, labels=False), + id="close_extreme_false", + ), + param( + lambda d: d.bucket([0, 10, 25, 50], closed="right", close_extreme=False), + lambda s: pd.cut( + s, + [0, 10, 25, 50], + include_lowest=False, + right=True, + labels=False, + ), + id="closed_right", + ), + param( + lambda d: d.bucket([10, 25, 50, 100], include_under=True), + lambda s: pd.cut(s, [0, 10, 25, 50, 100], right=False, labels=False).astype( + "int8" + ), + id="include_under_true", + ), + ], +) +def test_bucket(alltypes, df, func, pandas_func): + expr = func(alltypes.double_col) + result = expr.execute() + expected = pandas_func(df.double_col) + tm.assert_series_equal(result, expected, check_names=False) + + +def test_category_label(alltypes, df): + t = alltypes + d = t.double_col + + bins = [0, 10, 25, 50, 100] + labels = ["a", "b", "c", "d"] + bucket = d.bucket(bins) + expr = bucket.label(labels) + result = expr.execute() + + with warnings.catch_warnings(): + warnings.simplefilter("ignore") + result = pd.Series(pd.Categorical(result, ordered=True)) + + result.name = "double_col" + + expected = pd.cut(df.double_col, bins, labels=labels, right=False) + + tm.assert_series_equal(result, expected) + + +@pytest.mark.parametrize("distinct", [True, False]) +def test_union_cte(alltypes, distinct, snapshot): + t = alltypes + expr1 = t.group_by(t.string_col).aggregate(metric=t.double_col.sum()) + expr2 = expr1.view() + expr3 = expr1.view() + expr = expr1.union(expr2, distinct=distinct).union(expr3, distinct=distinct) + result = " ".join(line.strip() for line in expr.compile().splitlines()) + snapshot.assert_match(result, "out.sql") + + +@pytest.mark.parametrize( + ("func", "pandas_func"), + [ + param( + lambda t, cond: t.bool_col.count(), + lambda df, cond: df.bool_col.count(), + id="count", + ), + param( + lambda t, cond: t.double_col.mean(), + lambda df, cond: df.double_col.mean(), + id="mean", + ), + param( + lambda t, cond: t.double_col.min(), + lambda df, cond: df.double_col.min(), + id="min", + ), + param( + lambda t, cond: t.double_col.max(), + lambda df, cond: df.double_col.max(), + id="max", + ), + param( + lambda t, cond: t.double_col.var(), + lambda df, cond: df.double_col.var(), + id="var", + ), + param( + lambda t, cond: t.double_col.std(), + lambda df, cond: df.double_col.std(), + id="std", + ), + param( + lambda t, cond: t.double_col.var(how="sample"), + lambda df, cond: df.double_col.var(ddof=1), + id="samp_var", + ), + param( + lambda t, cond: t.double_col.std(how="pop"), + lambda df, cond: df.double_col.std(ddof=0), + id="pop_std", + ), + param( + lambda t, cond: t.bool_col.count(where=cond), + lambda df, cond: df.bool_col[cond].count(), + id="count_where", + ), + param( + lambda t, cond: t.double_col.mean(where=cond), + lambda df, cond: df.double_col[cond].mean(), + id="mean_where", + ), + param( + lambda t, cond: t.double_col.min(where=cond), + lambda df, cond: df.double_col[cond].min(), + id="min_where", + ), + param( + lambda t, cond: t.double_col.max(where=cond), + lambda df, cond: df.double_col[cond].max(), + id="max_where", + ), + param( + lambda t, cond: t.double_col.var(where=cond), + lambda df, cond: df.double_col[cond].var(), + id="var_where", + ), + param( + lambda t, cond: t.double_col.std(where=cond), + lambda df, cond: df.double_col[cond].std(), + id="std_where", + ), + param( + lambda t, cond: t.double_col.var(where=cond, how="sample"), + lambda df, cond: df.double_col[cond].var(), + id="samp_var_where", + ), + param( + lambda t, cond: t.double_col.std(where=cond, how="pop"), + lambda df, cond: df.double_col[cond].std(ddof=0), + id="pop_std_where", + ), + ], +) +def test_aggregations(alltypes, df, func, pandas_func): + table = alltypes.limit(100) + df = df.head(table.count().execute()) + + cond = table.string_col.isin(["1", "7"]) + expr = func(table, cond) + result = expr.execute() + expected = pandas_func(df, cond.execute()) + + np.testing.assert_allclose(result, expected) + + +def test_not_contains(alltypes, df): + n = 100 + table = alltypes.limit(n) + expr = table.string_col.notin(["1", "7"]) + result = expr.execute() + expected = ~df.head(n).string_col.isin(["1", "7"]) + tm.assert_series_equal(result, expected, check_names=False) + + +def test_group_concat(alltypes, df): + expr = alltypes.string_col.group_concat() + result = expr.execute() + expected = ",".join(df.string_col.dropna()) + assert result == expected + + +def test_distinct_aggregates(alltypes, df): + expr = alltypes.limit(100).double_col.nunique() + result = expr.execute() + assert result == df.head(100).double_col.nunique() + + +def test_not_exists(alltypes, df): + t = alltypes + t2 = t.view() + + expr = t[~((t.string_col == t2.string_col).any())] + result = expr.execute() + + left, right = df, t2.execute() + expected = left[left.string_col != right.string_col] + + tm.assert_frame_equal(result, expected, check_index_type=False, check_dtype=False) + + +def test_subquery(alltypes, df): + t = alltypes + + expr = t.mutate(d=t.double_col.fillna(0)).limit(1000).group_by("string_col").size() + result = expr.execute().sort_values("string_col").reset_index(drop=True) + expected = ( + df.assign(d=df.double_col.fillna(0)) + .head(1000) + .groupby("string_col") + .string_col.count() + .rename("CountStar()") + .reset_index() + .sort_values("string_col") + .reset_index(drop=True) + ) + tm.assert_frame_equal(result, expected) + + +@pytest.mark.parametrize("func", ["mean", "sum", "min", "max"]) +def test_simple_window(alltypes, func, df): + t = alltypes + f = getattr(t.double_col, func) + df_f = getattr(df.double_col, func) + result = t.select((t.double_col - f()).name("double_col")).execute().double_col + expected = df.double_col - df_f() + tm.assert_series_equal(result, expected) + + +@pytest.mark.parametrize("func", ["mean", "sum", "min", "max"]) +@pytest.mark.xfail( + reason="Window function with empty PARTITION BY is not supported yet" +) +def test_rolling_window(alltypes, func, df): + t = alltypes + df = ( + df[["double_col", "timestamp_col"]] + .sort_values("timestamp_col") + .reset_index(drop=True) + ) + window = ibis.window(order_by=t.timestamp_col, preceding=6, following=0) + f = getattr(t.double_col, func) + df_f = getattr(df.double_col.rolling(7, min_periods=0), func) + result = t.select(f().over(window).name("double_col")).execute().double_col + expected = df_f() + tm.assert_series_equal(result, expected) + + +def test_rolling_window_with_mlb(alltypes): + t = alltypes + window = ibis.trailing_window( + preceding=ibis.rows_with_max_lookback(3, ibis.interval(days=5)), + order_by=t.timestamp_col, + ) + expr = t["double_col"].sum().over(window) + with pytest.raises(NotImplementedError): + expr.execute() + + +@pytest.mark.parametrize("func", ["mean", "sum", "min", "max"]) +@pytest.mark.xfail( + reason="Window function with empty PARTITION BY is not supported yet" +) +def test_partitioned_window(alltypes, func, df): + t = alltypes + window = ibis.window( + group_by=t.string_col, + order_by=t.timestamp_col, + preceding=6, + following=0, + ) + + def roller(func): + def rolled(df): + torder = df.sort_values("timestamp_col") + rolling = torder.double_col.rolling(7, min_periods=0) + return getattr(rolling, func)() + + return rolled + + f = getattr(t.double_col, func) + expr = f().over(window).name("double_col") + result = t.select(expr).execute().double_col + expected = df.groupby("string_col").apply(roller(func)).reset_index(drop=True) + tm.assert_series_equal(result, expected) + + +@pytest.mark.parametrize("func", ["sum", "min", "max"]) +@pytest.mark.xfail( + reason="Window function with empty PARTITION BY is not supported yet" +) +def test_cumulative_simple_window(alltypes, func, df): + t = alltypes + f = getattr(t.double_col, func) + col = t.double_col - f().over(ibis.cumulative_window()) + expr = t.select(col.name("double_col")) + result = expr.execute().double_col + expected = df.double_col - getattr(df.double_col, "cum%s" % func)() + tm.assert_series_equal(result, expected) + + +@pytest.mark.parametrize("func", ["sum", "min", "max"]) +@pytest.mark.xfail( + reason="Window function with empty PARTITION BY is not supported yet" +) +def test_cumulative_ordered_window(alltypes, func, df): + t = alltypes + df = df.sort_values("timestamp_col").reset_index(drop=True) + window = ibis.cumulative_window(order_by=t.timestamp_col) + f = getattr(t.double_col, func) + expr = t.select((t.double_col - f().over(window)).name("double_col")) + result = expr.execute().double_col + expected = df.double_col - getattr(df.double_col, "cum%s" % func)() + tm.assert_series_equal(result, expected) + + +@pytest.mark.parametrize( + ("func", "shift_amount"), [("lead", -1), ("lag", 1)], ids=["lead", "lag"] +) +@pytest.mark.xfail( + reason="Window function with empty PARTITION BY is not supported yet" +) +def test_analytic_shift_functions(alltypes, df, func, shift_amount): + method = getattr(alltypes.double_col, func) + expr = method(1) + result = expr.execute().rename("double_col") + expected = df.double_col.shift(shift_amount) + tm.assert_series_equal(result, expected) + + +@pytest.mark.parametrize( + ("func", "expected_index"), [("first", -1), ("last", 0)], ids=["first", "last"] +) +@pytest.mark.xfail(reason="Unsupported expr: (first(t0.double_col) + 1) - 1") +def test_first_last_value(alltypes, df, func, expected_index): + col = alltypes.order_by(ibis.desc(alltypes.string_col)).double_col + method = getattr(col, func) + # test that we traverse into expression trees + expr = (1 + method()) - 1 + result = expr.execute() + expected = df.double_col.iloc[expected_index] + assert result == expected + + +def test_null_column(alltypes): + t = alltypes + nrows = t.count().execute() + expr = t.mutate(na_column=ibis.NA).na_column + result = expr.execute() + tm.assert_series_equal(result, pd.Series([None] * nrows, name="na_column")) + + +@pytest.mark.xfail( + reason="Window function with empty PARTITION BY is not supported yet" +) +def test_window_with_arithmetic(alltypes, df): + t = alltypes + w = ibis.window(order_by=t.timestamp_col) + expr = t.mutate(new_col=ibis.row_number().over(w) / 2) + + df = df[["timestamp_col"]].sort_values("timestamp_col").reset_index(drop=True) + expected = df.assign(new_col=[x / 2.0 for x in range(len(df))]) + result = expr["timestamp_col", "new_col"].execute() + tm.assert_frame_equal(result, expected) + + +def test_anonymous_aggregate(alltypes, df): + t = alltypes + expr = t[t.double_col > t.double_col.mean()] + result = expr.execute() + expected = df[df.double_col > df.double_col.mean()].reset_index(drop=True) + tm.assert_frame_equal(result, expected) + + +@pytest.fixture +def array_types(con): + return con.table("array_types") + + +def test_array_length(array_types): + expr = array_types.select( + array_types.x.length().name("x_length"), + array_types.y.length().name("y_length"), + array_types.z.length().name("z_length"), + ) + result = expr.execute() + expected = pd.DataFrame( + { + "x_length": [3, 2, 2, 3, 3, 4], + "y_length": [3, 2, 2, 3, 3, 4], + "z_length": [3, 2, 2, 0, None, 4], + } + ) + result_sorted = result.sort_values( + by=["x_length", "y_length", "z_length"], na_position="first" + ).reset_index(drop=True) + expected_sorted = expected.sort_values( + by=["x_length", "y_length", "z_length"], na_position="first" + ).reset_index(drop=True) + tm.assert_frame_equal(result_sorted, expected_sorted) + + +def custom_sort_none_first(arr): + return sorted(arr, key=lambda x: (x is not None, x)) + + +def test_head(con): + t = con.table("functional_alltypes") + result = t.head().execute() + expected = t.limit(5).execute() + tm.assert_frame_equal(result, expected) + + +def test_identical_to(con, df): + # TODO: abstract this testing logic out into parameterized fixtures + t = con.table("functional_alltypes") + dt = df[["tinyint_col", "double_col"]] + expr = t.tinyint_col.identical_to(t.double_col) + result = expr.execute() + expected = (dt.tinyint_col.isnull() & dt.double_col.isnull()) | ( + dt.tinyint_col == dt.double_col + ) + expected.name = result.name + tm.assert_series_equal(result, expected) + + +@pytest.mark.parametrize("opname", ["invert", "neg"]) +def test_not_and_negate_bool(con, opname, df): + op = getattr(operator, opname) + t = con.table("functional_alltypes").limit(10) + expr = t.select(op(t.bool_col).name("bool_col")) + result = expr.execute().bool_col + expected = op(df.head(10).bool_col) + tm.assert_series_equal(result, expected) + + +@pytest.mark.parametrize( + "field", + [ + "tinyint_col", + "smallint_col", + "int_col", + "bigint_col", + "float_col", + "double_col", + "year", + "month", + ], +) +def test_negate_non_boolean(con, field, df): + t = con.table("functional_alltypes").limit(10) + expr = t.select((-t[field]).name(field)) + result = expr.execute()[field] + expected = -df.head(10)[field] + tm.assert_series_equal(result, expected) + + +def test_negate_boolean(con, df): + t = con.table("functional_alltypes").limit(10) + expr = t.select((-t.bool_col).name("bool_col")) + result = expr.execute().bool_col + expected = -df.head(10).bool_col + tm.assert_series_equal(result, expected) + + +@pytest.mark.parametrize("opname", ["sum", "mean", "min", "max", "std", "var"]) +def test_boolean_reduction(alltypes, opname, df): + op = operator.methodcaller(opname) + expr = op(alltypes.bool_col) + result = expr.execute() + assert result == op(df.bool_col) + + +def test_timestamp_with_timezone(con): + t = con.table("tzone") + result = t.ts.execute() + assert str(result.dtype.tz) + + +@pytest.mark.parametrize( + ("left", "right", "type"), + [ + param( + L("2017-04-01 01:02:33"), + datetime(2017, 4, 1, 1, 3, 34), + dt.timestamp, + id="ibis_timestamp", + ), + param( + datetime(2017, 4, 1, 1, 3, 34), + L("2017-04-01 01:02:33"), + dt.timestamp, + id="python_datetime", + ), + ], +) +@pytest.mark.parametrize("opname", ["eq", "ne", "lt", "le", "gt", "ge"]) +def test_string_temporal_compare(con, opname, left, right, type): + op = getattr(operator, opname) + expr = op(left, right) + result = con.execute(expr) + left_raw = con.execute(L(left).cast(type)) + right_raw = con.execute(L(right).cast(type)) + expected = op(left_raw, right_raw) + assert result == expected + + +@pytest.mark.parametrize( + ("left", "right"), + [ + param( + L("2017-03-31 00:02:33").cast(dt.timestamp), + datetime(2017, 4, 1, 1, 3, 34), + id="ibis_timestamp", + ), + param( + datetime(2017, 3, 31, 0, 2, 33), + L("2017-04-01 01:03:34").cast(dt.timestamp), + id="python_datetime", + ), + ], +) +@pytest.mark.parametrize( + "op", + [ + param( + lambda left, right: ibis.timestamp("2017-04-01 00:02:34").between( + left, right + ), + id="timestamp", + ), + param( + lambda left, right: ( + ibis.timestamp("2017-04-01").cast(dt.date).between(left, right) + ), + id="date", + ), + ], +) +def test_string_temporal_compare_between(con, op, left, right): + expr = op(left, right) + result = con.execute(expr) + assert isinstance(result, (bool, np.bool_)) + assert result + + +@pytest.mark.xfail( + reason="function make_date(integer, integer, integer) does not exist" +) +def test_scalar_parameter(con): + start_string, end_string = "2009-03-01", "2010-07-03" + + start = ibis.param(dt.date) + end = ibis.param(dt.date) + t = con.table("functional_alltypes") + col = t.date_string_col.cast("date") + expr = col.between(start, end).name("res") + expected_expr = col.between(start_string, end_string).name("res") + + result = expr.execute(params={start: start_string, end: end_string}) + expected = expected_expr.execute() + tm.assert_series_equal(result, expected) + + +def test_string_to_binary_cast(con): + t = con.table("functional_alltypes").limit(10) + expr = t.string_col.cast("binary") + result = expr.execute() + name = expr.get_name() + sql_string = ( + f"SELECT decode(string_col, 'escape') AS \"{name}\" " + "FROM functional_alltypes LIMIT 10" + ) + with con.begin() as c: + c.execute(sql_string) + raw_data = [row[0][0] for row in c.fetchall()] + expected = pd.Series(raw_data, name=name) + tm.assert_series_equal(result, expected) + + +def test_string_to_binary_round_trip(con): + t = con.table("functional_alltypes").limit(10) + expr = t.string_col.cast("binary").cast("string") + result = expr.execute() + name = expr.get_name() + sql_string = ( + "SELECT encode(decode(string_col, 'escape'), 'escape') AS " + f'"{name}"' + "FROM functional_alltypes LIMIT 10" + ) + with con.begin() as c: + c.execute(sql_string) + expected = pd.Series([row[0][0] for row in c.fetchall()], name=name) + tm.assert_series_equal(result, expected) diff --git a/ibis/backends/risingwave/tests/test_json.py b/ibis/backends/risingwave/tests/test_json.py new file mode 100644 index 000000000000..18edda8e3741 --- /dev/null +++ b/ibis/backends/risingwave/tests/test_json.py @@ -0,0 +1,17 @@ +"""Tests for json data types.""" +from __future__ import annotations + +import json + +import pytest +from pytest import param + +import ibis + + +@pytest.mark.parametrize("data", [param({"status": True}, id="status")]) +def test_json(data, alltypes): + lit = ibis.literal(json.dumps(data), type="json").name("tmp") + expr = alltypes[[alltypes.id, lit]].head(1) + df = expr.execute() + assert df["tmp"].iloc[0] == data diff --git a/ibis/backends/tests/errors.py b/ibis/backends/tests/errors.py index 4cd3f96ecad9..37ddc4f48f17 100644 --- a/ibis/backends/tests/errors.py +++ b/ibis/backends/tests/errors.py @@ -91,14 +91,18 @@ try: from psycopg2.errors import DivisionByZero as PsycoPg2DivisionByZero from psycopg2.errors import IndeterminateDatatype as PsycoPg2IndeterminateDatatype + from psycopg2.errors import InternalError_ as PsycoPg2InternalError from psycopg2.errors import ( InvalidTextRepresentation as PsycoPg2InvalidTextRepresentation, ) + from psycopg2.errors import ProgrammingError as PsycoPg2ProgrammingError from psycopg2.errors import SyntaxError as PsycoPg2SyntaxError except ImportError: PsycoPg2SyntaxError = ( PsycoPg2IndeterminateDatatype - ) = PsycoPg2InvalidTextRepresentation = PsycoPg2DivisionByZero = None + ) = ( + PsycoPg2InvalidTextRepresentation + ) = PsycoPg2DivisionByZero = PsycoPg2InternalError = PsycoPg2ProgrammingError = None try: from pymysql.err import NotSupportedError as MySQLNotSupportedError diff --git a/ibis/backends/tests/snapshots/test_dot_sql/test_cte/risingwave/out.sql b/ibis/backends/tests/snapshots/test_dot_sql/test_cte/risingwave/out.sql new file mode 100644 index 000000000000..efc0daaef0d6 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_dot_sql/test_cte/risingwave/out.sql @@ -0,0 +1,8 @@ +WITH "foo" AS ( + SELECT + * + FROM "test_risingwave_temp_mem_t_for_cte" AS "t0" +) +SELECT + COUNT(*) AS "x" +FROM "foo" \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_interactive/test_default_limit/risingwave/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_default_limit/risingwave/out.sql new file mode 100644 index 000000000000..b309cd65374d --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_default_limit/risingwave/out.sql @@ -0,0 +1,5 @@ +SELECT + "t0"."id", + "t0"."bool_col" +FROM "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/risingwave/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_disable_query_limit/risingwave/out.sql new file mode 100644 index 000000000000..b309cd65374d --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_disable_query_limit/risingwave/out.sql @@ -0,0 +1,5 @@ +SELECT + "t0"."id", + "t0"."bool_col" +FROM "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/risingwave/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_interactive_execute_on_repr/risingwave/out.sql new file mode 100644 index 000000000000..6bd0ba8c995d --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_interactive_execute_on_repr/risingwave/out.sql @@ -0,0 +1,3 @@ +SELECT + SUM("t0"."bigint_col") AS "Sum(bigint_col)" +FROM "functional_alltypes" AS "t0" \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_interactive/test_respect_set_limit/risingwave/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_respect_set_limit/risingwave/out.sql new file mode 100644 index 000000000000..97338646649f --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_respect_set_limit/risingwave/out.sql @@ -0,0 +1,10 @@ +SELECT + * +FROM ( + SELECT + "t0"."id", + "t0"."bool_col" + FROM "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/risingwave/out.sql b/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/risingwave/out.sql new file mode 100644 index 000000000000..d3969647c9ea --- /dev/null +++ b/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/risingwave/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/risingwave/out.sql b/ibis/backends/tests/snapshots/test_sql/test_isin_bug/risingwave/out.sql new file mode 100644 index 000000000000..c1611d8cecc3 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_sql/test_isin_bug/risingwave/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/risingwave/out.sql b/ibis/backends/tests/snapshots/test_sql/test_union_aliasing/risingwave/out.sql new file mode 100644 index 000000000000..b7508b9ef535 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_sql/test_union_aliasing/risingwave/out.sql @@ -0,0 +1,60 @@ +WITH "t5" AS ( + SELECT + "t4"."field_of_study", + FIRST("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", + CAST(TO_JSONB("t1"."__pivoted__") -> 'f1' AS VARCHAR) AS "years", + CAST(TO_JSONB("t1"."__pivoted__") -> 'f2' AS BIGINT) AS "degrees" + FROM ( + SELECT + "t0"."field_of_study", + UNNEST( + ARRAY[ROW(CAST('1970-71' AS VARCHAR), CAST("t0"."1970-71" AS BIGINT)), ROW(CAST('1975-76' AS VARCHAR), CAST("t0"."1975-76" AS BIGINT)), ROW(CAST('1980-81' AS VARCHAR), CAST("t0"."1980-81" AS BIGINT)), ROW(CAST('1985-86' AS VARCHAR), CAST("t0"."1985-86" AS BIGINT)), ROW(CAST('1990-91' AS VARCHAR), CAST("t0"."1990-91" AS BIGINT)), ROW(CAST('1995-96' AS VARCHAR), CAST("t0"."1995-96" AS BIGINT)), ROW(CAST('2000-01' AS VARCHAR), CAST("t0"."2000-01" AS BIGINT)), ROW(CAST('2005-06' AS VARCHAR), CAST("t0"."2005-06" AS BIGINT)), ROW(CAST('2010-11' AS VARCHAR), CAST("t0"."2010-11" AS BIGINT)), ROW(CAST('2011-12' AS VARCHAR), CAST("t0"."2011-12" AS BIGINT)), ROW(CAST('2012-13' AS VARCHAR), CAST("t0"."2012-13" AS BIGINT)), ROW(CAST('2013-14' AS VARCHAR), CAST("t0"."2013-14" AS BIGINT)), ROW(CAST('2014-15' AS VARCHAR), CAST("t0"."2014-15" AS BIGINT)), ROW(CAST('2015-16' AS VARCHAR), CAST("t0"."2015-16" AS BIGINT)), ROW(CAST('2016-17' AS VARCHAR), CAST("t0"."2016-17" AS BIGINT)), ROW(CAST('2017-18' AS VARCHAR), CAST("t0"."2017-18" AS BIGINT)), ROW(CAST('2018-19' AS VARCHAR), CAST("t0"."2018-19" AS BIGINT)), ROW(CAST('2019-20' AS VARCHAR), CAST("t0"."2019-20" AS BIGINT))] + ) AS "__pivoted__" + FROM "humanities" AS "t0" + ) AS "t1" + ) AS "t2" + ) AS "t3" + ) AS "t4" + GROUP BY + 1 +) +SELECT + "t11"."field_of_study", + "t11"."diff" +FROM ( + SELECT + "t6"."field_of_study", + "t6"."diff" + FROM "t5" AS "t6" + ORDER BY + "t6"."diff" DESC NULLS LAST + LIMIT 10 + UNION ALL + SELECT + "t6"."field_of_study", + "t6"."diff" + FROM "t5" AS "t6" + WHERE + "t6"."diff" < 0 + ORDER BY + "t6"."diff" ASC + LIMIT 10 +) AS "t11" \ No newline at end of file diff --git a/ibis/backends/tests/test_aggregation.py b/ibis/backends/tests/test_aggregation.py index 90a3850b26b9..5f8f0a15b793 100644 --- a/ibis/backends/tests/test_aggregation.py +++ b/ibis/backends/tests/test_aggregation.py @@ -22,6 +22,7 @@ MySQLNotSupportedError, OracleDatabaseError, PolarsInvalidOperationError, + PsycoPg2InternalError, Py4JError, PyDruidProgrammingError, PyODBCProgrammingError, @@ -49,6 +50,7 @@ def mean_udf(s): "bigquery", "datafusion", "postgres", + "risingwave", "clickhouse", "impala", "duckdb", @@ -91,6 +93,7 @@ def mean_udf(s): "druid", "oracle", "flink", + "risingwave", "exasol", ], raises=com.OperationNotDefinedError, @@ -205,6 +208,7 @@ def test_aggregate_grouped(backend, alltypes, df, result_fn, expected_fn): "impala", "mysql", "postgres", + "risingwave", "sqlite", "snowflake", "polars", @@ -437,6 +441,7 @@ def mean_and_std(v): "oracle", "exasol", "flink", + "risingwave", ], raises=com.OperationNotDefinedError, ), @@ -518,39 +523,51 @@ def mean_and_std(v): lambda t, where: t.double_col.arbitrary(where=where), lambda t, where: t.double_col[where].iloc[0], id="arbitrary_default", - marks=pytest.mark.notimpl( - [ - "impala", - "mysql", - "polars", - "datafusion", - "mssql", - "druid", - "oracle", - "exasol", - "flink", - ], - raises=com.OperationNotDefinedError, - ), + marks=[ + pytest.mark.notimpl( + [ + "impala", + "mysql", + "polars", + "datafusion", + "mssql", + "druid", + "oracle", + "exasol", + "flink", + ], + raises=com.OperationNotDefinedError, + ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + ), + ], ), param( lambda t, where: t.double_col.arbitrary(how="first", where=where), lambda t, where: t.double_col[where].iloc[0], id="arbitrary_first", - marks=pytest.mark.notimpl( - [ - "impala", - "mysql", - "polars", - "datafusion", - "mssql", - "druid", - "oracle", - "exasol", - "flink", - ], - raises=com.OperationNotDefinedError, - ), + marks=[ + pytest.mark.notimpl( + [ + "impala", + "mysql", + "polars", + "datafusion", + "mssql", + "druid", + "oracle", + "exasol", + "flink", + ], + raises=com.OperationNotDefinedError, + ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + ), + ], ), param( lambda t, where: t.double_col.arbitrary(how="last", where=where), @@ -576,6 +593,10 @@ def mean_and_std(v): raises=com.UnsupportedOperationError, reason="backend only supports the `first` option for `.arbitrary()", ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + ), ], ), param( @@ -602,7 +623,14 @@ def mean_and_std(v): raises=com.OperationNotDefinedError, ), pytest.mark.notimpl( - ["bigquery", "duckdb", "postgres", "pyspark", "trino"], + [ + "bigquery", + "duckdb", + "postgres", + "risingwave", + "pyspark", + "trino", + ], raises=com.UnsupportedOperationError, reason="how='heavy' not supported in the backend", ), @@ -617,19 +645,31 @@ def mean_and_std(v): lambda t, where: t.double_col.first(where=where), lambda t, where: t.double_col[where].iloc[0], id="first", - marks=pytest.mark.notimpl( - ["dask", "druid", "impala", "mssql", "mysql", "oracle", "flink"], - raises=com.OperationNotDefinedError, - ), + marks=[ + pytest.mark.notimpl( + ["dask", "druid", "impala", "mssql", "mysql", "oracle", "flink"], + raises=com.OperationNotDefinedError, + ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + ), + ], ), param( lambda t, where: t.double_col.last(where=where), lambda t, where: t.double_col[where].iloc[-1], id="last", - marks=pytest.mark.notimpl( - ["dask", "druid", "impala", "mssql", "mysql", "oracle", "flink"], - raises=com.OperationNotDefinedError, - ), + marks=[ + pytest.mark.notimpl( + ["dask", "druid", "impala", "mssql", "mysql", "oracle", "flink"], + raises=com.OperationNotDefinedError, + ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + ), + ], ), param( lambda t, where: t.bigint_col.bit_and(where=where), @@ -734,21 +774,25 @@ def mean_and_std(v): param( lambda t: t.string_col.isin(["1", "7"]), lambda t: t.string_col.isin(["1", "7"]), - marks=pytest.mark.notimpl( - ["exasol"], - raises=(com.OperationNotDefinedError, ExaQueryError), - strict=False, - ), + marks=[ + pytest.mark.notimpl( + ["exasol"], + raises=(com.OperationNotDefinedError, ExaQueryError), + strict=False, + ), + ], id="is_in", ), param( lambda _: ibis._.string_col.isin(["1", "7"]), lambda t: t.string_col.isin(["1", "7"]), - marks=pytest.mark.notimpl( - ["exasol"], - raises=(com.OperationNotDefinedError, ExaQueryError), - strict=False, - ), + marks=[ + pytest.mark.notimpl( + ["exasol"], + raises=(com.OperationNotDefinedError, ExaQueryError), + strict=False, + ), + ], id="is_in_deferred", ), ], @@ -899,6 +943,11 @@ def test_count_distinct_star(alltypes, df, ibis_cond, pandas_cond): reason="backend doesn't implement approximate quantiles yet", raises=com.OperationNotDefinedError, ), + pytest.mark.broken( + ["risingwave"], + reason="Invalid input syntax: direct arg in `percentile_cont` must be castable to float64", + raises=PsycoPg2InternalError, + ), ], ), ], @@ -912,7 +961,14 @@ def test_count_distinct_star(alltypes, df, ibis_cond, pandas_cond): lambda t: t.string_col.isin(["1", "7"]), id="is_in", marks=[ - pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError) + pytest.mark.notimpl( + ["datafusion"], raises=com.OperationNotDefinedError + ), + pytest.mark.notimpl( + "risingwave", + raises=PsycoPg2InternalError, + reason="probably incorrect filter syntax but not sure", + ), ], ), ], @@ -947,6 +1003,11 @@ def test_quantile( ["mysql", "impala", "sqlite", "flink"], raises=com.OperationNotDefinedError, ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function covar_pop(integer, integer) does not exist", + ), ], ), param( @@ -962,6 +1023,11 @@ def test_quantile( ["mysql", "impala", "sqlite", "flink"], raises=com.OperationNotDefinedError, ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function covar_pop(integer, integer) does not exist", + ), ], ), param( @@ -982,6 +1048,11 @@ def test_quantile( raises=(ValueError, AttributeError), reason="ClickHouse only implements `sample` correlation coefficient", ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function covar_pop(integer, integer) does not exist", + ), ], ), param( @@ -994,7 +1065,7 @@ def test_quantile( raises=com.OperationNotDefinedError, ), pytest.mark.notyet( - ["postgres", "duckdb", "snowflake"], + ["postgres", "duckdb", "snowflake", "risingwave"], raises=com.UnsupportedOperationError, reason="backend only implements population correlation coefficient", ), @@ -1032,6 +1103,11 @@ def test_quantile( ["mysql", "impala", "sqlite", "flink"], raises=com.OperationNotDefinedError, ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function covar_pop(integer, integer) does not exist", + ), ], ), param( @@ -1056,6 +1132,11 @@ def test_quantile( raises=ValueError, reason="ClickHouse only implements `sample` correlation coefficient", ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function covar_pop(integer, integer) does not exist", + ), ], ), ], @@ -1374,6 +1455,7 @@ def test_topk_filter_op(con, alltypes, df, result_fn, expected_fn): "impala", "mysql", "postgres", + "risingwave", "sqlite", "snowflake", "polars", @@ -1414,6 +1496,7 @@ def test_aggregate_list_like(backend, alltypes, df, agg_fn): "impala", "mysql", "postgres", + "risingwave", "sqlite", "snowflake", "polars", @@ -1535,7 +1618,9 @@ def test_grouped_case(backend, con): @pytest.mark.notyet(["oracle"], raises=OracleDatabaseError) @pytest.mark.notyet(["pyspark"], raises=PySparkAnalysisException) @pytest.mark.notyet(["mssql"], raises=PyODBCProgrammingError) +@pytest.mark.notyet(["risingwave"], raises=AssertionError, strict=False) def test_group_concat_over_window(backend, con): + # TODO: this test is flaky on risingwave and I DO NOT LIKE IT input_df = pd.DataFrame( { "s": ["a|b|c", "b|a|c", "b|b|b|c|a"], diff --git a/ibis/backends/tests/test_array.py b/ibis/backends/tests/test_array.py index a784eddf3c44..7c5ad3ba2550 100644 --- a/ibis/backends/tests/test_array.py +++ b/ibis/backends/tests/test_array.py @@ -23,6 +23,8 @@ MySQLOperationalError, PolarsComputeError, PsycoPg2IndeterminateDatatype, + PsycoPg2InternalError, + PsycoPg2ProgrammingError, PsycoPg2SyntaxError, Py4JJavaError, PySparkAnalysisException, @@ -79,6 +81,7 @@ def test_array_column(backend, alltypes, df): "bigquery": "ARRAY", "duckdb": "DOUBLE[]", "postgres": "numeric[]", + "risingwave": "numeric[]", "flink": "ARRAY NOT NULL", } @@ -209,6 +212,11 @@ def test_array_index(con, idx): reason="backend does not support nullable nested types", raises=AssertionError, ) +@pytest.mark.notimpl( + ["risingwave"], + raises=AssertionError, + reason="Do not nest ARRAY types; ARRAY(basetype) handles multi-dimensional arrays of basetype", +) @pytest.mark.never( ["bigquery"], reason="doesn't support arrays of arrays", raises=AssertionError ) @@ -240,6 +248,12 @@ def test_array_discovery(backend): ) @pytest.mark.notimpl(["dask"], raises=ValueError) @pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl( + ["risingwave"], + # TODO: valueerror -> assertion error + raises=AssertionError, + reason="Do not nest ARRAY types; ARRAY(basetype) handles multi-dimensional arrays of basetype", +) def test_unnest_simple(backend): array_types = backend.array_types expected = ( @@ -343,6 +357,7 @@ def test_unnest_no_nulls(backend): @builtin_array @pytest.mark.notimpl("dask", raises=ValueError) @pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError) +@pytest.mark.broken(["risingwave"], raises=AssertionError) def test_unnest_default_name(backend): array_types = backend.array_types df = array_types.execute() @@ -378,6 +393,12 @@ def test_unnest_default_name(backend): ["datafusion", "flink"], raises=Exception, reason="array_types table isn't defined" ) @pytest.mark.notimpl(["dask"], raises=com.OperationNotDefinedError) +@pytest.mark.broken( + ["risingwave"], + raises=AssertionError, + reason="not broken; row ordering is not guaranteed and sometimes this test will pass", + strict=False, +) def test_array_slice(backend, start, stop): array_types = backend.array_types expr = array_types.select(sliced=array_types.y[start:stop]) @@ -392,6 +413,11 @@ def test_array_slice(backend, start, stop): @pytest.mark.notimpl( ["datafusion", "polars", "snowflake", "sqlite"], raises=com.OperationNotDefinedError ) +@pytest.mark.broken( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="TODO(Kexiang): seems a bug", +) @pytest.mark.notimpl( ["dask", "pandas"], raises=com.OperationNotDefinedError, @@ -419,6 +445,7 @@ def test_array_slice(backend, start, stop): ], ) def test_array_map(con, input, output): + t = ibis.memtable(input, schema=ibis.schema(dict(a="!array"))) t = ibis.memtable(input, schema=ibis.schema(dict(a="!array"))) expected = pd.DataFrame(output) @@ -462,6 +489,11 @@ def test_array_map(con, input, output): param({"a": [[1, 2], [4]]}, {"a": [[2], [4]]}, id="no_nulls"), ], ) +@pytest.mark.notyet( + "risingwave", + raises=PsycoPg2InternalError, + reason="no support for not null column constraint", +) def test_array_filter(con, input, output): t = ibis.memtable(input, schema=ibis.schema(dict(a="!array"))) expected = pd.DataFrame(output) @@ -478,6 +510,12 @@ def test_array_filter(con, input, output): @builtin_array @pytest.mark.notimpl(["polars"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl(["dask"], raises=com.OperationNotDefinedError) +@pytest.mark.broken( + ["risingwave"], + raises=AssertionError, + reason="not broken; row ordering is not guaranteed and sometimes this test will pass", + strict=False, +) def test_array_contains(backend, con): t = backend.array_types expr = t.x.contains(1) @@ -531,6 +569,11 @@ def test_array_remove(con): raises=(AssertionError, GoogleBadRequest), reason="bigquery doesn't support null elements in arrays", ) +@pytest.mark.broken( + ["risingwave"], + raises=AssertionError, + reason="TODO(Kexiang): seems a bug", +) @pytest.mark.parametrize( ("input", "expected"), [ @@ -558,6 +601,11 @@ def test_array_unique(con, input, expected): @pytest.mark.notimpl( ["dask", "datafusion", "polars"], raises=com.OperationNotDefinedError ) +@pytest.mark.broken( + ["risingwave"], + raises=AssertionError, + reason="Refer to https://github.com/risingwavelabs/risingwave/issues/14735", +) def test_array_sort(backend, con): t = ibis.memtable({"a": [[3, 2], [], [42, 42], []], "id": range(4)}) expr = t.mutate(a=t.a.sort()).order_by("id") @@ -593,6 +641,11 @@ def test_array_union(con): @pytest.mark.notimpl( ["sqlite"], raises=com.UnsupportedBackendType, reason="Unsupported type: Array..." ) +@pytest.mark.broken( + ["risingwave"], + raises=AssertionError, + reason="TODO(Kexiang): seems a bug", +) @pytest.mark.parametrize( "data", [ @@ -631,6 +684,7 @@ def test_array_intersect(con, data): reason="ClickHouse won't accept dicts for struct type values", ) @pytest.mark.notimpl(["postgres"], raises=PsycoPg2SyntaxError) +@pytest.mark.notimpl(["risingwave"], raises=PsycoPg2InternalError) @pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError) @pytest.mark.broken( ["trino"], reason="inserting maps into structs doesn't work", raises=TrinoUserError @@ -648,7 +702,16 @@ def test_unnest_struct(con): @builtin_array @pytest.mark.notimpl( - ["dask", "datafusion", "druid", "oracle", "pandas", "polars", "postgres"], + [ + "dask", + "datafusion", + "druid", + "oracle", + "pandas", + "polars", + "postgres", + "risingwave", + ], raises=com.OperationNotDefinedError, ) def test_zip(backend): @@ -676,6 +739,7 @@ def test_zip(backend): reason="https://github.com/ClickHouse/ClickHouse/issues/41112", ) @pytest.mark.notimpl(["postgres"], raises=PsycoPg2SyntaxError) +@pytest.mark.notimpl(["risingwave"], raises=PsycoPg2ProgrammingError) @pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl( ["polars"], @@ -733,9 +797,13 @@ def flatten_data(): ["bigquery"], reason="BigQuery doesn't support arrays of arrays", raises=TypeError ) @pytest.mark.notyet( - ["postgres"], + ["postgres", "risingwave"], reason="Postgres doesn't truly support arrays of arrays", - raises=(com.OperationNotDefinedError, PsycoPg2IndeterminateDatatype), + raises=( + com.OperationNotDefinedError, + PsycoPg2IndeterminateDatatype, + PsycoPg2InternalError, + ), ) @pytest.mark.parametrize( ("column", "expected"), @@ -804,6 +872,7 @@ def test_range_single_argument(con, n): ) @pytest.mark.parametrize("n", [-2, 0, 2]) @pytest.mark.notimpl(["polars", "flink", "dask"], raises=com.OperationNotDefinedError) +@pytest.mark.skip("risingwave") def test_range_single_argument_unnest(backend, con, n): expr = ibis.range(n).unnest() result = con.execute(expr) @@ -850,6 +919,11 @@ def test_range_start_stop_step(con, start, stop, step): ["datafusion"], raises=com.OperationNotDefinedError, reason="not supported upstream" ) @pytest.mark.notimpl(["flink", "dask"], raises=com.OperationNotDefinedError) +@pytest.mark.never( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Invalid parameter step: step size cannot equal zero", +) def test_range_start_stop_step_zero(con, start, stop): expr = ibis.range(start, stop, 0) result = con.execute(expr) @@ -879,6 +953,11 @@ def test_unnest_empty_array(con): raises=com.OperationNotDefinedError, ) @pytest.mark.notimpl(["sqlite"], raises=com.UnsupportedBackendType) +@pytest.mark.notyet( + "risingwave", + raises=PsycoPg2InternalError, + reason="no support for not null column constraint", +) def test_array_map_with_conflicting_names(backend, con): t = ibis.memtable({"x": [[1, 2]]}, schema=ibis.schema(dict(x="!array"))) expr = t.select(a=t.x.map(lambda x: x + 1)).select( @@ -957,6 +1036,11 @@ def swap(token): ibis.interval(hours=1), "1H", id="pos", + marks=pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function make_interval() does not exist", + ), ), param( datetime(2017, 1, 2), @@ -967,7 +1051,12 @@ def swap(token): marks=[ pytest.mark.broken( ["polars"], raises=AssertionError, reason="returns an empty array" - ) + ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function neg(interval) does not exist", + ), ], ), param( @@ -983,6 +1072,11 @@ def swap(token): ["clickhouse", "snowflake"], raises=com.UnsupportedOperationError, ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function neg(interval) does not exist", + ), ], ), ], @@ -1008,7 +1102,14 @@ def test_timestamp_range(con, start, stop, step, freq, tzinfo): datetime(2017, 1, 2, tzinfo=pytz.UTC), ibis.interval(hours=0), id="pos", - marks=[pytest.mark.notyet(["polars"], raises=PolarsComputeError)], + marks=[ + pytest.mark.notyet(["polars"], raises=PolarsComputeError), + pytest.mark.notyet( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function make_interval() does not exist", + ), + ], ), param( datetime(2017, 1, 1, tzinfo=pytz.UTC), @@ -1022,6 +1123,11 @@ def test_timestamp_range(con, start, stop, step, freq, tzinfo): ["clickhouse", "snowflake"], raises=com.UnsupportedOperationError, ), + pytest.mark.notyet( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function neg(interval) does not exist", + ), ], ), ], @@ -1051,3 +1157,15 @@ def test_repr_timestamp_array(con, monkeypatch): expr = ibis.array(pd.date_range("2010-01-01", "2010-01-03", freq="D").tolist()) assert "Translation to backend failed" not in repr(expr) + + +@pytest.mark.notyet( + ["dask", "datafusion", "flink", "polars"], + raises=com.OperationNotDefinedError, +) +@pytest.mark.broken(["pandas"], raises=ValueError, reason="reindex on duplicate values") +def test_unnest_range(con): + expr = ibis.range(2).unnest().name("x").as_table().mutate({"y": 1.0}) + result = con.execute(expr) + expected = pd.DataFrame({"x": np.array([0, 1], dtype="int8"), "y": [1.0, 1.0]}) + tm.assert_frame_equal(result, expected) diff --git a/ibis/backends/tests/test_asof_join.py b/ibis/backends/tests/test_asof_join.py index ffe86146b65f..2a1901efc520 100644 --- a/ibis/backends/tests/test_asof_join.py +++ b/ibis/backends/tests/test_asof_join.py @@ -95,6 +95,7 @@ def time_keyed_right(time_keyed_df2): "oracle", "mssql", "sqlite", + "risingwave", ] ) def test_asof_join(con, time_left, time_right, time_df1, time_df2, direction, op): @@ -135,6 +136,7 @@ def test_asof_join(con, time_left, time_right, time_df1, time_df2, direction, op "oracle", "mssql", "sqlite", + "risingwave", ] ) def test_keyed_asof_join_with_tolerance( diff --git a/ibis/backends/tests/test_benchmarks.py b/ibis/backends/tests/test_benchmarks.py deleted file mode 100644 index 4805a5ccf5bc..000000000000 --- a/ibis/backends/tests/test_benchmarks.py +++ /dev/null @@ -1,900 +0,0 @@ -from __future__ import annotations - -import copy -import functools -import inspect -import itertools -import os -import string - -import numpy as np -import pandas as pd -import pytest -import sqlalchemy as sa -from packaging.version import parse as vparse - -import ibis -import ibis.expr.datatypes as dt -import ibis.expr.operations as ops -import ibis.expr.types as ir -from ibis.backends.base import _get_backend_names - -# from ibis.backends.pandas.udf import udf - -# FIXME(kszucs): pytestmark = pytest.mark.benchmark -pytestmark = pytest.mark.skip(reason="the backends must be rewritten first") - - -def make_t(): - return ibis.table( - [ - ("_timestamp", "int32"), - ("dim1", "int32"), - ("dim2", "int32"), - ("valid_seconds", "int32"), - ("meas1", "int32"), - ("meas2", "int32"), - ("year", "int32"), - ("month", "int32"), - ("day", "int32"), - ("hour", "int32"), - ("minute", "int32"), - ], - name="t", - ) - - -@pytest.fixture(scope="module") -def t(): - return make_t() - - -def make_base(t): - return t[ - ( - (t.year > 2016) - | ((t.year == 2016) & (t.month > 6)) - | ((t.year == 2016) & (t.month == 6) & (t.day > 6)) - | ((t.year == 2016) & (t.month == 6) & (t.day == 6) & (t.hour > 6)) - | ( - (t.year == 2016) - & (t.month == 6) - & (t.day == 6) - & (t.hour == 6) - & (t.minute >= 5) - ) - ) - & ( - (t.year < 2016) - | ((t.year == 2016) & (t.month < 6)) - | ((t.year == 2016) & (t.month == 6) & (t.day < 6)) - | ((t.year == 2016) & (t.month == 6) & (t.day == 6) & (t.hour < 6)) - | ( - (t.year == 2016) - & (t.month == 6) - & (t.day == 6) - & (t.hour == 6) - & (t.minute <= 5) - ) - ) - ] - - -@pytest.fixture(scope="module") -def base(t): - return make_base(t) - - -def make_large_expr(base): - src_table = base - src_table = src_table.mutate( - _timestamp=(src_table["_timestamp"] - src_table["_timestamp"] % 3600) - .cast("int32") - .name("_timestamp"), - valid_seconds=300, - ) - - aggs = [] - for meas in ["meas1", "meas2"]: - aggs.append(src_table[meas].sum().cast("float").name(meas)) - src_table = src_table.aggregate( - aggs, by=["_timestamp", "dim1", "dim2", "valid_seconds"] - ) - - part_keys = ["year", "month", "day", "hour", "minute"] - ts_col = src_table["_timestamp"].cast("timestamp") - new_cols = {} - for part_key in part_keys: - part_col = getattr(ts_col, part_key)() - new_cols[part_key] = part_col - src_table = src_table.mutate(**new_cols) - return src_table[ - [ - "_timestamp", - "dim1", - "dim2", - "meas1", - "meas2", - "year", - "month", - "day", - "hour", - "minute", - ] - ] - - -@pytest.fixture(scope="module") -def large_expr(base): - return make_large_expr(base) - - -@pytest.mark.benchmark(group="construction") -@pytest.mark.parametrize( - "construction_fn", - [ - pytest.param(lambda *_: make_t(), id="small"), - pytest.param(lambda t, *_: make_base(t), id="medium"), - pytest.param(lambda _, base: make_large_expr(base), id="large"), - ], -) -def test_construction(benchmark, construction_fn, t, base): - benchmark(construction_fn, t, base) - - -@pytest.mark.benchmark(group="builtins") -@pytest.mark.parametrize( - "expr_fn", - [ - pytest.param(lambda t, _base, _large_expr: t, id="small"), - pytest.param(lambda _t, base, _large_expr: base, id="medium"), - pytest.param(lambda _t, _base, large_expr: large_expr, id="large"), - ], -) -@pytest.mark.parametrize("builtin", [hash, str]) -def test_builtins(benchmark, expr_fn, builtin, t, base, large_expr): - expr = expr_fn(t, base, large_expr) - benchmark(builtin, expr) - - -_backends = set(_get_backend_names()) -# compile is a no-op -_backends.remove("pandas") - -_XFAIL_COMPILE_BACKENDS = {"dask", "pyspark", "polars"} - - -@pytest.mark.benchmark(group="compilation") -@pytest.mark.parametrize( - "module", - [ - pytest.param( - mod, - marks=pytest.mark.xfail( - condition=mod in _XFAIL_COMPILE_BACKENDS, - reason=f"{mod} backend doesn't support compiling UnboundTable", - ), - ) - for mod in _backends - ], -) -@pytest.mark.parametrize( - "expr_fn", - [ - pytest.param(lambda t, _base, _large_expr: t, id="small"), - pytest.param(lambda _t, base, _large_expr: base, id="medium"), - pytest.param(lambda _t, _base, large_expr: large_expr, id="large"), - ], -) -def test_compile(benchmark, module, expr_fn, t, base, large_expr): - try: - mod = getattr(ibis, module) - except (AttributeError, ImportError) as e: - pytest.skip(str(e)) - else: - expr = expr_fn(t, base, large_expr) - try: - benchmark(mod.compile, expr) - except (sa.exc.NoSuchModuleError, ImportError) as e: # delayed imports - pytest.skip(str(e)) - - -@pytest.fixture(scope="module") -def pt(): - n = 60_000 - data = pd.DataFrame( - { - "key": np.random.choice(16000, size=n), - "low_card_key": np.random.choice(30, size=n), - "value": np.random.rand(n), - "timestamps": pd.date_range( - start="2023-05-05 16:37:57", periods=n, freq="s" - ).values, - "timestamp_strings": pd.date_range( - start="2023-05-05 16:37:39", periods=n, freq="s" - ).values.astype(str), - "repeated_timestamps": pd.date_range(start="2018-09-01", periods=30).repeat( - int(n / 30) - ), - } - ) - - return ibis.pandas.connect(dict(df=data)).table("df") - - -def high_card_group_by(t): - return t.group_by(t.key).aggregate(avg_value=t.value.mean()) - - -def cast_to_dates(t): - return t.timestamps.cast(dt.date) - - -def cast_to_dates_from_strings(t): - return t.timestamp_strings.cast(dt.date) - - -def multikey_group_by_with_mutate(t): - return ( - t.mutate(dates=t.timestamps.cast("date")) - .group_by(["low_card_key", "dates"]) - .aggregate(avg_value=lambda t: t.value.mean()) - ) - - -def simple_sort(t): - return t.order_by([t.key]) - - -def simple_sort_projection(t): - return t[["key", "value"]].order_by(["key"]) - - -def multikey_sort(t): - return t.order_by(["low_card_key", "key"]) - - -def multikey_sort_projection(t): - return t[["low_card_key", "key", "value"]].order_by(["low_card_key", "key"]) - - -def low_card_rolling_window(t): - return ibis.trailing_range_window( - ibis.interval(days=2), - order_by=t.repeated_timestamps, - group_by=t.low_card_key, - ) - - -def low_card_grouped_rolling(t): - return t.value.mean().over(low_card_rolling_window(t)) - - -def high_card_rolling_window(t): - return ibis.trailing_range_window( - ibis.interval(days=2), - order_by=t.repeated_timestamps, - group_by=t.key, - ) - - -def high_card_grouped_rolling(t): - return t.value.mean().over(high_card_rolling_window(t)) - - -# @udf.reduction(["double"], "double") -# def my_mean(series): -# return series.mean() - - -def low_card_grouped_rolling_udf_mean(t): - return my_mean(t.value).over(low_card_rolling_window(t)) - - -def high_card_grouped_rolling_udf_mean(t): - return my_mean(t.value).over(high_card_rolling_window(t)) - - -# @udf.analytic(["double"], "double") -# def my_zscore(series): -# return (series - series.mean()) / series.std() - - -def low_card_window(t): - return ibis.window(group_by=t.low_card_key) - - -def high_card_window(t): - return ibis.window(group_by=t.key) - - -def low_card_window_analytics_udf(t): - return my_zscore(t.value).over(low_card_window(t)) - - -def high_card_window_analytics_udf(t): - return my_zscore(t.value).over(high_card_window(t)) - - -# @udf.reduction(["double", "double"], "double") -# def my_wm(v, w): -# return np.average(v, weights=w) - - -def low_card_grouped_rolling_udf_wm(t): - return my_wm(t.value, t.value).over(low_card_rolling_window(t)) - - -def high_card_grouped_rolling_udf_wm(t): - return my_wm(t.value, t.value).over(low_card_rolling_window(t)) - - -broken_pandas_grouped_rolling = pytest.mark.xfail( - condition=vparse("1.4") <= vparse(pd.__version__) < vparse("1.4.2"), - raises=ValueError, - reason="https://github.com/pandas-dev/pandas/pull/44068", -) - - -@pytest.mark.benchmark(group="execution") -@pytest.mark.parametrize( - "expression_fn", - [ - pytest.param(high_card_group_by, id="high_card_group_by"), - pytest.param(cast_to_dates, id="cast_to_dates"), - pytest.param(cast_to_dates_from_strings, id="cast_to_dates_from_strings"), - pytest.param(multikey_group_by_with_mutate, id="multikey_group_by_with_mutate"), - pytest.param(simple_sort, id="simple_sort"), - pytest.param(simple_sort_projection, id="simple_sort_projection"), - pytest.param(multikey_sort, id="multikey_sort"), - pytest.param(multikey_sort_projection, id="multikey_sort_projection"), - pytest.param( - low_card_grouped_rolling, - id="low_card_grouped_rolling", - marks=[broken_pandas_grouped_rolling], - ), - pytest.param( - high_card_grouped_rolling, - id="high_card_grouped_rolling", - marks=[broken_pandas_grouped_rolling], - ), - pytest.param( - low_card_grouped_rolling_udf_mean, - id="low_card_grouped_rolling_udf_mean", - marks=[broken_pandas_grouped_rolling], - ), - pytest.param( - high_card_grouped_rolling_udf_mean, - id="high_card_grouped_rolling_udf_mean", - marks=[broken_pandas_grouped_rolling], - ), - pytest.param(low_card_window_analytics_udf, id="low_card_window_analytics_udf"), - pytest.param( - high_card_window_analytics_udf, id="high_card_window_analytics_udf" - ), - pytest.param( - low_card_grouped_rolling_udf_wm, - id="low_card_grouped_rolling_udf_wm", - marks=[broken_pandas_grouped_rolling], - ), - pytest.param( - high_card_grouped_rolling_udf_wm, - id="high_card_grouped_rolling_udf_wm", - marks=[broken_pandas_grouped_rolling], - ), - ], -) -def test_execute(benchmark, expression_fn, pt): - expr = expression_fn(pt) - benchmark(expr.execute) - - -@pytest.fixture(scope="module") -def part(): - return ibis.table( - dict( - p_partkey="int64", - p_size="int64", - p_type="string", - p_mfgr="string", - ), - name="part", - ) - - -@pytest.fixture(scope="module") -def supplier(): - return ibis.table( - dict( - s_suppkey="int64", - s_nationkey="int64", - s_name="string", - s_acctbal="decimal(15, 3)", - s_address="string", - s_phone="string", - s_comment="string", - ), - name="supplier", - ) - - -@pytest.fixture(scope="module") -def partsupp(): - return ibis.table( - dict( - ps_partkey="int64", - ps_suppkey="int64", - ps_supplycost="decimal(15, 3)", - ), - name="partsupp", - ) - - -@pytest.fixture(scope="module") -def nation(): - return ibis.table( - dict(n_nationkey="int64", n_regionkey="int64", n_name="string"), - name="nation", - ) - - -@pytest.fixture(scope="module") -def region(): - return ibis.table(dict(r_regionkey="int64", r_name="string"), name="region") - - -@pytest.fixture(scope="module") -def tpc_h02(part, supplier, partsupp, nation, region): - REGION = "EUROPE" - SIZE = 25 - TYPE = "BRASS" - - expr = ( - part.join(partsupp, part.p_partkey == partsupp.ps_partkey) - .join(supplier, supplier.s_suppkey == partsupp.ps_suppkey) - .join(nation, supplier.s_nationkey == nation.n_nationkey) - .join(region, nation.n_regionkey == region.r_regionkey) - ) - - subexpr = ( - partsupp.join(supplier, supplier.s_suppkey == partsupp.ps_suppkey) - .join(nation, supplier.s_nationkey == nation.n_nationkey) - .join(region, nation.n_regionkey == region.r_regionkey) - ) - - subexpr = subexpr[ - (subexpr.r_name == REGION) & (expr.p_partkey == subexpr.ps_partkey) - ] - - filters = [ - expr.p_size == SIZE, - expr.p_type.like(f"%{TYPE}"), - expr.r_name == REGION, - expr.ps_supplycost == subexpr.ps_supplycost.min(), - ] - q = expr.filter(filters) - - q = q.select( - [ - q.s_acctbal, - q.s_name, - q.n_name, - q.p_partkey, - q.p_mfgr, - q.s_address, - q.s_phone, - q.s_comment, - ] - ) - - return q.order_by( - [ - ibis.desc(q.s_acctbal), - q.n_name, - q.s_name, - q.p_partkey, - ] - ).limit(100) - - -@pytest.mark.benchmark(group="repr") -def test_repr_tpc_h02(benchmark, tpc_h02): - benchmark(repr, tpc_h02) - - -@pytest.mark.benchmark(group="repr") -def test_repr_huge_union(benchmark): - n = 10 - raw_types = [ - "int64", - "float64", - "string", - "array, b: map>>>", - ] - tables = [ - ibis.table( - list(zip(string.ascii_letters, itertools.cycle(raw_types))), - name=f"t{i:d}", - ) - for i in range(n) - ] - expr = functools.reduce(ir.Table.union, tables) - benchmark(repr, expr) - - -@pytest.mark.benchmark(group="node_args") -def test_op_argnames(benchmark): - t = ibis.table([("a", "int64")]) - expr = t[["a"]] - benchmark(lambda op: op.argnames, expr.op()) - - -@pytest.mark.benchmark(group="node_args") -def test_op_args(benchmark): - t = ibis.table([("a", "int64")]) - expr = t[["a"]] - benchmark(lambda op: op.args, expr.op()) - - -@pytest.mark.benchmark(group="datatype") -def test_complex_datatype_parse(benchmark): - type_str = "array, b: map>>>" - expected = dt.Array( - dt.Struct(dict(a=dt.Array(dt.string), b=dt.Map(dt.string, dt.Array(dt.int64)))) - ) - assert dt.parse(type_str) == expected - benchmark(dt.parse, type_str) - - -@pytest.mark.benchmark(group="datatype") -@pytest.mark.parametrize("func", [str, hash]) -def test_complex_datatype_builtins(benchmark, func): - datatype = dt.Array( - dt.Struct(dict(a=dt.Array(dt.string), b=dt.Map(dt.string, dt.Array(dt.int64)))) - ) - benchmark(func, datatype) - - -@pytest.mark.benchmark(group="equality") -def test_large_expr_equals(benchmark, tpc_h02): - benchmark(ir.Expr.equals, tpc_h02, copy.deepcopy(tpc_h02)) - - -@pytest.mark.benchmark(group="datatype") -@pytest.mark.parametrize( - "dtypes", - [ - pytest.param( - [ - obj - for _, obj in inspect.getmembers( - dt, - lambda obj: isinstance(obj, dt.DataType), - ) - ], - id="singletons", - ), - pytest.param( - dt.Array( - dt.Struct( - dict( - a=dt.Array(dt.string), - b=dt.Map(dt.string, dt.Array(dt.int64)), - ) - ) - ), - id="complex", - ), - ], -) -def test_eq_datatypes(benchmark, dtypes): - def eq(a, b): - assert a == b - - benchmark(eq, dtypes, copy.deepcopy(dtypes)) - - -def multiple_joins(table, num_joins): - for _ in range(num_joins): - table = table.mutate(dummy=ibis.literal("")) - table = table.left_join(table, ["dummy"])[[table]] - - -@pytest.mark.parametrize("num_joins", [1, 10]) -@pytest.mark.parametrize("num_columns", [1, 10, 100]) -def test_multiple_joins(benchmark, num_joins, num_columns): - table = ibis.table( - {f"col_{i:d}": "string" for i in range(num_columns)}, - name="t", - ) - benchmark(multiple_joins, table, num_joins) - - -@pytest.fixture -def customers(): - return ibis.table( - dict( - customerid="int32", - name="string", - address="string", - citystatezip="string", - birthdate="date", - phone="string", - timezone="string", - lat="float64", - long="float64", - ), - name="customers", - ) - - -@pytest.fixture -def orders(): - return ibis.table( - dict( - orderid="int32", - customerid="int32", - ordered="timestamp", - shipped="timestamp", - items="string", - total="float64", - ), - name="orders", - ) - - -@pytest.fixture -def orders_items(): - return ibis.table( - dict(orderid="int32", sku="string", qty="int32", unit_price="float64"), - name="orders_items", - ) - - -@pytest.fixture -def products(): - return ibis.table( - dict( - sku="string", - desc="string", - weight_kg="float64", - cost="float64", - dims_cm="string", - ), - name="products", - ) - - -@pytest.mark.benchmark(group="compilation") -@pytest.mark.parametrize( - "module", - [ - pytest.param( - mod, - marks=pytest.mark.xfail( - condition=mod in _XFAIL_COMPILE_BACKENDS, - reason=f"{mod} backend doesn't support compiling UnboundTable", - ), - ) - for mod in _backends - ], -) -def test_compile_with_drops( - benchmark, module, customers, orders, orders_items, products -): - expr = ( - customers.join(orders, "customerid") - .join(orders_items, "orderid") - .join(products, "sku") - .drop("customerid", "qty", "total", "items") - .drop("dims_cm", "cost") - .mutate(o_date=lambda t: t.shipped.date()) - .filter(lambda t: t.ordered == t.shipped) - ) - - try: - mod = getattr(ibis, module) - except (AttributeError, ImportError) as e: - pytest.skip(str(e)) - else: - try: - benchmark(mod.compile, expr) - except sa.exc.NoSuchModuleError as e: - pytest.skip(str(e)) - - -def test_repr_join(benchmark, customers, orders, orders_items, products): - expr = ( - customers.join(orders, "customerid") - .join(orders_items, "orderid") - .join(products, "sku") - .drop("customerid", "qty", "total", "items") - ) - op = expr.op() - benchmark(repr, op) - - -@pytest.mark.parametrize("overwrite", [True, False], ids=["overwrite", "no_overwrite"]) -def test_insert_duckdb(benchmark, overwrite, tmp_path): - pytest.importorskip("duckdb") - - n_rows = int(1e4) - table_name = "t" - schema = ibis.schema(dict(a="int64", b="int64", c="int64")) - t = ibis.memtable(dict.fromkeys(list("abc"), range(n_rows)), schema=schema) - - con = ibis.duckdb.connect(tmp_path / "test_insert.ddb") - con.create_table(table_name, schema=schema) - benchmark(con.insert, table_name, t, overwrite=overwrite) - - -def test_snowflake_medium_sized_to_pandas(benchmark): - pytest.importorskip("snowflake.connector") - - if (url := os.environ.get("SNOWFLAKE_URL")) is None: - pytest.skip("SNOWFLAKE_URL environment variable not set") - - con = ibis.connect(url) - - # LINEITEM at scale factor 1 is around 6MM rows, but we limit to 1,000,000 - # to make the benchmark fast enough for development, yet large enough to show a - # difference if there's a performance hit - lineitem = con.table("LINEITEM", schema="SNOWFLAKE_SAMPLE_DATA.TPCH_SF1").limit( - 1_000_000 - ) - - benchmark.pedantic(lineitem.to_pandas, rounds=5, iterations=1, warmup_rounds=1) - - -def test_parse_many_duckdb_types(benchmark): - parse = pytest.importorskip("ibis.backends.duckdb.datatypes").DuckDBType.from_string - - def parse_many(types): - list(map(parse, types)) - - types = ["VARCHAR", "INTEGER", "DOUBLE", "BIGINT"] * 1000 - benchmark(parse_many, types) - - -@pytest.fixture(scope="session") -def sql() -> str: - return """ - SELECT t1.id as t1_id, x, t2.id as t2_id, y - FROM t1 INNER JOIN t2 - ON t1.id = t2.id - """ - - -@pytest.fixture(scope="session") -def ddb(tmp_path_factory): - duckdb = pytest.importorskip("duckdb") - - N = 20_000_000 - - con = duckdb.connect() - - path = str(tmp_path_factory.mktemp("duckdb") / "data.ddb") - sql = ( - lambda var, table, n=N: f""" - CREATE TABLE {table} AS - SELECT ROW_NUMBER() OVER () AS id, {var} - FROM ( - SELECT {var} - FROM RANGE({n}) _ ({var}) - ORDER BY RANDOM() - ) - """ - ) - - with duckdb.connect(path) as con: - con.execute(sql("x", table="t1")) - con.execute(sql("y", table="t2")) - return path - - -def test_duckdb_to_pyarrow(benchmark, sql, ddb) -> None: - # yes, we're benchmarking duckdb here, not ibis - # - # we do this to get a baseline for comparison - duckdb = pytest.importorskip("duckdb") - con = duckdb.connect(ddb, read_only=True) - - benchmark(lambda sql: con.sql(sql).to_arrow_table(), sql) - - -def test_ibis_duckdb_to_pyarrow(benchmark, sql, ddb) -> None: - pytest.importorskip("duckdb") - - con = ibis.duckdb.connect(ddb, read_only=True) - - expr = con.sql(sql) - benchmark(expr.to_pyarrow) - - -@pytest.fixture -def diffs(): - return ibis.table( - { - "id": "int64", - "validation_name": "string", - "difference": "float64", - "pct_difference": "float64", - "pct_threshold": "float64", - "validation_status": "string", - }, - name="diffs", - ) - - -@pytest.fixture -def srcs(): - return ibis.table( - { - "id": "int64", - "validation_name": "string", - "validation_type": "string", - "aggregation_type": "string", - "table_name": "string", - "column_name": "string", - "primary_keys": "string", - "num_random_rows": "string", - "agg_value": "float64", - }, - name="srcs", - ) - - -@pytest.fixture -def nrels(): - return 300 - - -def make_big_union(t, nrels): - return ibis.union(*[t] * nrels) - - -@pytest.fixture -def src(srcs, nrels): - return make_big_union(srcs, nrels) - - -@pytest.fixture -def diff(diffs, nrels): - return make_big_union(diffs, nrels) - - -def test_big_eq_expr(benchmark, src, diff): - benchmark(ops.core.Node.equals, src.op(), diff.op()) - - -def test_big_join_expr(benchmark, src, diff): - benchmark(ir.Table.join, src, diff, ["validation_name"], how="outer") - - -def test_big_join_execute(benchmark, nrels): - pytest.importorskip("duckdb") - - con = ibis.duckdb.connect() - - # cache to avoid a request-per-union operand - src = make_big_union( - con.read_csv( - "https://github.com/ibis-project/ibis/files/12580336/source_pivot.csv" - ) - .rename(id="column0") - .cache(), - nrels, - ) - - diff = make_big_union( - con.read_csv( - "https://github.com/ibis-project/ibis/files/12580340/differences_pivot.csv" - ) - .rename(id="column0") - .cache(), - nrels, - ) - - expr = src.join(diff, ["validation_name"], how="outer") - t = benchmark.pedantic(expr.to_pyarrow, rounds=1, iterations=1, warmup_rounds=1) - assert len(t) diff --git a/ibis/backends/tests/test_binary.py b/ibis/backends/tests/test_binary.py index 0a5790c64631..1d9f7cfa0516 100644 --- a/ibis/backends/tests/test_binary.py +++ b/ibis/backends/tests/test_binary.py @@ -15,6 +15,7 @@ "sqlite": "blob", "trino": "varbinary", "postgres": "bytea", + "risingwave": "bytea", "flink": "BINARY(1) NOT NULL", } diff --git a/ibis/backends/tests/test_client.py b/ibis/backends/tests/test_client.py index 3f2336cafe9b..a6143d178934 100644 --- a/ibis/backends/tests/test_client.py +++ b/ibis/backends/tests/test_client.py @@ -25,7 +25,11 @@ import ibis.expr.datatypes as dt import ibis.expr.operations as ops from ibis.backends.conftest import ALL_BACKENDS -from ibis.backends.tests.errors import Py4JJavaError, PyDruidProgrammingError +from ibis.backends.tests.errors import ( + PsycoPg2InternalError, + Py4JJavaError, + PyDruidProgrammingError, +) from ibis.util import gen_name, guid if TYPE_CHECKING: @@ -115,7 +119,8 @@ def test_create_table(backend, con, temp_table, lamduh, sch): marks=[ pytest.mark.notyet(["clickhouse"], reason="Can't specify both"), pytest.mark.notyet( - ["pyspark", "trino", "exasol"], reason="No support for temp tables" + ["pyspark", "trino", "exasol", "risingwave"], + reason="No support for temp tables", ), pytest.mark.never(["polars"], reason="Everything in-memory is temp"), pytest.mark.broken(["mssql"], reason="Incorrect temp table syntax"), @@ -132,7 +137,8 @@ def test_create_table(backend, con, temp_table, lamduh, sch): id="temp, no overwrite", marks=[ pytest.mark.notyet( - ["pyspark", "trino", "exasol"], reason="No support for temp tables" + ["pyspark", "trino", "exasol", "risingwave"], + reason="No support for temp tables", ), pytest.mark.never(["polars"], reason="Everything in-memory is temp"), pytest.mark.broken(["mssql"], reason="Incorrect temp table syntax"), @@ -306,6 +312,11 @@ def tmpcon(alchemy_con): reason="mssql supports support temporary tables through naming conventions", ) @mark.notimpl(["exasol"], reason="Exasol does not support temporary tables") +@pytest.mark.never( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", +) def test_create_temporary_table_from_schema(tmpcon, new_schema): temp_table = f"_{guid()}" table = tmpcon.create_table(temp_table, schema=new_schema, temp=True) @@ -338,6 +349,7 @@ def test_create_temporary_table_from_schema(tmpcon, new_schema): "pandas", "polars", "postgres", + "risingwave", "snowflake", "sqlite", "trino", @@ -367,6 +379,11 @@ def test_rename_table(con, temp_table, temp_table_orig): raises=com.IbisError, reason="`tbl_properties` is required when creating table with schema", ) +@pytest.mark.never( + ["risingwave"], + raises=PsycoPg2InternalError, + reason='Feature is not yet implemented: column constraints "NOT NULL"', +) def test_nullable_input_output(con, temp_table): sch = ibis.schema( [("foo", "int64"), ("bar", dt.int64(nullable=False)), ("baz", "boolean")] @@ -412,7 +429,7 @@ def test_create_drop_view(ddl_con, temp_view): assert set(t_expr.schema().names) == set(v_expr.schema().names) -@mark.notimpl(["postgres", "polars"]) +@mark.notimpl(["postgres", "risingwave", "polars"]) @mark.notimpl( ["datafusion"], raises=NotImplementedError, @@ -622,6 +639,7 @@ def test_list_databases(alchemy_con): test_databases = { "sqlite": {"main"}, "postgres": {"postgres", "ibis_testing"}, + "risingwave": {"dev"}, "mssql": {"ibis_testing"}, "mysql": {"ibis_testing", "information_schema"}, "duckdb": {"memory"}, @@ -634,7 +652,7 @@ def test_list_databases(alchemy_con): @pytest.mark.never( - ["bigquery", "postgres", "mssql", "mysql", "oracle"], + ["bigquery", "postgres", "risingwave", "mssql", "mysql", "oracle"], reason="backend does not support client-side in-memory tables", raises=(sa.exc.OperationalError, TypeError, sa.exc.InterfaceError), ) @@ -1147,6 +1165,7 @@ def test_set_backend_url(url, monkeypatch): "pandas", "polars", "postgres", + "risingwave", "pyspark", "sqlite", ], @@ -1183,6 +1202,11 @@ def test_create_table_timestamp(con, temp_table): reason="mssql supports support temporary tables through naming conventions", ) @mark.notimpl(["exasol"], reason="Exasol does not support temporary tables") +@pytest.mark.never( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", +) def test_persist_expression_ref_count(backend, con, alltypes): non_persisted_table = alltypes.mutate(test_column="calculation") persisted_table = non_persisted_table.cache() @@ -1203,6 +1227,11 @@ def test_persist_expression_ref_count(backend, con, alltypes): reason="mssql supports support temporary tables through naming conventions", ) @mark.notimpl(["exasol"], reason="Exasol does not support temporary tables") +@pytest.mark.never( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", +) def test_persist_expression(backend, alltypes): non_persisted_table = alltypes.mutate(test_column="calculation", other_calc="xyz") persisted_table = non_persisted_table.cache() @@ -1217,6 +1246,11 @@ def test_persist_expression(backend, alltypes): reason="mssql supports support temporary tables through naming conventions", ) @mark.notimpl(["exasol"], reason="Exasol does not support temporary tables") +@pytest.mark.never( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", +) def test_persist_expression_contextmanager(backend, alltypes): non_cached_table = alltypes.mutate( test_column="calculation", other_column="big calc" @@ -1233,6 +1267,11 @@ def test_persist_expression_contextmanager(backend, alltypes): reason="mssql supports support temporary tables through naming conventions", ) @mark.notimpl(["exasol"], reason="Exasol does not support temporary tables") +@pytest.mark.never( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", +) def test_persist_expression_contextmanager_ref_count(backend, con, alltypes): non_cached_table = alltypes.mutate( test_column="calculation", other_column="big calc 2" @@ -1251,6 +1290,11 @@ def test_persist_expression_contextmanager_ref_count(backend, con, alltypes): ["mssql"], reason="mssql supports support temporary tables through naming conventions", ) +@pytest.mark.never( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", +) @mark.notimpl(["exasol"], reason="Exasol does not support temporary tables") def test_persist_expression_multiple_refs(backend, con, alltypes): non_cached_table = alltypes.mutate( @@ -1288,6 +1332,11 @@ def test_persist_expression_multiple_refs(backend, con, alltypes): reason="mssql supports support temporary tables through naming conventions", ) @mark.notimpl(["exasol"], reason="Exasol does not support temporary tables") +@pytest.mark.never( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", +) def test_persist_expression_repeated_cache(alltypes): non_cached_table = alltypes.mutate( test_column="calculation", other_column="big calc 2" @@ -1307,6 +1356,11 @@ def test_persist_expression_repeated_cache(alltypes): ["oracle"], reason="Oracle error message for a missing table/view doesn't include the name of the table", ) +@pytest.mark.never( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: CREATE TEMPORARY TABLE", +) def test_persist_expression_release(con, alltypes): non_cached_table = alltypes.mutate( test_column="calculation", other_column="big calc 3" @@ -1391,6 +1445,11 @@ def test_create_schema(con_create_schema): assert schema not in con_create_schema.list_schemas() +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: information_schema.schemata is not supported,", +) def test_list_schemas(con_create_schema): schemas = con_create_schema.list_schemas() assert len(schemas) == len(set(schemas)) diff --git a/ibis/backends/tests/test_column.py b/ibis/backends/tests/test_column.py index f26b2a876ded..f6b4bd8ee0f4 100644 --- a/ibis/backends/tests/test_column.py +++ b/ibis/backends/tests/test_column.py @@ -19,6 +19,7 @@ "pandas", "polars", "postgres", + "risingwave", "pyspark", "snowflake", "trino", diff --git a/ibis/backends/tests/test_examples.py b/ibis/backends/tests/test_examples.py index b73ea8e2f3da..113fe70102a4 100644 --- a/ibis/backends/tests/test_examples.py +++ b/ibis/backends/tests/test_examples.py @@ -16,7 +16,7 @@ reason="nix on linux cannot download duckdb extensions or data due to sandboxing", ) @pytest.mark.notimpl(["dask", "exasol", "pyspark"]) -@pytest.mark.notyet(["clickhouse", "druid", "impala", "mssql", "trino"]) +@pytest.mark.notyet(["clickhouse", "druid", "impala", "mssql", "trino", "risingwave"]) @pytest.mark.parametrize( ("example", "columns"), [ diff --git a/ibis/backends/tests/test_export.py b/ibis/backends/tests/test_export.py index 658850c6096e..25d833f20d36 100644 --- a/ibis/backends/tests/test_export.py +++ b/ibis/backends/tests/test_export.py @@ -254,6 +254,7 @@ def test_table_to_parquet_writer_kwargs(version, tmp_path, backend, awards_playe "pandas", "polars", "postgres", + "risingwave", "pyspark", "snowflake", "sqlite", @@ -390,6 +391,7 @@ def test_to_pyarrow_decimal(backend, dtype, pyarrow_dtype): "mysql", "oracle", "postgres", + "risingwave", "snowflake", "sqlite", "bigquery", @@ -488,7 +490,13 @@ def test_to_pandas_batches_empty_table(backend, con): @pytest.mark.notimpl(["flink"]) -@pytest.mark.parametrize("n", [None, 1]) +@pytest.mark.parametrize( + "n", + [ + None, + 1, + ], +) def test_to_pandas_batches_nonempty_table(backend, con, n): t = backend.functional_alltypes.limit(n) n = t.count().execute() @@ -498,7 +506,15 @@ def test_to_pandas_batches_nonempty_table(backend, con, n): @pytest.mark.notimpl(["flink"]) -@pytest.mark.parametrize("n", [None, 0, 1, 2]) +@pytest.mark.parametrize( + "n", + [ + None, + 0, + 1, + 2, + ], +) def test_to_pandas_batches_column(backend, con, n): t = backend.functional_alltypes.limit(n).timestamp_col n = t.count().execute() diff --git a/ibis/backends/tests/test_generic.py b/ibis/backends/tests/test_generic.py index 241c57ce8870..319edce5807d 100644 --- a/ibis/backends/tests/test_generic.py +++ b/ibis/backends/tests/test_generic.py @@ -25,6 +25,7 @@ ImpalaHiveServer2Error, MySQLProgrammingError, OracleDatabaseError, + PsycoPg2InternalError, PyDruidProgrammingError, PyODBCDataError, PyODBCProgrammingError, @@ -43,6 +44,7 @@ "sqlite": "null", "trino": "unknown", "postgres": "null", + "risingwave": "null", } @@ -66,6 +68,7 @@ def test_null_literal(con, backend): "trino": "boolean", "duckdb": "BOOLEAN", "postgres": "boolean", + "risingwave": "boolean", "flink": "BOOLEAN NOT NULL", } @@ -150,6 +153,7 @@ def test_isna(backend, alltypes, col, value, filt): "duckdb", "impala", "postgres", + "risingwave", "mysql", "snowflake", "polars", @@ -307,6 +311,7 @@ def test_filter(backend, alltypes, sorted_df, predicate_fn, expected_fn): "impala", "mysql", "postgres", + "risingwave", "sqlite", "snowflake", "polars", @@ -540,6 +545,11 @@ def test_order_by(backend, alltypes, df, key, df_kwargs): @pytest.mark.notimpl(["dask", "pandas", "polars", "mssql", "druid"]) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function random() does not exist", +) def test_order_by_random(alltypes): expr = alltypes.filter(_.id < 100).order_by(ibis.random()).limit(5) r1 = expr.execute() @@ -761,6 +771,11 @@ def test_correlated_subquery(alltypes): @pytest.mark.notimpl(["polars", "pyspark"]) +@pytest.mark.broken( + ["risingwave"], + raises=AssertionError, + reason='DataFrame.iloc[:, 0] (column name="playerID") are different', +) def test_uncorrelated_subquery(backend, batting, batting_df): subset_batting = batting[batting.yearID <= 2000] expr = batting[_.yearID == subset_batting.yearID.max()]["playerID", "yearID"] @@ -833,6 +848,11 @@ def test_typeof(con): @pytest.mark.notimpl(["pyspark"], condition=is_older_than("pyspark", "3.5.0")) @pytest.mark.notyet(["dask"], reason="not supported by the backend") @pytest.mark.notyet(["exasol"], raises=ExaQueryError, reason="not supported by exasol") +@pytest.mark.broken( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="https://github.com/risingwavelabs/risingwave/issues/1343", +) def test_isin_uncorrelated( backend, batting, awards_players, batting_df, awards_players_df ): @@ -985,6 +1005,11 @@ def test_memtable_column_naming_mismatch(backend, con, monkeypatch, df, columns) ) @pytest.mark.notimpl(["druid", "flink"], reason="no sqlglot dialect", raises=ValueError) @pytest.mark.notimpl(["exasol"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl( + ["risingwave"], + raises=ValueError, + reason="risingwave doesn't support sqlglot.dialects.dialect.Dialect", +) def test_many_subqueries(con, snapshot): def query(t, group_cols): t2 = t.mutate(key=ibis.row_number().over(ibis.window(order_by=group_cols))) @@ -1126,6 +1151,11 @@ def test_pivot_wider(backend): ["exasol"], raises=com.OperationNotDefinedError, ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function last(double precision) does not exist, do you mean left or least", +) def test_distinct_on_keep(backend, on, keep): from ibis import _ @@ -1191,6 +1221,11 @@ def test_distinct_on_keep(backend, on, keep): raises=com.OperationNotDefinedError, reason="backend doesn't implement deduplication", ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function first(double precision) does not exist", +) def test_distinct_on_keep_is_none(backend, on): from ibis import _ @@ -1209,7 +1244,7 @@ def test_distinct_on_keep_is_none(backend, on): assert len(result) == len(expected) -@pytest.mark.notimpl(["dask", "pandas", "postgres", "flink", "exasol"]) +@pytest.mark.notimpl(["dask", "pandas", "postgres", "risingwave", "flink", "exasol"]) @pytest.mark.notyet( [ "sqlite", @@ -1226,7 +1261,87 @@ def test_hash_consistent(backend, alltypes): assert h1.dtype in ("i8", "uint64") # polars likes returning uint64 for this -@pytest.mark.notimpl(["pandas", "dask", "oracle", "snowflake", "sqlite"]) +@pytest.mark.notimpl(["trino", "oracle", "exasol", "snowflake"]) +@pytest.mark.notyet( + [ + "dask", + "datafusion", + "druid", + "duckdb", + "flink", + "impala", + "mysql", + "pandas", + "polars", + "postgres", + "pyspark", + "risingwave", + "sqlite", + ] +) +def test_hashbytes(backend, alltypes): + h1 = alltypes.order_by("id").string_col.hashbytes().execute(limit=10) + df = alltypes.order_by("id").execute(limit=10) + + import hashlib + + def hash_256(col): + return hashlib.sha256(col.encode()).digest() + + h2 = df["string_col"].apply(hash_256).rename("HashBytes(string_col)") + + backend.assert_series_equal(h1, h2) + + +@pytest.mark.notimpl( + [ + "bigquery", + "clickhouse", + "dask", + "datafusion", + "exasol", + "flink", + "impala", + "mysql", + "oracle", + "pandas", + "polars", + "postgres", + "risingwave", + "snowflake", + "trino", + ] +) +@pytest.mark.notyet( + [ + "druid", + "polars", + "sqlite", + ] +) +def test_hexdigest(backend, alltypes): + h1 = alltypes.order_by("id").string_col.hexdigest().execute(limit=10) + df = alltypes.order_by("id").execute(limit=10) + + import hashlib + + def hash_256(col): + return hashlib.sha256(col.encode()).hexdigest() + + h2 = df["string_col"].apply(hash_256).rename("HexDigest(string_col)") + + backend.assert_series_equal(h1, h2) + + +@pytest.mark.notimpl( + [ + "pandas", + "dask", + "oracle", + "snowflake", + "sqlite", + ] +) @pytest.mark.parametrize( ("from_val", "to_type", "expected"), [ @@ -1275,6 +1390,7 @@ def test_try_cast(con, from_val, to_type, expected): "oracle", "pandas", "postgres", + "risingwave", "snowflake", "sqlite", ] @@ -1312,6 +1428,7 @@ def test_try_cast_null(con, from_val, to_type): "mysql", "oracle", "postgres", + "risingwave", "snowflake", "sqlite", "exasol", @@ -1337,6 +1454,7 @@ def test_try_cast_table(backend, con): "mysql", "oracle", "postgres", + "risingwave", "snowflake", "sqlite", "exasol", @@ -1520,6 +1638,11 @@ def test_static_table_slice(backend, slc, expected_count_fn): raises=com.UnsupportedArgumentError, reason="Removed half-baked dynamic offset functionality for now", ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="risingwave doesn't support limit/offset", +) @pytest.mark.notyet( ["trino"], raises=TrinoUserError, @@ -1610,6 +1733,11 @@ def test_dynamic_table_slice(backend, slc, expected_count_fn): reason="doesn't support dynamic limit/offset; compiles incorrectly in sqlglot", raises=AssertionError, ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="risingwave doesn't support limit/offset", +) def test_dynamic_table_slice_with_computed_offset(backend): t = backend.functional_alltypes @@ -1629,6 +1757,11 @@ def test_dynamic_table_slice_with_computed_offset(backend): @pytest.mark.notimpl(["druid", "flink", "polars", "snowflake"]) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function random() does not exist", +) def test_sample(backend): t = backend.functional_alltypes.filter(_.int_col >= 2) @@ -1645,6 +1778,11 @@ def test_sample(backend): @pytest.mark.notimpl(["druid", "flink", "polars", "snowflake"]) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function random() does not exist", +) def test_sample_memtable(con, backend): df = pd.DataFrame({"x": [1, 2, 3, 4]}) res = con.execute(ibis.memtable(df).sample(0.5)) @@ -1665,6 +1803,7 @@ def test_sample_memtable(con, backend): "oracle", "polars", "postgres", + "risingwave", "snowflake", "sqlite", "trino", diff --git a/ibis/backends/tests/test_join.py b/ibis/backends/tests/test_join.py index a20cd61a98d3..887c10547b6d 100644 --- a/ibis/backends/tests/test_join.py +++ b/ibis/backends/tests/test_join.py @@ -198,7 +198,7 @@ def test_semi_join_topk(con, batting, awards_players, func): @pytest.mark.notimpl(["dask", "druid", "exasol", "oracle"]) @pytest.mark.notimpl( - ["postgres", "mssql"], + ["postgres", "mssql", "risingwave"], raises=com.IbisTypeError, reason="postgres can't handle null types columns", ) diff --git a/ibis/backends/tests/test_json.py b/ibis/backends/tests/test_json.py index 78d379ae0bde..c52721e2a293 100644 --- a/ibis/backends/tests/test_json.py +++ b/ibis/backends/tests/test_json.py @@ -40,13 +40,16 @@ ["flink"], reason="https://github.com/ibis-project/ibis/pull/6920#discussion_r1373212503", ) +@pytest.mark.broken( + ["risingwave"], reason="TODO(Kexiang): order mismatch in array", strict=False +) def test_json_getitem(json_t, expr_fn, expected): expr = expr_fn(json_t) result = expr.execute() tm.assert_series_equal(result.fillna(pd.NA), expected.fillna(pd.NA)) -@pytest.mark.notimpl(["dask", "mysql", "pandas"]) +@pytest.mark.notimpl(["dask", "mysql", "pandas", "risingwave"]) @pytest.mark.notyet(["bigquery", "sqlite"], reason="doesn't support maps") @pytest.mark.notyet(["postgres"], reason="only supports map") @pytest.mark.notyet( @@ -70,7 +73,7 @@ def test_json_map(backend, json_t): backend.assert_series_equal(result, expected) -@pytest.mark.notimpl(["dask", "mysql", "pandas"]) +@pytest.mark.notimpl(["dask", "mysql", "pandas", "risingwave"]) @pytest.mark.notyet(["sqlite"], reason="doesn't support arrays") @pytest.mark.notyet( ["pyspark", "trino", "flink"], reason="should work but doesn't deserialize JSON" diff --git a/ibis/backends/tests/test_map.py b/ibis/backends/tests/test_map.py index cb0386dc0ff8..6b8191b43534 100644 --- a/ibis/backends/tests/test_map.py +++ b/ibis/backends/tests/test_map.py @@ -8,6 +8,7 @@ import ibis import ibis.common.exceptions as exc import ibis.expr.datatypes as dt +from ibis.backends.tests.errors import PsycoPg2InternalError pytestmark = [ pytest.mark.never( @@ -34,6 +35,11 @@ def test_map_table(backend): @pytest.mark.xfail_version( duckdb=["duckdb<0.8.0"], raises=exc.UnsupportedOperationError ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function hstore(character varying[], character varying[]) does not exist", +) def test_column_map_values(backend): table = backend.map expr = table.select("idx", vals=table.kv.values()).order_by("idx") @@ -64,6 +70,11 @@ def test_column_map_merge(backend): raises=exc.OperationNotDefinedError, reason="No translation rule for ", ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function hstore(character varying[], character varying[]) does not exist", +) def test_literal_map_keys(con): mapping = ibis.literal({"1": "a", "2": "b"}) expr = mapping.keys().name("tmp") @@ -79,6 +90,11 @@ def test_literal_map_keys(con): raises=exc.OperationNotDefinedError, reason="No translation rule for ", ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function hstore(character varying[], character varying[]) does not exist", +) def test_literal_map_values(con): mapping = ibis.literal({"1": "a", "2": "b"}) expr = mapping.values().name("tmp") @@ -87,7 +103,7 @@ def test_literal_map_values(con): assert np.array_equal(result, ["a", "b"]) -@pytest.mark.notimpl(["postgres"]) +@pytest.mark.notimpl(["postgres", "risingwave"]) @pytest.mark.notimpl( ["flink"], raises=exc.OperationNotDefinedError, @@ -103,7 +119,9 @@ def test_scalar_isin_literal_map_keys(con): assert con.execute(false) == False # noqa: E712 -@pytest.mark.notyet(["postgres"], reason="only support maps of string -> string") +@pytest.mark.notyet( + ["postgres", "risingwave"], reason="only support maps of string -> string" +) @pytest.mark.notimpl( ["flink"], raises=exc.OperationNotDefinedError, @@ -124,6 +142,11 @@ def test_map_scalar_contains_key_scalar(con): raises=exc.OperationNotDefinedError, reason="No translation rule for ", ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function hstore(character varying[], character varying[]) does not exist", +) def test_map_scalar_contains_key_column(backend, alltypes, df): value = {"1": "a", "3": "c"} mapping = ibis.literal(value) @@ -133,7 +156,9 @@ def test_map_scalar_contains_key_column(backend, alltypes, df): backend.assert_series_equal(result, expected) -@pytest.mark.notyet(["postgres"], reason="only support maps of string -> string") +@pytest.mark.notyet( + ["postgres", "risingwave"], reason="only support maps of string -> string" +) @pytest.mark.notimpl( ["flink"], raises=exc.OperationNotDefinedError, @@ -149,7 +174,9 @@ def test_map_column_contains_key_scalar(backend, alltypes, df): backend.assert_series_equal(result, series) -@pytest.mark.notyet(["postgres"], reason="only support maps of string -> string") +@pytest.mark.notyet( + ["postgres", "risingwave"], reason="only support maps of string -> string" +) @pytest.mark.notimpl( ["flink"], raises=exc.OperationNotDefinedError, @@ -164,7 +191,9 @@ def test_map_column_contains_key_column(alltypes): assert result.all() -@pytest.mark.notyet(["postgres"], reason="only support maps of string -> string") +@pytest.mark.notyet( + ["postgres", "risingwave"], reason="only support maps of string -> string" +) @pytest.mark.notimpl( ["flink"], raises=exc.OperationNotDefinedError, @@ -183,6 +212,11 @@ def test_literal_map_merge(con): raises=NotImplementedError, reason="No translation rule for map", ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function hstore(character varying[], character varying[]) does not exist", +) def test_literal_map_getitem_broadcast(backend, alltypes, df): value = {"1": "a", "2": "b"} @@ -200,6 +234,11 @@ def test_literal_map_getitem_broadcast(backend, alltypes, df): raises=NotImplementedError, reason="No translation rule for map", ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function hstore(character varying[], character varying[]) does not exist", +) def test_literal_map_get_broadcast(backend, alltypes, df): value = {"1": "a", "2": "b"} @@ -220,19 +259,27 @@ def test_literal_map_get_broadcast(backend, alltypes, df): [1, 2], id="string", marks=pytest.mark.notyet( - ["postgres"], reason="only support maps of string -> string" + ["postgres", "risingwave"], + reason="only support maps of string -> string", ), ), param(["a", "b"], ["1", "2"], id="int"), ], ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function hstore(character varying[], character varying[]) does not exist", +) def test_map_construct_dict(con, keys, values): expr = ibis.map(keys, values) result = con.execute(expr.name("tmp")) assert result == dict(zip(keys, values)) -@pytest.mark.notyet(["postgres"], reason="only support maps of string -> string") +@pytest.mark.notyet( + ["postgres", "risingwave"], reason="only support maps of string -> string" +) @pytest.mark.notimpl( ["flink"], raises=exc.OperationNotDefinedError, @@ -246,7 +293,9 @@ def test_map_construct_array_column(con, alltypes, df): assert result.to_list() == expected.to_list() -@pytest.mark.notyet(["postgres"], reason="only support maps of string -> string") +@pytest.mark.notyet( + ["postgres", "risingwave"], reason="only support maps of string -> string" +) @pytest.mark.notimpl( ["flink"], raises=NotImplementedError, @@ -258,7 +307,9 @@ def test_map_get_with_compatible_value_smaller(con): assert con.execute(expr) == 3 -@pytest.mark.notyet(["postgres"], reason="only support maps of string -> string") +@pytest.mark.notyet( + ["postgres", "risingwave"], reason="only support maps of string -> string" +) @pytest.mark.notimpl( ["flink"], raises=NotImplementedError, @@ -270,7 +321,9 @@ def test_map_get_with_compatible_value_bigger(con): assert con.execute(expr) == 3000 -@pytest.mark.notyet(["postgres"], reason="only support maps of string -> string") +@pytest.mark.notyet( + ["postgres", "risingwave"], reason="only support maps of string -> string" +) @pytest.mark.notimpl( ["flink"], raises=NotImplementedError, @@ -283,7 +336,9 @@ def test_map_get_with_incompatible_value_different_kind(con): @pytest.mark.parametrize("null_value", [None, ibis.NA]) -@pytest.mark.notyet(["postgres"], reason="only support maps of string -> string") +@pytest.mark.notyet( + ["postgres", "risingwave"], reason="only support maps of string -> string" +) @pytest.mark.notimpl( ["flink"], raises=NotImplementedError, @@ -303,6 +358,11 @@ def test_map_get_with_null_on_not_nullable(con, null_value): raises=NotImplementedError, reason="No translation rule for map", ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function hstore(character varying[], character varying[]) does not exist", +) def test_map_get_with_null_on_null_type_with_null(con, null_value): value = ibis.literal({"A": None, "B": None}) expr = value.get("C", null_value) @@ -310,7 +370,9 @@ def test_map_get_with_null_on_null_type_with_null(con, null_value): assert pd.isna(result) -@pytest.mark.notyet(["postgres"], reason="only support maps of string -> string") +@pytest.mark.notyet( + ["postgres", "risingwave"], reason="only support maps of string -> string" +) @pytest.mark.notimpl( ["flink"], raises=NotImplementedError, @@ -327,6 +389,11 @@ def test_map_get_with_null_on_null_type_with_non_null(con): raises=exc.IbisError, reason="`tbl_properties` is required when creating table with schema", ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function hstore(character varying[], character varying[]) does not exist", +) def test_map_create_table(con, temp_table): t = con.create_table( temp_table, @@ -340,6 +407,11 @@ def test_map_create_table(con, temp_table): raises=exc.OperationNotDefinedError, reason="No translation rule for ", ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function hstore(character varying[], character varying[]) does not exist", +) def test_map_length(con): expr = ibis.literal(dict(a="A", b="B")).length() assert con.execute(expr) == 2 diff --git a/ibis/backends/tests/test_network.py b/ibis/backends/tests/test_network.py index e6048ee907b2..dca5815c6855 100644 --- a/ibis/backends/tests/test_network.py +++ b/ibis/backends/tests/test_network.py @@ -20,6 +20,7 @@ "trino": "varchar(17)", "impala": "STRING", "postgres": "text", + "risingwave": "text", "flink": "CHAR(17) NOT NULL", } @@ -50,6 +51,7 @@ def test_macaddr_literal(con, backend): "trino": "127.0.0.1", "impala": "127.0.0.1", "postgres": "127.0.0.1", + "risingwave": "127.0.0.1", "pandas": "127.0.0.1", "pyspark": "127.0.0.1", "mysql": "127.0.0.1", @@ -67,6 +69,7 @@ def test_macaddr_literal(con, backend): "trino": "varchar(9)", "impala": "STRING", "postgres": "text", + "risingwave": "text", "flink": "CHAR(9) NOT NULL", }, id="ipv4", @@ -82,6 +85,7 @@ def test_macaddr_literal(con, backend): "trino": "2001:db8::1", "impala": "2001:db8::1", "postgres": "2001:db8::1", + "risingwave": "2001:db8::1", "pandas": "2001:db8::1", "pyspark": "2001:db8::1", "mysql": "2001:db8::1", @@ -99,6 +103,7 @@ def test_macaddr_literal(con, backend): "trino": "varchar(11)", "impala": "STRING", "postgres": "text", + "risingwave": "text", "flink": "CHAR(11) NOT NULL", }, id="ipv6", diff --git a/ibis/backends/tests/test_numeric.py b/ibis/backends/tests/test_numeric.py index 049aa20f0a8d..7a45f7c52147 100644 --- a/ibis/backends/tests/test_numeric.py +++ b/ibis/backends/tests/test_numeric.py @@ -25,6 +25,7 @@ MySQLOperationalError, OracleDatabaseError, PsycoPg2DivisionByZero, + PsycoPg2InternalError, Py4JError, PyDruidProgrammingError, PyODBCDataError, @@ -52,6 +53,7 @@ "trino": "integer", "duckdb": "TINYINT", "postgres": "integer", + "risingwave": "integer", "flink": "TINYINT NOT NULL", }, id="int8", @@ -67,6 +69,7 @@ "trino": "integer", "duckdb": "SMALLINT", "postgres": "integer", + "risingwave": "integer", "flink": "SMALLINT NOT NULL", }, id="int16", @@ -82,6 +85,7 @@ "trino": "integer", "duckdb": "INTEGER", "postgres": "integer", + "risingwave": "integer", "flink": "INT NOT NULL", }, id="int32", @@ -97,6 +101,7 @@ "trino": "integer", "duckdb": "BIGINT", "postgres": "integer", + "risingwave": "integer", "flink": "BIGINT NOT NULL", }, id="int64", @@ -112,6 +117,7 @@ "trino": "integer", "duckdb": "UTINYINT", "postgres": "integer", + "risingwave": "integer", "flink": "TINYINT NOT NULL", }, id="uint8", @@ -127,6 +133,7 @@ "trino": "integer", "duckdb": "USMALLINT", "postgres": "integer", + "risingwave": "integer", "flink": "SMALLINT NOT NULL", }, id="uint16", @@ -142,6 +149,7 @@ "trino": "integer", "duckdb": "UINTEGER", "postgres": "integer", + "risingwave": "integer", "flink": "INT NOT NULL", }, id="uint32", @@ -157,6 +165,7 @@ "trino": "integer", "duckdb": "UBIGINT", "postgres": "integer", + "risingwave": "integer", "flink": "BIGINT NOT NULL", }, id="uint64", @@ -172,6 +181,7 @@ "trino": "real", "duckdb": "FLOAT", "postgres": "numeric", + "risingwave": "numeric", "flink": "FLOAT NOT NULL", }, marks=[ @@ -199,6 +209,7 @@ "trino": "real", "duckdb": "FLOAT", "postgres": "numeric", + "risingwave": "numeric", "flink": "FLOAT NOT NULL", }, id="float32", @@ -214,6 +225,7 @@ "trino": "double", "duckdb": "DOUBLE", "postgres": "numeric", + "risingwave": "numeric", "flink": "DOUBLE NOT NULL", }, id="float64", @@ -245,6 +257,7 @@ def test_numeric_literal(con, backend, expr, expected_types): "duckdb": decimal.Decimal("1.1"), "impala": decimal.Decimal("1"), "postgres": decimal.Decimal("1.1"), + "risingwave": decimal.Decimal("1.1"), "pandas": decimal.Decimal("1.1"), "pyspark": decimal.Decimal("1.1"), "mysql": decimal.Decimal("1"), @@ -263,6 +276,7 @@ def test_numeric_literal(con, backend, expr, expected_types): "trino": "decimal(18,3)", "duckdb": "DECIMAL(18,3)", "postgres": "numeric", + "risingwave": "numeric", "flink": "DECIMAL(38, 18) NOT NULL", }, marks=[ @@ -285,6 +299,7 @@ def test_numeric_literal(con, backend, expr, expected_types): "duckdb": decimal.Decimal("1.100000000"), "impala": decimal.Decimal("1.1"), "postgres": decimal.Decimal("1.1"), + "risingwave": decimal.Decimal("1.1"), "pandas": decimal.Decimal("1.1"), "pyspark": decimal.Decimal("1.1"), "mysql": decimal.Decimal("1.1"), @@ -305,6 +320,7 @@ def test_numeric_literal(con, backend, expr, expected_types): "trino": "decimal(38,9)", "duckdb": "DECIMAL(38,9)", "postgres": "numeric", + "risingwave": "numeric", "flink": "DECIMAL(38, 9) NOT NULL", }, marks=[pytest.mark.notimpl(["exasol"], raises=ExaQueryError)], @@ -318,6 +334,7 @@ def test_numeric_literal(con, backend, expr, expected_types): "sqlite": decimal.Decimal("1.1"), "dask": decimal.Decimal("1.1"), "postgres": decimal.Decimal("1.1"), + "risingwave": decimal.Decimal("1.1"), "pandas": decimal.Decimal("1.1"), "pyspark": decimal.Decimal("1.1"), "clickhouse": decimal.Decimal( @@ -333,6 +350,7 @@ def test_numeric_literal(con, backend, expr, expected_types): "trino": "decimal(2,1)", "duckdb": "DECIMAL(18,3)", "postgres": "numeric", + "risingwave": "numeric", }, marks=[ pytest.mark.notimpl(["exasol"], raises=ExaQueryError), @@ -370,6 +388,7 @@ def test_numeric_literal(con, backend, expr, expected_types): "bigquery": float("inf"), "sqlite": decimal.Decimal("Infinity"), "postgres": decimal.Decimal("Infinity"), + "risingwave": decimal.Decimal("Infinity"), "pandas": decimal.Decimal("Infinity"), "dask": decimal.Decimal("Infinity"), "pyspark": decimal.Decimal("Infinity"), @@ -380,6 +399,7 @@ def test_numeric_literal(con, backend, expr, expected_types): "sqlite": "real", "postgres": "numeric", "duckdb": "FLOAT", + "risingwave": "numeric", }, marks=[ pytest.mark.broken( @@ -439,6 +459,7 @@ def test_numeric_literal(con, backend, expr, expected_types): "bigquery": float("-inf"), "sqlite": decimal.Decimal("-Infinity"), "postgres": decimal.Decimal("-Infinity"), + "risingwave": decimal.Decimal("-Infinity"), "pandas": decimal.Decimal("-Infinity"), "dask": decimal.Decimal("-Infinity"), "pyspark": decimal.Decimal("-Infinity"), @@ -449,6 +470,7 @@ def test_numeric_literal(con, backend, expr, expected_types): "sqlite": "real", "postgres": "numeric", "duckdb": "FLOAT", + "risingwave": "numeric", }, marks=[ pytest.mark.broken( @@ -509,6 +531,7 @@ def test_numeric_literal(con, backend, expr, expected_types): "snowflake": float("nan"), "sqlite": None, "postgres": float("nan"), + "risingwave": float("nan"), "pandas": decimal.Decimal("NaN"), "dask": decimal.Decimal("NaN"), "pyspark": decimal.Decimal("NaN"), @@ -521,6 +544,7 @@ def test_numeric_literal(con, backend, expr, expected_types): "sqlite": "null", "postgres": "numeric", "duckdb": "FLOAT", + "risingwave": "numeric", }, marks=[ pytest.mark.broken( @@ -730,14 +754,53 @@ def test_isnan_isinf( L(5.556).log(2), math.log(5.556, 2), id="log-base", - marks=[pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError)], + marks=[ + pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function log10(numeric, numeric) does not exist", + ), + ], + ), + param( + L(5.556).ln(), + math.log(5.556), + id="ln", ), param(L(5.556).ln(), math.log(5.556), id="ln"), param( L(5.556).log2(), math.log(5.556, 2), id="log2", - marks=[pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError)], + marks=[ + pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function log10(numeric, numeric) does not exist", + ), + ], + ), + param( + L(5.556).log10(), + math.log10(5.556), + id="log10", + ), + param( + L(5.556).radians(), + math.radians(5.556), + id="radians", + ), + param( + L(5.556).degrees(), + math.degrees(5.556), + id="degrees", + ), + param( + L(11) % 3, + 11 % 3, + id="mod", ), param(L(5.556).log10(), math.log10(5.556), id="log10"), param(L(5.556).radians(), math.radians(5.556), id="radians"), @@ -874,7 +937,14 @@ def test_simple_math_functions_columns( param( lambda t: t.double_col.add(1).log(2), lambda t: np.log2(t.double_col + 1), - marks=[pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError)], + marks=[ + pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function log10(numeric, numeric) does not exist", + ), + ], id="log2", ), param( @@ -908,6 +978,11 @@ def test_simple_math_functions_columns( reason="Base greatest(9000, t0.bigint_col) for logarithm not supported!", ), pytest.mark.notimpl(["polars"], raises=com.UnsupportedArgumentError), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function log10(numeric, numeric) does not exist", + ), ], ), ], @@ -1211,6 +1286,7 @@ def test_divide_by_zero(backend, alltypes, df, column, denominator): ( { "postgres": None, + "risingwave": None, "mysql": 10, "snowflake": 38, "trino": 18, @@ -1220,6 +1296,7 @@ def test_divide_by_zero(backend, alltypes, df, column, denominator): }, { "postgres": None, + "risingwave": None, "mysql": 0, "snowflake": 0, "trino": 3, @@ -1247,6 +1324,7 @@ def test_divide_by_zero(backend, alltypes, df, column, denominator): "snowflake", "trino", "postgres", + "risingwave", "mysql", "druid", "mssql", @@ -1289,6 +1367,11 @@ def test_sa_default_numeric_precision_and_scale( @pytest.mark.notimpl(["dask", "pandas", "polars"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function random() does not exist", +) def test_random(con): expr = ibis.random() result = con.execute(expr) diff --git a/ibis/backends/tests/test_param.py b/ibis/backends/tests/test_param.py index 3fea928999e2..cc99183ec1bc 100644 --- a/ibis/backends/tests/test_param.py +++ b/ibis/backends/tests/test_param.py @@ -11,7 +11,11 @@ import ibis import ibis.expr.datatypes as dt from ibis import _ -from ibis.backends.tests.errors import OracleDatabaseError, Py4JJavaError +from ibis.backends.tests.errors import ( + OracleDatabaseError, + PsycoPg2InternalError, + Py4JJavaError, +) @pytest.mark.parametrize( @@ -76,6 +80,7 @@ def test_scalar_param_array(con): "impala", "flink", "postgres", + "risingwave", "druid", "oracle", "exasol", @@ -108,6 +113,11 @@ def test_scalar_param_struct(con): "sql= SELECT MAP_FROM_ARRAYS(ARRAY['a', 'b', 'c'], ARRAY['ghi', 'def', 'abc']) '[' 'b' ']' AS `MapGet(param_0, 'b', None)`" ), ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function make_date(integer, integer, integer) does not exist", +) def test_scalar_param_map(con): value = {"a": "ghi", "b": "def", "c": "abc"} param = ibis.param(dt.Map(dt.string, dt.string)) @@ -203,6 +213,7 @@ def test_scalar_param_date(backend, alltypes, value): @pytest.mark.notimpl( [ "postgres", + "risingwave", "datafusion", "clickhouse", "polars", diff --git a/ibis/backends/tests/test_register.py b/ibis/backends/tests/test_register.py index 4e1739e30cf0..d967e45d80c7 100644 --- a/ibis/backends/tests/test_register.py +++ b/ibis/backends/tests/test_register.py @@ -93,6 +93,7 @@ def gzip_csv(data_dir, tmp_path): "mysql", "pandas", "postgres", + "risingwave", "snowflake", "sqlite", "trino", @@ -119,6 +120,7 @@ def test_register_csv(con, data_dir, fname, in_table_name, out_table_name): "mysql", "pandas", "postgres", + "risingwave", "snowflake", "sqlite", "trino", @@ -142,6 +144,7 @@ def test_register_csv_gz(con, data_dir, gzip_csv): "mysql", "pandas", "postgres", + "risingwave", "snowflake", "sqlite", "trino", @@ -198,6 +201,7 @@ def read_table(path: Path) -> Iterator[tuple[str, pa.Table]]: "mysql", "pandas", "postgres", + "risingwave", "snowflake", "sqlite", "trino", @@ -234,6 +238,7 @@ def test_register_parquet( "mysql", "pandas", "postgres", + "risingwave", "pyspark", "snowflake", "sqlite", @@ -273,6 +278,7 @@ def test_register_iterator_parquet( "mysql", "pandas", "postgres", + "risingwave", "pyspark", "snowflake", "sqlite", @@ -303,6 +309,7 @@ def test_register_pandas(con): "mysql", "pandas", "postgres", + "risingwave", "pyspark", "snowflake", "sqlite", @@ -328,6 +335,7 @@ def test_register_pyarrow_tables(con): "mysql", "pandas", "postgres", + "risingwave", "snowflake", "sqlite", "trino", @@ -370,6 +378,7 @@ def test_csv_reregister_schema(con, tmp_path): "pandas", "polars", "postgres", + "risingwave", "pyspark", "snowflake", "sqlite", @@ -400,7 +409,7 @@ def test_register_garbage(con, monkeypatch): ], ) @pytest.mark.notyet( - ["flink", "impala", "mssql", "mysql", "postgres", "sqlite", "trino"] + ["flink", "impala", "mssql", "mysql", "postgres", "risingwave", "sqlite", "trino"] ) def test_read_parquet(con, tmp_path, data_dir, fname, in_table_name): pq = pytest.importorskip("pyarrow.parquet") @@ -431,7 +440,17 @@ def ft_data(data_dir): @pytest.mark.notyet( - ["flink", "impala", "mssql", "mysql", "pandas", "postgres", "sqlite", "trino"] + [ + "flink", + "impala", + "mssql", + "mysql", + "pandas", + "postgres", + "risingwave", + "sqlite", + "trino", + ] ) def test_read_parquet_glob(con, tmp_path, ft_data): pq = pytest.importorskip("pyarrow.parquet") @@ -450,7 +469,17 @@ def test_read_parquet_glob(con, tmp_path, ft_data): @pytest.mark.notyet( - ["flink", "impala", "mssql", "mysql", "pandas", "postgres", "sqlite", "trino"] + [ + "flink", + "impala", + "mssql", + "mysql", + "pandas", + "postgres", + "risingwave", + "sqlite", + "trino", + ] ) def test_read_csv_glob(con, tmp_path, ft_data): pc = pytest.importorskip("pyarrow.csv") @@ -479,6 +508,7 @@ def test_read_csv_glob(con, tmp_path, ft_data): "mysql", "pandas", "postgres", + "risingwave", "sqlite", "trino", ] @@ -527,7 +557,7 @@ def num_diamonds(data_dir): [param(None, id="default"), param("fancy_stones", id="file_name")], ) @pytest.mark.notyet( - ["flink", "impala", "mssql", "mysql", "postgres", "sqlite", "trino"] + ["flink", "impala", "mssql", "mysql", "postgres", "risingwave", "sqlite", "trino"] ) def test_read_csv(con, data_dir, in_table_name, num_diamonds): fname = "diamonds.csv" diff --git a/ibis/backends/tests/test_set_ops.py b/ibis/backends/tests/test_set_ops.py index 0fb52fa10f0f..4df076da7f97 100644 --- a/ibis/backends/tests/test_set_ops.py +++ b/ibis/backends/tests/test_set_ops.py @@ -10,7 +10,7 @@ import ibis.common.exceptions as com import ibis.expr.types as ir from ibis import _ -from ibis.backends.tests.errors import PyDruidProgrammingError +from ibis.backends.tests.errors import PsycoPg2InternalError, PyDruidProgrammingError @pytest.fixture @@ -67,19 +67,26 @@ def test_union_mixed_distinct(backend, union_subsets): [ param( False, - marks=pytest.mark.notyet( - [ - "impala", - "bigquery", - "dask", - "pandas", - "sqlite", - "snowflake", - "mssql", - "exasol", - ], - reason="backend doesn't support INTERSECT ALL", - ), + marks=[ + pytest.mark.notyet( + [ + "impala", + "bigquery", + "dask", + "pandas", + "sqlite", + "snowflake", + "mssql", + "exasol", + ], + reason="backend doesn't support INTERSECT ALL", + ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: INTERSECT all", + ), + ], id="all", ), param(True, id="distinct"), @@ -114,19 +121,26 @@ def test_intersect(backend, alltypes, df, distinct): [ param( False, - marks=pytest.mark.notyet( - [ - "impala", - "bigquery", - "dask", - "pandas", - "sqlite", - "snowflake", - "mssql", - "exasol", - ], - reason="backend doesn't support EXCEPT ALL", - ), + marks=[ + pytest.mark.notyet( + [ + "impala", + "bigquery", + "dask", + "pandas", + "sqlite", + "snowflake", + "mssql", + "exasol", + ], + reason="backend doesn't support EXCEPT ALL", + ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: EXCEPT all", + ), + ], id="all", ), param(True, id="distinct"), @@ -193,18 +207,25 @@ def test_top_level_union(backend, con, alltypes, distinct): True, param( False, - marks=pytest.mark.notimpl( - [ - "impala", - "bigquery", - "dask", - "mssql", - "pandas", - "snowflake", - "sqlite", - "exasol", - ] - ), + marks=[ + pytest.mark.notimpl( + [ + "impala", + "bigquery", + "dask", + "mssql", + "pandas", + "snowflake", + "sqlite", + "exasol", + ] + ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: INTERSECT all", + ), + ], ), ], ) diff --git a/ibis/backends/tests/test_string.py b/ibis/backends/tests/test_string.py index 3ff40d13ce67..efb14b4939e6 100644 --- a/ibis/backends/tests/test_string.py +++ b/ibis/backends/tests/test_string.py @@ -13,6 +13,7 @@ from ibis.backends.tests.errors import ( ClickHouseDatabaseError, OracleDatabaseError, + PsycoPg2InternalError, PyDruidProgrammingError, PyODBCProgrammingError, ) @@ -33,6 +34,7 @@ "duckdb": "VARCHAR", "impala": "STRING", "postgres": "text", + "risingwave": "text", "flink": "CHAR(6) NOT NULL", }, id="string", @@ -48,14 +50,22 @@ "duckdb": "VARCHAR", "impala": "STRING", "postgres": "text", + "risingwave": "text", "flink": "CHAR(7) NOT NULL", }, id="string-quote1", - marks=pytest.mark.broken( - ["oracle"], - raises=OracleDatabaseError, - reason="ORA-01741: illegal zero length identifier", - ), + marks=[ + pytest.mark.broken( + ["oracle"], + raises=OracleDatabaseError, + reason="ORA-01741: illegal zero length identifier", + ), + pytest.mark.broken( + ["risingwave"], + raises=PsycoPg2InternalError, + reason='sql parser error: Expected end of statement, found: "NG\'" at line:1, column:31 Near "SELECT \'STRI"NG\' AS "\'STRI""', + ), + ], ), param( 'STRI"NG', @@ -68,14 +78,22 @@ "duckdb": "VARCHAR", "impala": "STRING", "postgres": "text", + "risingwave": "text", "flink": "CHAR(7) NOT NULL", }, id="string-quote2", - marks=pytest.mark.broken( - ["oracle"], - raises=OracleDatabaseError, - reason="ORA-25716", - ), + marks=[ + pytest.mark.broken( + ["oracle"], + raises=OracleDatabaseError, + reason="ORA-25716", + ), + pytest.mark.broken( + ["risingwave"], + raises=PsycoPg2InternalError, + reason='sql parser error: Expected end of statement, found: "NG\'" at line:1, column:31 Near "SELECT \'STRI"NG\' AS "\'STRI""', + ), + ], ), ], ) @@ -846,6 +864,7 @@ def test_substr_with_null_values(backend, alltypes, df): "mysql", "polars", "postgres", + "risingwave", "pyspark", "druid", "oracle", @@ -917,6 +936,11 @@ def test_multiple_subs(con): ], raises=com.OperationNotDefinedError, ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function levenshtein(character varying, character varying) does not exist", +) @pytest.mark.parametrize( "right", ["sitting", ibis.literal("sitting")], ids=["python", "ibis"] ) diff --git a/ibis/backends/tests/test_struct.py b/ibis/backends/tests/test_struct.py index d368c6b16305..f2b8c99fc73b 100644 --- a/ibis/backends/tests/test_struct.py +++ b/ibis/backends/tests/test_struct.py @@ -65,7 +65,7 @@ def test_all_fields(struct, struct_df): _NULL_STRUCT_LITERAL = ibis.NA.cast("struct") -@pytest.mark.notimpl(["postgres"]) +@pytest.mark.notimpl(["postgres", "risingwave"]) @pytest.mark.parametrize("field", ["a", "b", "c"]) @pytest.mark.notyet( ["flink"], reason="flink doesn't support creating struct columns from literals" @@ -95,7 +95,7 @@ def test_null_literal(backend, con, field): backend.assert_series_equal(result, expected) -@pytest.mark.notimpl(["dask", "pandas", "postgres"]) +@pytest.mark.notimpl(["dask", "pandas", "postgres", "risingwave"]) @pytest.mark.notyet( ["flink"], reason="flink doesn't support creating struct columns from literals" ) @@ -111,7 +111,7 @@ def test_struct_column(backend, alltypes, df): tm.assert_series_equal(result, expected) -@pytest.mark.notimpl(["dask", "pandas", "postgres", "polars"]) +@pytest.mark.notimpl(["dask", "pandas", "postgres", "risingwave", "polars"]) @pytest.mark.notyet( ["flink"], reason="flink doesn't support creating struct columns from collect" ) diff --git a/ibis/backends/tests/test_temporal.py b/ibis/backends/tests/test_temporal.py index 8b5f7e897815..acf7ddfdcd9d 100644 --- a/ibis/backends/tests/test_temporal.py +++ b/ibis/backends/tests/test_temporal.py @@ -29,6 +29,7 @@ OracleDatabaseError, PolarsComputeError, PolarsPanicException, + PsycoPg2InternalError, Py4JJavaError, PyDruidProgrammingError, PyODBCProgrammingError, @@ -155,6 +156,11 @@ def test_timestamp_extract(backend, alltypes, df, attr): "Ref: https://nightlies.apache.org/flink/flink-docs-release-1.13/docs/dev/table/functions/systemfunctions/#temporal-functions" ), ), + pytest.mark.broken( + ["risingwave"], + raises=AssertionError, + reason="Refer to https://github.com/risingwavelabs/risingwave/issues/14670", + ), ], ), ], @@ -434,6 +440,7 @@ def test_timestamp_extract_week_of_year(backend, alltypes, df): "impala", "mysql", "postgres", + "risingwave", "pyspark", "sqlite", "snowflake", @@ -634,6 +641,11 @@ def test_date_truncate(backend, alltypes, df, unit): raises=Py4JJavaError, reason="ParseException: Encountered 'WEEK'. Was expecting one of: DAY, DAYS, HOUR", ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Bind error: Invalid unit: week", + ), ], ), param("D", pd.offsets.DateOffset), @@ -652,6 +664,11 @@ def test_date_truncate(backend, alltypes, df, unit): raises=Py4JJavaError, reason="ParseException: Encountered 'MILLISECOND'. Was expecting one of: DAY, DAYS, HOUR, ...", ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Bind error: Invalid unit: millisecond", + ), ], ), param( @@ -671,6 +688,11 @@ def test_date_truncate(backend, alltypes, df, unit): raises=Py4JJavaError, reason="ParseException: Encountered 'MICROSECOND'. Was expecting one of: DAY, DAYS, HOUR, ...", ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Bind error: Invalid unit: microsecond", + ), ], ), ], @@ -721,7 +743,14 @@ def convert_to_offset(offset, displacement_type=displacement_type): ), param( "W", - marks=pytest.mark.notyet(["trino"], raises=com.UnsupportedOperationError), + marks=[ + pytest.mark.notyet(["trino"], raises=com.UnsupportedOperationError), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Bind error: Invalid unit: week", + ), + ], ), "D", ], @@ -1511,7 +1540,7 @@ def test_strftime(backend, alltypes, df, expr_fn, pandas_pattern): ], ) @pytest.mark.notimpl( - ["mysql", "postgres", "sqlite", "druid", "oracle"], + ["mysql", "postgres", "risingwave", "sqlite", "druid", "oracle"], raises=com.OperationNotDefinedError, ) @pytest.mark.notimpl(["exasol"], raises=com.OperationNotDefinedError) @@ -1631,6 +1660,11 @@ def test_string_to_timestamp(alltypes, fmt): reason="DayOfWeekName is not supported in Flink", ) @pytest.mark.notimpl(["exasol"], raises=com.OperationNotDefinedError) +@pytest.mark.broken( + ["risingwave"], + raises=AssertionError, + reason="Refer to https://github.com/risingwavelabs/risingwave/issues/14670", +) def test_day_of_week_scalar(con, date, expected_index, expected_day): expr = ibis.literal(date).cast(dt.date) result_index = con.execute(expr.day_of_week.index().name("tmp")) @@ -1656,6 +1690,11 @@ def test_day_of_week_scalar(con, date, expected_index, expected_day): ), ) @pytest.mark.notimpl(["exasol"], raises=com.OperationNotDefinedError) +@pytest.mark.broken( + ["risingwave"], + raises=AssertionError, + reason="Refer to https://github.com/risingwavelabs/risingwave/issues/14670", +) def test_day_of_week_column(backend, alltypes, df): expr = alltypes.timestamp_col.day_of_week @@ -1692,6 +1731,11 @@ def test_day_of_week_column(backend, alltypes, df): "Ref: https://nightlies.apache.org/flink/flink-docs-release-1.13/docs/dev/table/functions/systemfunctions/#temporal-functions" ), ), + pytest.mark.broken( + ["risingwave"], + raises=AssertionError, + reason="Refer to https://github.com/risingwavelabs/risingwave/issues/14670", + ), ], ), ], @@ -1759,6 +1803,7 @@ def test_now_from_projection(alltypes): "snowflake": "DATE", "sqlite": "text", "trino": "date", + "risingwave": "date", } @@ -1769,6 +1814,11 @@ def test_now_from_projection(alltypes): @pytest.mark.notimpl( ["oracle"], raises=OracleDatabaseError, reason="ORA-00936 missing expression" ) +@pytest.mark.notimpl( + ["risingwave"], + raises=com.OperationNotDefinedError, + reason="function make_date(integer, integer, integer) does not exist", +) def test_date_literal(con, backend): expr = ibis.date(2022, 2, 4) result = con.execute(expr) @@ -1788,6 +1838,7 @@ def test_date_literal(con, backend): "trino": "timestamp(3)", "duckdb": "TIMESTAMP", "postgres": "timestamp without time zone", + "risingwave": "timestamp without time zone", "flink": "TIMESTAMP(6) NOT NULL", } @@ -1797,6 +1848,11 @@ def test_date_literal(con, backend): raises=com.OperationNotDefinedError, ) @pytest.mark.notyet(["impala"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function make_timestamp(integer, integer, integer, integer, integer, integer) does not exist", +) def test_timestamp_literal(con, backend): expr = ibis.timestamp(2022, 2, 4, 16, 20, 0) result = con.execute(expr) @@ -1850,6 +1906,11 @@ def test_timestamp_literal(con, backend): ", , , )" ), ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function make_timestamp(integer, integer, integer, integer, integer, integer) does not exist", +) def test_timestamp_with_timezone_literal(con, timezone, expected): expr = ibis.timestamp(2022, 2, 4, 16, 20, 0).cast(dt.Timestamp(timezone=timezone)) result = con.execute(expr) @@ -1866,6 +1927,7 @@ def test_timestamp_with_timezone_literal(con, timezone, expected): "trino": "time(3)", "duckdb": "TIME", "postgres": "time without time zone", + "risingwave": "time without time zone", } @@ -1877,6 +1939,11 @@ def test_timestamp_with_timezone_literal(con, timezone, expected): ["clickhouse", "impala", "exasol"], raises=com.OperationNotDefinedError ) @pytest.mark.notimpl(["druid"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function make_time(integer, integer, integer) does not exist", +) def test_time_literal(con, backend): expr = ibis.time(16, 20, 0) result = con.execute(expr) @@ -1953,6 +2020,7 @@ def test_extract_time_from_timestamp(con, microsecond): "trino": "interval day to second", "duckdb": "INTERVAL", "postgres": "interval", + "risingwave": "interval", } @@ -2021,6 +2089,11 @@ def test_interval_literal(con, backend): @pytest.mark.broken( ["oracle"], raises=OracleDatabaseError, reason="ORA-00936: missing expression" ) +@pytest.mark.notimpl( + ["risingwave"], + raises=com.OperationNotDefinedError, + reason="function make_date(integer, integer, integer) does not exist", +) def test_date_column_from_ymd(backend, con, alltypes, df): c = alltypes.timestamp_col expr = ibis.date(c.year(), c.month(), c.day()) @@ -2041,6 +2114,11 @@ def test_date_column_from_ymd(backend, con, alltypes, df): reason="StringColumn' object has no attribute 'year'", ) @pytest.mark.notyet(["impala", "oracle"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function make_timestamp(smallint, smallint, smallint, smallint, smallint, smallint) does not exist", +) def test_timestamp_column_from_ymdhms(backend, con, alltypes, df): c = alltypes.timestamp_col expr = ibis.timestamp( @@ -2291,6 +2369,11 @@ def test_large_timestamp(con): raises=AssertionError, ), pytest.mark.notimpl(["exasol"], raises=AssertionError), + pytest.mark.notyet( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Parse error: timestamp without time zone Can't cast string to timestamp (expected format is YYYY-MM-DD HH:MM:SS[.D+{up to 6 digits}] or YYYY-MM-DD HH:MM or YYYY-MM-DD or ISO 8601 format)", + ), ], ), ], @@ -2337,7 +2420,7 @@ def test_timestamp_precision_output(con, ts, scale, unit): reason="time types not yet implemented in ibis for the clickhouse backend", ), pytest.mark.notyet( - ["postgres"], + ["postgres", "risingwave"], reason="postgres doesn't have any easy way to accurately compute the delta in specific units", raises=com.OperationNotDefinedError, ), @@ -2467,6 +2550,11 @@ def test_delta(con, start, end, unit, expected): ], ) @pytest.mark.notimpl(["exasol"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function date_bin(interval, timestamp without time zone, timestamp without time zone) does not exist", +) def test_timestamp_bucket(backend, kws, pd_freq): ts = backend.functional_alltypes.timestamp_col.execute().rename("ts") res = backend.functional_alltypes.timestamp_col.bucket(**kws).execute().rename("ts") @@ -2502,6 +2590,11 @@ def test_timestamp_bucket(backend, kws, pd_freq): ) @pytest.mark.parametrize("offset_mins", [2, -2], ids=["pos", "neg"]) @pytest.mark.notimpl(["exasol"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="function date_bin(interval, timestamp without time zone, timestamp without time zone) does not exist", +) def test_timestamp_bucket_offset(backend, offset_mins): ts = backend.functional_alltypes.timestamp_col expr = ts.bucket(minutes=5, offset=ibis.interval(minutes=offset_mins)) diff --git a/ibis/backends/tests/test_timecontext.py b/ibis/backends/tests/test_timecontext.py index 2488ee5b3659..88376a4f961b 100644 --- a/ibis/backends/tests/test_timecontext.py +++ b/ibis/backends/tests/test_timecontext.py @@ -19,6 +19,7 @@ "impala", "mysql", "postgres", + "risingwave", "sqlite", "snowflake", "polars", diff --git a/ibis/backends/tests/test_udf.py b/ibis/backends/tests/test_udf.py index f5a3a5d01e61..75021f577133 100644 --- a/ibis/backends/tests/test_udf.py +++ b/ibis/backends/tests/test_udf.py @@ -19,6 +19,7 @@ "oracle", "pandas", "trino", + "risingwave", ] ) diff --git a/ibis/backends/tests/test_uuid.py b/ibis/backends/tests/test_uuid.py index a01a1c124ad7..205a2b972239 100644 --- a/ibis/backends/tests/test_uuid.py +++ b/ibis/backends/tests/test_uuid.py @@ -20,6 +20,7 @@ "flink": "CHAR(36) NOT NULL", "impala": "STRING", "postgres": "uuid", + "risingwave": "character varying", "snowflake": "VARCHAR", "sqlite": "text", "trino": "uuid", diff --git a/ibis/backends/tests/test_vectorized_udf.py b/ibis/backends/tests/test_vectorized_udf.py index c2414db0b58d..fa6728acb7f2 100644 --- a/ibis/backends/tests/test_vectorized_udf.py +++ b/ibis/backends/tests/test_vectorized_udf.py @@ -10,7 +10,7 @@ import ibis.expr.datatypes as dt from ibis.legacy.udf.vectorized import analytic, elementwise, reduction -pytestmark = pytest.mark.notimpl(["druid", "oracle"]) +pytestmark = pytest.mark.notimpl(["druid", "oracle", "risingwave"]) def _format_udf_return_type(func, result_formatter): diff --git a/ibis/backends/tests/test_window.py b/ibis/backends/tests/test_window.py index 1e3627afab8f..d492315447d9 100644 --- a/ibis/backends/tests/test_window.py +++ b/ibis/backends/tests/test_window.py @@ -18,6 +18,7 @@ ImpalaHiveServer2Error, MySQLOperationalError, OracleDatabaseError, + PsycoPg2InternalError, Py4JJavaError, PyDruidProgrammingError, PyODBCProgrammingError, @@ -137,6 +138,11 @@ def calc_zscore(s): raises=com.OperationNotDefinedError, ), pytest.mark.notimpl(["dask"], raises=NotImplementedError), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Unrecognized window function: percent_rank", + ), ], ), param( @@ -148,6 +154,12 @@ def calc_zscore(s): ["clickhouse", "exasol"], raises=com.OperationNotDefinedError ), pytest.mark.notimpl(["dask"], raises=NotImplementedError), + pytest.mark.notimpl(["dask"], raises=NotImplementedError), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Unrecognized window function: cume_dist", + ), ], ), param( @@ -174,6 +186,11 @@ def calc_zscore(s): raises=com.UnsupportedOperationError, reason="Windows in Flink can only be ordered by a single time column", ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Unrecognized window function: ntile", + ), ], ), param( @@ -212,6 +229,7 @@ def calc_zscore(s): ), pytest.mark.notimpl(["dask"], raises=NotImplementedError), pytest.mark.notimpl(["flink"], raises=com.OperationNotDefinedError), + pytest.mark.notimpl(["risingwave"], raises=PsycoPg2InternalError), ], ), param( @@ -373,7 +391,14 @@ def test_grouped_bounded_expanding_window( lambda t, win: t.double_col.mean().over(win), lambda df: (df.double_col.expanding().mean()), id="mean", - marks=[pytest.mark.notimpl(["dask"], raises=NotImplementedError)], + marks=[ + pytest.mark.notimpl(["dask"], raises=NotImplementedError), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", + ), + ], ), param( # Disabled on PySpark and Spark backends because in pyspark<3.0.0, @@ -393,6 +418,7 @@ def test_grouped_bounded_expanding_window( "mysql", "oracle", "postgres", + "risingwave", "sqlite", "snowflake", "datafusion", @@ -548,6 +574,7 @@ def test_grouped_bounded_preceding_window(backend, alltypes, df, window_fn): "mysql", "oracle", "postgres", + "risingwave", "sqlite", "snowflake", "trino", @@ -625,6 +652,11 @@ def test_grouped_unbounded_window( @pytest.mark.notyet(["mssql"], raises=PyODBCProgrammingError) @pytest.mark.notimpl(["polars"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl(["flink"], raises=com.UnsupportedOperationError) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", +) def test_simple_ungrouped_unbound_following_window( backend, alltypes, ibis_method, pandas_fn ): @@ -652,6 +684,11 @@ def test_simple_ungrouped_unbound_following_window( ["mssql"], raises=Exception, reason="order by constant is not supported" ) @pytest.mark.notimpl(["polars"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", +) def test_simple_ungrouped_window_with_scalar_order_by(alltypes): t = alltypes[alltypes.double_col < 50].order_by("id") w = ibis.window(rows=(0, None), order_by=ibis.NA) @@ -680,6 +717,11 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): reason="default window semantics are different", raises=AssertionError, ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", + ), ], ), param( @@ -713,6 +755,11 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): raises=Py4JJavaError, reason="CalciteContextException: Argument to function 'NTILE' must be a literal", ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Unrecognized window function: ntile", + ), ], ), param( @@ -732,6 +779,7 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): "mysql", "oracle", "postgres", + "risingwave", "sqlite", "snowflake", "trino", @@ -763,6 +811,7 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): "mysql", "oracle", "postgres", + "risingwave", "sqlite", "snowflake", "trino", @@ -783,6 +832,13 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): lambda df: df.float_col.shift(1), True, id="ordered-lag", + marks=[ + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", + ), + ], ), param( lambda t, win: t.float_col.lag().over(win), @@ -812,6 +868,11 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): reason="backend requires ordering", raises=SnowflakeProgrammingError, ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", + ), ], ), param( @@ -819,6 +880,13 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): lambda df: df.float_col.shift(-1), True, id="ordered-lead", + marks=[ + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", + ), + ], ), param( lambda t, win: t.float_col.lead().over(win), @@ -851,6 +919,11 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): reason="backend requires ordering", raises=SnowflakeProgrammingError, ), + pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", + ), ], ), param( @@ -870,6 +943,7 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): "mysql", "oracle", "postgres", + "risingwave", "pyspark", "sqlite", "snowflake", @@ -902,6 +976,7 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): "mysql", "oracle", "postgres", + "risingwave", "pyspark", "sqlite", "snowflake", @@ -968,6 +1043,11 @@ def test_ungrouped_unbounded_window( raises=MySQLOperationalError, reason="https://github.com/tobymao/sqlglot/issues/2779", ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: window frame in `RANGE` mode is not supported yet", +) def test_grouped_bounded_range_window(backend, alltypes, df): # Explanation of the range window spec below: # @@ -1023,6 +1103,11 @@ def gb_fn(df): reason="clickhouse doesn't implement percent_rank", raises=com.OperationNotDefinedError, ) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Unrecognized window function: percent_rank", +) def test_percent_rank_whole_table_no_order_by(backend, alltypes, df): expr = alltypes.mutate(val=lambda t: t.id.percent_rank()) @@ -1069,6 +1154,11 @@ def agg(df): @pytest.mark.notimpl(["polars"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", +) def test_mutate_window_filter(backend, alltypes): t = alltypes win = ibis.window(order_by=[t.id]) @@ -1143,6 +1233,11 @@ def test_first_last(backend): raises=ExaQueryError, reason="database can't handle UTC timestamps in DataFrames", ) +@pytest.mark.broken( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="sql parser error: Expected literal int, found: INTERVAL at line:1, column:99", +) def test_range_expression_bounds(backend): t = ibis.memtable( { @@ -1187,6 +1282,11 @@ def test_range_expression_bounds(backend): @pytest.mark.broken( ["mssql"], reason="lack of support for booleans", raises=PyODBCProgrammingError ) +@pytest.mark.broken( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Unrecognized window function: percent_rank", +) def test_rank_followed_by_over_call_merge_frames(backend, alltypes, df): # GH #7631 t = alltypes @@ -1217,6 +1317,11 @@ 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.notimpl( + ["risingwave"], + raises=PsycoPg2InternalError, + reason="Feature is not yet implemented: Window function with empty PARTITION BY is not supported yet", +) def test_windowed_order_by_sequence_is_preserved(con): table = ibis.memtable({"bool_col": [True, False, False, None, True]}) window = ibis.window( diff --git a/ibis/backends/trino/tests/conftest.py b/ibis/backends/trino/tests/conftest.py index 5abc5eaa859a..764f39b456c8 100644 --- a/ibis/backends/trino/tests/conftest.py +++ b/ibis/backends/trino/tests/conftest.py @@ -38,6 +38,7 @@ class TestConf(ServiceBackendTest): # for numeric and decimal service_name = "minio" + data_volume = "/bitnami/minio/data" returned_timestamp_unit = "s" supports_structs = True supports_map = True @@ -45,21 +46,6 @@ class TestConf(ServiceBackendTest): deps = ("trino",) def preload(self): - # create buckets - subprocess.run( - [ - "docker", - "compose", - "exec", - "minio", - "mc", - "mb", - "--ignore-existing", - "trino/warehouse", - ], - check=True, - ) - # copy files to the minio host super().preload() @@ -76,7 +62,7 @@ def preload(self): "mc", "cp", f"{self.data_volume}/{path.name}", - f"trino/warehouse/{dirname}/{path.name}", + f"data/trino/{dirname}/{path.name}", ], check=True, ) diff --git a/ibis/expr/operations/generic.py b/ibis/expr/operations/generic.py index 5ffd4585c1d5..15cbd5a5d345 100644 --- a/ibis/expr/operations/generic.py +++ b/ibis/expr/operations/generic.py @@ -232,6 +232,20 @@ class HashBytes(Value): shape = rlz.shape_like("arg") +@public +class HexDigest(Value): + arg: Value[dt.String | dt.Binary] + how: LiteralType[ + "md5", + "sha1", + "sha256", + "sha512", + ] + + dtype = dt.str + shape = rlz.shape_like("arg") + + # TODO(kszucs): we should merge the case operations by making the # cases, results and default optional arguments like they are in # api.py diff --git a/ibis/expr/types/strings.py b/ibis/expr/types/strings.py index e4532e0030f4..32835d2bb23f 100644 --- a/ibis/expr/types/strings.py +++ b/ibis/expr/types/strings.py @@ -465,6 +465,40 @@ def hashbytes( """ return ops.HashBytes(self, how).to_expr() + def hexdigest( + self, + how: Literal["md5", "sha1", "sha256", "sha512"] = "sha256", + ) -> ir.StringValue: + """Return the hash digest of the input as a hex encoded string. + + Parameters + ---------- + how + Hash algorithm to use + + Returns + ------- + StringValue + Hexadecimal representation of the hash as a string + + Examples + -------- + >>> import ibis + >>> ibis.options.interactive = True + >>> t = ibis.memtable({"species": ["Adelie", "Chinstrap", "Gentoo"]}) + >>> t.species.hexdigest() + ┏━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┓ + ┃ HexDigest(species) ┃ + ┡━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━━┩ + │ string │ + ├──────────────────────────────────────────────────────────────────┤ + │ a4d7d46b27480037bc1e513e0e157cbf258baae6ee69e3110d0f9ff418b57a3c │ + │ cb97d113ca69899ae4f1fb581f4a90d86989db77b4a33873d604b0ee412b4cc9 │ + │ b5e90cdff65949fe6bc226823245f7698110e563a12363fc57b3eed3e4a0a612 │ + └──────────────────────────────────────────────────────────────────┘ + """ + return ops.HexDigest(self, how.lower()).to_expr() + def substr( self, start: int | ir.IntegerValue, diff --git a/poetry.lock b/poetry.lock index 2c7ba36701c8..91b8b4f683e6 100644 --- a/poetry.lock +++ b/poetry.lock @@ -7331,6 +7331,7 @@ pandas = ["regex"] polars = ["packaging", "polars"] postgres = ["psycopg2"] pyspark = ["packaging", "pyspark"] +risingwave = ["psycopg2"] snowflake = ["packaging", "snowflake-connector-python"] sqlite = ["regex"] trino = ["trino"] @@ -7339,4 +7340,4 @@ visualization = ["graphviz"] [metadata] lock-version = "2.0" python-versions = "^3.9" -content-hash = "46f6575d9e668129872ccb5c2fd5de6c3e2fc808b8620e1c0082b18239b36639" +content-hash = "09804aaeee25b130ab30ec88414c0c7cd3e5914ca320bb417346665dfc489f20" diff --git a/pyproject.toml b/pyproject.toml index 6b5ef6dffa75..14cd3b59dabe 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -193,6 +193,7 @@ postgres = ["psycopg2"] pyspark = ["pyspark", "packaging"] snowflake = ["snowflake-connector-python", "packaging"] sqlite = ["regex"] +risingwave = ["psycopg2"] trino = ["trino"] # non-backend extras visualization = ["graphviz"] @@ -216,6 +217,7 @@ oracle = "ibis.backends.oracle" pandas = "ibis.backends.pandas" polars = "ibis.backends.polars" postgres = "ibis.backends.postgres" +risingwave = "ibis.backends.risingwave" pyspark = "ibis.backends.pyspark" snowflake = "ibis.backends.snowflake" sqlite = "ibis.backends.sqlite" @@ -352,6 +354,7 @@ markers = [ "pandas: Pandas tests", "polars: Polars tests", "postgres: PostgreSQL tests", + "risingwave: Risingwave tests", "pyspark: PySpark tests", "snowflake: Snowflake tests", "sqlite: SQLite tests",