From 518779214c252bf5b8d962933ceb58ad136206cb Mon Sep 17 00:00:00 2001 From: Phillip Cloud <417981+cpcloud@users.noreply.github.com> Date: Thu, 25 Jan 2024 09:22:23 -0500 Subject: [PATCH] refactor(flink): port to sqlglot --- .github/workflows/ibis-backends.yml | 90 +- ibis/backends/base/sqlglot/compiler.py | 2 +- ibis/backends/base/sqlglot/datatypes.py | 10 + ibis/backends/flink/__init__.py | 105 +-- ibis/backends/flink/compiler.py | 563 ++++++++++++ ibis/backends/flink/compiler/__init__.py | 0 ibis/backends/flink/compiler/core.py | 170 ---- ibis/backends/flink/ddl.py | 12 +- ibis/backends/flink/registry.py | 398 --------- ibis/backends/flink/tests/conftest.py | 47 +- .../test_complex_filtered_agg/out.sql | 16 +- .../test_complex_groupby_aggregation/out.sql | 14 +- .../test_complex_projections/out.sql | 20 +- .../test_compiler/test_count_star/out.sql | 9 +- .../test_extract_fields/day/out.sql | 5 +- .../test_extract_fields/day_of_year/out.sql | 5 +- .../test_extract_fields/hour/out.sql | 5 +- .../test_extract_fields/minute/out.sql | 5 +- .../test_extract_fields/month/out.sql | 5 +- .../test_extract_fields/quarter/out.sql | 5 +- .../test_extract_fields/second/out.sql | 5 +- .../test_extract_fields/week_of_year/out.sql | 5 +- .../test_extract_fields/year/out.sql | 5 +- .../test_compiler/test_filter/out.sql | 25 +- .../test_compiler/test_having/out.sql | 18 +- .../test_simple_filtered_agg/out.sql | 6 +- .../snapshots/test_compiler/test_sum/out.sql | 5 +- .../timestamp_ms/out.sql | 5 +- .../timestamp_s/out.sql | 5 +- .../test_compiler/test_value_counts/out.sql | 14 +- .../test_window_aggregation/out.sql | 17 +- .../test_compiler/test_window_topn/out.sql | 47 +- .../cumulate_window/out.sql | 7 +- .../test_windowing_tvf/hop_window/out.sql | 7 +- .../test_windowing_tvf/tumble_window/out.sql | 5 +- .../datetime/out.sql | 3 +- .../datetime_with_microseconds/out.sql | 3 +- .../string_time/out.sql | 3 +- .../string_timestamp/out.sql | 3 +- .../time/out.sql | 3 +- .../timestamp/out.sql | 3 +- .../test_window/test_range_window/out.sql | 5 +- .../test_window/test_rows_window/out.sql | 5 +- ibis/backends/flink/tests/test_ddl.py | 19 +- ibis/backends/flink/tests/test_literals.py | 80 -- ibis/backends/flink/tests/test_window.py | 30 +- ibis/backends/flink/translator.py | 16 - .../test_many_subqueries/flink/out.sql | 42 + .../test_default_limit/flink/out.sql | 5 + .../test_disable_query_limit/flink/out.sql | 5 + .../flink/out.sql | 3 + .../test_respect_set_limit/flink/out.sql | 10 + .../test_cte_refs_in_topo_order/flink/out.sql | 20 + .../test_group_by_has_index/flink/out.sql | 38 + .../test_sql/test_isin_bug/flink/out.sql | 9 + ibis/backends/tests/test_aggregation.py | 43 +- ibis/backends/tests/test_array.py | 29 +- ibis/backends/tests/test_asof_join.py | 2 + ibis/backends/tests/test_client.py | 43 +- ibis/backends/tests/test_dot_sql.py | 21 +- ibis/backends/tests/test_examples.py | 2 +- ibis/backends/tests/test_export.py | 40 +- ibis/backends/tests/test_generic.py | 38 +- ibis/backends/tests/test_join.py | 1 - ibis/backends/tests/test_map.py | 42 +- ibis/backends/tests/test_network.py | 4 +- ibis/backends/tests/test_numeric.py | 56 +- ibis/backends/tests/test_param.py | 15 +- ibis/backends/tests/test_sql.py | 2 - ibis/backends/tests/test_string.py | 40 +- ibis/backends/tests/test_temporal.py | 161 +--- ibis/backends/tests/test_timecontext.py | 2 +- ibis/backends/tests/test_window.py | 46 +- poetry-overrides.nix | 19 + poetry.lock | 806 +++++++++++++----- pyproject.toml | 10 +- requirements-dev.txt | 33 +- 77 files changed, 1848 insertions(+), 1574 deletions(-) create mode 100644 ibis/backends/flink/compiler.py delete mode 100644 ibis/backends/flink/compiler/__init__.py delete mode 100644 ibis/backends/flink/compiler/core.py delete mode 100644 ibis/backends/flink/tests/test_literals.py delete mode 100644 ibis/backends/flink/translator.py create mode 100644 ibis/backends/tests/snapshots/test_generic/test_many_subqueries/flink/out.sql create mode 100644 ibis/backends/tests/snapshots/test_interactive/test_default_limit/flink/out.sql create mode 100644 ibis/backends/tests/snapshots/test_interactive/test_disable_query_limit/flink/out.sql create mode 100644 ibis/backends/tests/snapshots/test_interactive/test_interactive_execute_on_repr/flink/out.sql create mode 100644 ibis/backends/tests/snapshots/test_interactive/test_respect_set_limit/flink/out.sql create mode 100644 ibis/backends/tests/snapshots/test_sql/test_cte_refs_in_topo_order/flink/out.sql create mode 100644 ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/flink/out.sql create mode 100644 ibis/backends/tests/snapshots/test_sql/test_isin_bug/flink/out.sql diff --git a/.github/workflows/ibis-backends.yml b/.github/workflows/ibis-backends.yml index 46cfba02f80fc..db9dd3496304a 100644 --- a/.github/workflows/ibis-backends.yml +++ b/.github/workflows/ibis-backends.yml @@ -177,22 +177,16 @@ jobs: - oracle services: - oracle - # - name: flink - # title: Flink - # serial: true - # extras: - # - flink - # additional_deps: - # - apache-flink - # - pytest-split - # services: - # - flink - # - name: risingwave - # title: Risingwave - # services: - # - risingwave - # extras: - # - risingwave + - name: flink + title: Flink + serial: true + extras: + - flink + additional_deps: + - apache-flink + - pytest-split + services: + - flink exclude: - os: windows-latest backend: @@ -296,32 +290,15 @@ jobs: - oracle services: - oracle - # - os: windows-latest - # backend: - # name: flink - # title: Flink - # serial: true - # extras: - # - flink - # services: - # - flink - # - python-version: "3.11" - # backend: - # name: flink - # title: Flink - # serial: true - # extras: - # - flink - # services: - # - flink - # - os: windows-latest - # backend: - # name: risingwave - # title: Risingwave - # services: - # - risingwave - # extras: - # - risingwave + - os: windows-latest + backend: + name: flink + title: Flink + serial: true + extras: + - flink + services: + - flink - os: windows-latest backend: name: exasol @@ -390,29 +367,18 @@ jobs: IBIS_TEST_IMPALA_PORT: 21050 IBIS_EXAMPLES_DATA: ${{ runner.temp }}/examples-${{ matrix.backend.name }}-${{ matrix.os }}-${{ steps.install_python.outputs.python-version }} - # FIXME(deepyaman): If some backend-specific test, in test_ddl.py, - # executes before common tests, they will fail with: - # org.apache.flink.table.api.ValidationException: Table `default_catalog`.`default_database`.`functional_alltypes` was not found. - # Therefore, we run backend-specific tests second to avoid this. - # - name: "run serial tests: ${{ matrix.backend.name }}" - # if: matrix.backend.serial && matrix.backend.name == 'flink' - # run: | - # just ci-check -m ${{ matrix.backend.name }} ibis/backends/tests - # just ci-check -m ${{ matrix.backend.name }} ibis/backends/flink/tests - # env: - # IBIS_EXAMPLES_DATA: ${{ runner.temp }}/examples-${{ matrix.backend.name }}-${{ matrix.os }}-${{ steps.install_python.outputs.python-version }} - # FLINK_REMOTE_CLUSTER_ADDR: localhost - # FLINK_REMOTE_CLUSTER_PORT: "8081" - # - name: "run serial tests: ${{ matrix.backend.name }}" - if: matrix.backend.serial # && matrix.backend.name != 'flink' + if: matrix.backend.serial run: just ci-check -m ${{ matrix.backend.name }} env: + FLINK_REMOTE_CLUSTER_ADDR: localhost + FLINK_REMOTE_CLUSTER_PORT: "8081" IBIS_EXAMPLES_DATA: ${{ runner.temp }}/examples-${{ matrix.backend.name }}-${{ matrix.os }}-${{ steps.install_python.outputs.python-version }} - name: check that no untracked files were produced shell: bash - run: git checkout poetry.lock pyproject.toml && ! git status --porcelain | tee /dev/stderr | grep . + run: | + ! git status --porcelain | tee /dev/stderr | grep . - name: upload code coverage if: success() @@ -513,10 +479,6 @@ jobs: - name: install poetry run: python -m pip install --upgrade pip 'poetry==1.7.1' - - name: remove lonboard - # it requires a version of pandas that min versions are not compatible with - run: poetry remove lonboard - - name: install minimum versions run: poetry add --lock --optional ${{ join(matrix.backend.deps, ' ') }} @@ -596,10 +558,6 @@ jobs: - name: install poetry run: python -m pip install --upgrade pip 'poetry==1.7.1' - - name: remove lonboard - # it requires a version of pandas that pyspark is not compatible with - run: poetry remove lonboard - - name: install exact versions of pyspark, pandas and numpy run: poetry add --lock ${{ join(matrix.deps, ' ') }} diff --git a/ibis/backends/base/sqlglot/compiler.py b/ibis/backends/base/sqlglot/compiler.py index d6c2f90d1e5c5..c6760b06cde12 100644 --- a/ibis/backends/base/sqlglot/compiler.py +++ b/ibis/backends/base/sqlglot/compiler.py @@ -129,7 +129,7 @@ def __getitem__(self, key: str) -> sge.Column: def paren(expr): """Wrap a sqlglot expression in parentheses.""" - return sge.Paren(this=expr) + return sge.Paren(this=sge.convert(expr)) def parenthesize(op, arg): diff --git a/ibis/backends/base/sqlglot/datatypes.py b/ibis/backends/base/sqlglot/datatypes.py index dd9827d2a5498..9cd03adbd6222 100644 --- a/ibis/backends/base/sqlglot/datatypes.py +++ b/ibis/backends/base/sqlglot/datatypes.py @@ -1029,3 +1029,13 @@ def _from_ibis_Map(cls, dtype: dt.Map) -> sge.DataType: key_type = cls.from_ibis(dtype.key_type.copy(nullable=False)) value_type = cls.from_ibis(dtype.value_type) return sge.DataType(this=typecode.MAP, expressions=[key_type, value_type]) + + +class FlinkType(SqlglotType): + dialect = "flink" + default_decimal_precision = 38 + default_decimal_scale = 18 + + @classmethod + def _from_ibis_Binary(cls, dtype: dt.Binary) -> sge.DataType: + return sge.DataType(this=sge.DataType.Type.VARBINARY) diff --git a/ibis/backends/flink/__init__.py b/ibis/backends/flink/__init__.py index 679c7680de62c..d50cb5311dcec 100644 --- a/ibis/backends/flink/__init__.py +++ b/ibis/backends/flink/__init__.py @@ -1,23 +1,22 @@ from __future__ import annotations import itertools -from functools import lru_cache from typing import TYPE_CHECKING, Any import pyarrow as pa import sqlglot as sg +import sqlglot.expressions as sge import ibis.common.exceptions as exc import ibis.expr.operations as ops import ibis.expr.schema as sch import ibis.expr.types as ir -from ibis.backends.base import BaseBackend, CanCreateDatabase, NoUrl -from ibis.backends.base.sql.ddl import fully_qualified_re, is_fully_qualified -from ibis.backends.flink.compiler.core import FlinkCompiler +from ibis.backends.base import CanCreateDatabase, NoUrl +from ibis.backends.base.sqlglot import SQLGlotBackend +from ibis.backends.flink.compiler import FlinkCompiler from ibis.backends.flink.ddl import ( CreateDatabase, CreateTableFromConnector, - CreateView, DropDatabase, DropTable, DropView, @@ -38,9 +37,9 @@ from ibis.api import Watermark -class Backend(BaseBackend, CanCreateDatabase, NoUrl): +class Backend(SQLGlotBackend, CanCreateDatabase, NoUrl): name = "flink" - compiler = FlinkCompiler + compiler = FlinkCompiler() supports_temporary_tables = True supports_python_udfs = True @@ -71,6 +70,17 @@ def do_connect(self, table_env: TableEnvironment) -> None: def raw_sql(self, query: str) -> TableResult: return self._table_env.execute_sql(query) + def _metadata(self, query: str): + from pyflink.table.types import create_arrow_schema + + table = self._table_env.sql_query(query) + schema = table.get_schema() + pa_schema = create_arrow_schema( + schema.get_field_names(), schema.get_field_data_types() + ) + # sort of wasteful, but less code to write + return sch.Schema.from_pyarrow(pa_schema).items() + def list_databases(self, like: str | None = None) -> list[str]: databases = self._table_env.list_databases() return self._filter_with_like(databases, like) @@ -207,21 +217,6 @@ def list_views( return self._filter_with_like(views, like) - def _fully_qualified_name( - self, - name: str, - database: str | None = None, - catalog: str | None = None, - ) -> str: - if name and is_fully_qualified(name): - return name - - return sg.table( - name, - db=database or self.current_database, - catalog=catalog or self.current_catalog, - ).sql(dialect="hive") - def table( self, name: str, @@ -250,15 +245,12 @@ def table( f"`database` must be a string; got {type(database)}" ) schema = self.get_schema(name, catalog=catalog, database=database) - qualified_name = self._fully_qualified_name(name, catalog, database) - _, quoted, unquoted = fully_qualified_re.search(qualified_name).groups() - unqualified_name = quoted or unquoted node = ops.DatabaseTable( - unqualified_name, - schema, - self, - namespace=ops.Namespace(schema=database, database=catalog), - ) # TODO(chloeh13q): look into namespacing with catalog + db + name, + schema=schema, + source=self, + namespace=ops.Namespace(schema=catalog, database=database), + ) return node.to_expr() def get_schema( @@ -288,7 +280,9 @@ def get_schema( from ibis.backends.flink.datatypes import get_field_data_types - qualified_name = self._fully_qualified_name(table_name, catalog, database) + qualified_name = sg.table(table_name, db=catalog, catalog=database).sql( + self.name + ) table = self._table_env.from_path(qualified_name) pyflink_schema = table.get_schema() @@ -305,12 +299,9 @@ def version(self) -> str: return pyflink.version.__version__ def compile( - self, - expr: ir.Expr, - params: Mapping[ir.Expr, Any] | None = None, - **kwargs: Any, + self, expr: ir.Expr, params: Mapping[ir.Expr, Any] | None = None, **_: Any ) -> Any: - """Compile an expression.""" + """Compile an Ibis expression to Flink.""" return super().compile(expr, params=params) # Discard `limit` and other kwargs. def _to_sql(self, expr: ir.Expr, **kwargs: Any) -> str: @@ -604,7 +595,9 @@ def create_view( ) if isinstance(obj, pd.DataFrame): - qualified_name = self._fully_qualified_name(name, database, catalog) + qualified_name = sg.table( + name, db=database, catalog=catalog, quoted=self.compiler.quoted + ).sql(self.name) if schema: table = self._table_env.from_pandas( obj, FlinkRowSchema.from_ibis(schema) @@ -617,15 +610,18 @@ def create_view( elif isinstance(obj, ir.Table): query_expression = self.compile(obj) - statement = CreateView( - name=name, - query_expression=query_expression, - database=database, - can_exist=force, - temporary=temp, + stmt = sge.Create( + kind="VIEW", + this=sg.table( + name, db=database, catalog=catalog, quoted=self.compiler.quoted + ), + expression=query_expression, + exists=force, + properties=sge.Properties(expressions=[sge.TemporaryProperty()]) + if temp + else None, ) - sql = statement.compile() - self.raw_sql(sql) + self.raw_sql(stmt.sql(self.name)) else: raise exc.IbisError(f"Unsupported `obj` type: {type(obj)}") @@ -803,16 +799,6 @@ def read_json( file_type="json", path=path, schema=schema, table_name=table_name ) - @classmethod - @lru_cache - def _get_operations(cls): - translator = cls.compiler.translator_class - return translator._registry.keys() - - @classmethod - def has_operation(cls, operation: type[ops.Value]) -> bool: - return operation in cls._get_operations() - def insert( self, table_name: str, @@ -852,12 +838,9 @@ def insert( import pyarrow_hotfix # noqa: F401 if isinstance(obj, ir.Table): - expr = obj - ast = self.compiler.to_ast(expr) - select = ast.queries[0] statement = InsertSelect( table_name, - select, + self.compile(obj), database=database, catalog=catalog, overwrite=overwrite, @@ -946,7 +929,9 @@ def _from_ibis_table_to_pyflink_table(self, table: ir.Table) -> Table | None: # `table` is not a registered table in Flink. return None - qualified_name = self._fully_qualified_name(table_name) + qualified_name = sg.table(table_name, quoted=self.compiler.quoted).sql( + self.name + ) try: return self._table_env.from_path(qualified_name) except Py4JJavaError: diff --git a/ibis/backends/flink/compiler.py b/ibis/backends/flink/compiler.py new file mode 100644 index 0000000000000..880e0062ce8cd --- /dev/null +++ b/ibis/backends/flink/compiler.py @@ -0,0 +1,563 @@ +"""Flink Ibis expression to SQL compiler.""" + +from __future__ import annotations + +from functools import singledispatchmethod + +import sqlglot as sg +import sqlglot.expressions as sge +from sqlglot.dialects import Hive +from sqlglot.dialects.dialect import rename_func + +import ibis.common.exceptions as com +import ibis.expr.datatypes as dt +import ibis.expr.operations as ops +from ibis.backends.base.sqlglot.compiler import STAR, SQLGlotCompiler, paren +from ibis.backends.base.sqlglot.datatypes import FlinkType +from ibis.backends.base.sqlglot.rewrites import ( + Window, + exclude_unsupported_window_frame_from_ops, + exclude_unsupported_window_frame_from_rank, + exclude_unsupported_window_frame_from_row_number, + rewrite_first_to_first_value, + rewrite_last_to_last_value, + rewrite_sample_as_filter, +) + + +class Flink(Hive): + class Generator(Hive.Generator): + pass + + +Flink.Generator.TRANSFORMS |= { + sge.Stddev: rename_func("stddev_samp"), + sge.StddevPop: rename_func("stddev_pop"), + sge.StddevSamp: rename_func("stddev_samp"), + sge.Variance: rename_func("var_samp"), + sge.VariancePop: rename_func("var_pop"), + sge.Array: ( + lambda self, + e: f"ARRAY[{', '.join(arg.sql(self.dialect) for arg in e.expressions)}]" + ), + sge.ArrayConcat: rename_func("array_concat"), + sge.Length: rename_func("char_length"), + sge.TryCast: lambda self, + e: f"TRY_CAST({e.this.sql(self.dialect)} AS {e.to.sql(self.dialect)})", + sge.DayOfYear: rename_func("dayofyear"), + sge.DayOfWeek: rename_func("dayofweek"), + sge.DayOfMonth: rename_func("dayofmonth"), +} + + +class FlinkCompiler(SQLGlotCompiler): + quoted = True + dialect = "flink" + type_mapper = FlinkType + rewrites = ( + rewrite_sample_as_filter, + exclude_unsupported_window_frame_from_row_number, + exclude_unsupported_window_frame_from_ops, + exclude_unsupported_window_frame_from_rank, + rewrite_first_to_first_value, + rewrite_last_to_last_value, + *SQLGlotCompiler.rewrites, + ) + + @property + def NAN(self): + raise NotImplementedError("Flink does not support NaN") + + @property + def POS_INF(self): + raise NotImplementedError("Flink does not support Infinity") + + NEG_INF = POS_INF + + def _aggregate(self, funcname: str, *args, where): + expr = self.f[funcname](*args) + if where is not None: + return sge.Filter(this=expr, expression=sge.Where(this=where)) + return expr + + @singledispatchmethod + def visit_node(self, op, **kwargs): + return super().visit_node(op, **kwargs) + + @staticmethod + def _minimize_spec(start, end, spec): + if ( + start is None + and isinstance(getattr(end, "value", None), ops.Literal) + and end.value.value == 0 + and end.following + ): + return None + return spec + + @visit_node.register(Window) + def visit_Window(self, op, *, how, func, start, end, group_by, order_by): + if not order_by or len(order_by) > 1: + raise com.UnsupportedOperationError( + "Windows must be ordered by exactly one time column when using the Flink backend" + ) + return super().visit_Window( + op, + how=how, + func=func, + start=start, + end=end, + group_by=group_by, + order_by=order_by, + ) + + @visit_node.register(ops.TumbleWindowingTVF) + def visit_TumbleWindowingTVF(self, op, *, table, time_col, window_size, offset): + args = [ + self.v[f"TABLE {table.this.sql(self.dialect)}"], + self.f.descriptor(time_col), + window_size, + offset, + ] + + return sg.select( + sge.Column( + this=STAR, table=sg.to_identifier(table.alias_or_name, quoted=True) + ) + ).from_( + self.f.table(self.f.tumble(*filter(None, args))).as_( + table.alias_or_name, quoted=True + ) + ) + + @visit_node.register(ops.HopWindowingTVF) + def visit_HopWindowingTVF( + self, op, *, table, time_col, window_size, window_slide, offset + ): + args = [ + self.v[f"TABLE {table.this.sql(self.dialect)}"], + self.f.descriptor(time_col), + window_slide, + window_size, + offset, + ] + return sg.select( + sge.Column( + this=STAR, table=sg.to_identifier(table.alias_or_name, quoted=True) + ) + ).from_( + self.f.table(self.f.hop(*filter(None, args))).as_( + table.alias_or_name, quoted=True + ) + ) + + @visit_node.register(ops.CumulateWindowingTVF) + def visit_CumulateWindowingTVF( + self, op, *, table, time_col, window_size, window_step, offset + ): + args = [ + self.v[f"TABLE {table.this.sql(self.dialect)}"], + self.f.descriptor(time_col), + window_step, + window_size, + offset, + ] + return sg.select( + sge.Column( + this=STAR, table=sg.to_identifier(table.alias_or_name, quoted=True) + ) + ).from_( + self.f.table(self.f.cumulate(*filter(None, args))).as_( + table.alias_or_name, quoted=True + ) + ) + + @visit_node.register(ops.InMemoryTable) + def visit_InMemoryTable(self, op, *, name, schema, data): + # the performance of this is rather terrible + tuples = data.to_frame().itertuples(index=False) + quoted = self.quoted + columns = [sg.column(col, quoted=quoted) for col in schema.names] + alias = sge.TableAlias( + this=sg.to_identifier(name, quoted=quoted), columns=columns + ) + expressions = [ + sge.Tuple( + expressions=[ + self.visit_Literal( + ops.Literal(col, dtype=dtype), value=col, dtype=dtype + ) + for col, dtype in zip(row, schema.types) + ] + ) + for row in tuples + ] + + expr = sge.Values(expressions=expressions, alias=alias) + return sg.select(*columns).from_(expr) + + def visit_NonNullLiteral(self, op, *, value, dtype): + if dtype.is_binary(): + # TODO: is this decode safe? + return self.cast(value.decode(), dtype) + elif dtype.is_uuid(): + return sge.convert(str(value)) + elif dtype.is_array(): + value_type = dtype.value_type + result = self.f.array( + *( + self.visit_Literal( + ops.Literal(v, dtype=value_type), value=v, dtype=value_type + ) + for v in value + ) + ) + if value: + return result + return sge.Cast(this=result, to=self.type_mapper.from_ibis(dtype)) + elif dtype.is_map(): + key_type = dtype.key_type + value_type = dtype.value_type + keys = self.f.array( + *( + self.visit_Literal( + ops.Literal(v, dtype=key_type), value=v, dtype=key_type + ) + for v in value.keys() + ) + ) + values = self.f.array( + *( + self.visit_Literal( + ops.Literal(v, dtype=value_type), value=v, dtype=value_type + ) + for v in value.values() + ) + ) + return self.cast(self.f.map_from_arrays(keys, values), dtype) + elif dtype.is_timestamp(): + return self.cast(value.replace(tzinfo=None).isoformat(sep=" "), dtype) + elif dtype.is_date(): + return self.cast(value.isoformat(), dtype) + elif dtype.is_time(): + return self.cast(value.isoformat(), dtype) + return None + + @visit_node.register(ops.ArrayIndex) + def visit_ArrayIndex(self, op, *, arg, index): + return sge.Bracket(this=arg, expressions=[index + 1]) + + @visit_node.register(ops.Xor) + def visit_Xor(self, op, *, left, right): + return sg.or_(sg.and_(left, sg.not_(right)), sg.and_(sg.not_(left), right)) + + @visit_node.register(ops.Literal) + def visit_Literal(self, op, *, value, dtype): + if value is None: + assert dtype.nullable, "dtype is not nullable but value is None" + if not dtype.is_null(): + return self.cast(sge.NULL, dtype) + return sge.NULL + return super().visit_Literal(op, value=value, dtype=dtype) + + @visit_node.register(ops.MapGet) + def visit_MapGet(self, op, *, arg, key, default): + if default is sge.NULL: + default = self.cast(default, op.dtype) + return self.f.coalesce(arg[key], default) + + @visit_node.register(ops.ArraySlice) + def visit_ArraySlice(self, op, *, arg, start, stop): + args = [arg, self.if_(start >= 0, start + 1, start)] + + if stop is not None: + args.append( + self.if_(stop >= 0, stop, self.f.cardinality(arg) - self.f.abs(stop)) + ) + + return self.f.array_slice(*args) + + @visit_node.register(ops.Not) + def visit_Not(self, op, *, arg): + return sg.not_(self.cast(arg, dt.boolean)) + + @visit_node.register(ops.Date) + def visit_Date(self, op, *, arg): + return self.cast(arg, dt.date) + + @visit_node.register(ops.TryCast) + def visit_TryCast(self, op, *, arg, to): + type_mapper = self.type_mapper + if op.arg.dtype.is_temporal() and to.is_numeric(): + return self.f.unix_timestamp( + sge.TryCast(this=arg, to=type_mapper.from_ibis(dt.string)) + ) + return sge.TryCast(this=arg, to=type_mapper.from_ibis(to)) + + @visit_node.register(ops.FloorDivide) + def visit_FloorDivide(self, op, *, left, right): + return self.f.floor(left / right) + + @visit_node.register(ops.JSONGetItem) + def visit_JSONGetItem(self, op, *, arg, index): + assert isinstance(op.index, ops.Literal) + idx = op.index + val = idx.value + if idx.dtype.is_integer(): + query_path = f"$[{val}]" + else: + assert idx.dtype.is_string(), idx.dtype + query_path = f"$.{val}" + + key_hack = f"{sge.convert(query_path).sql(self.dialect)} WITH CONDITIONAL ARRAY WRAPPER" + return self.f.json_query(arg, self.v[key_hack]) + + @visit_node.register(ops.TimestampFromUNIX) + def visit_TimestampFromUNIX(self, op, *, arg, unit): + from ibis.common.temporal import TimestampUnit + + if unit == TimestampUnit.MILLISECOND: + precision = 3 + elif unit == TimestampUnit.SECOND: + precision = 0 + else: + raise ValueError(f"{unit!r} unit is not supported!") + + return self.cast(self.f.to_timestamp_ltz(arg, precision), dt.timestamp) + + @visit_node.register(ops.Time) + def visit_Time(self, op, *, arg): + hour = self.f.extract("hour", arg) + minute = self.f.extract("minute", arg) + second = self.f.extract("second", arg) + padded_hour = self.f.lpad(self.cast(hour, dt.string), 2, "0") + padded_minute = self.f.lpad(self.cast(minute, dt.string), 2, "0") + padded_second = self.f.lpad(self.cast(second, dt.string), 2, "0") + return self.cast( + self.f.concat(padded_hour, ":", padded_minute, ":", padded_second), op.dtype + ) + + @visit_node.register(ops.TimeFromHMS) + def visit_TimeFromHMS(self, op, *, hours, minutes, seconds): + padded_hour = self.f.lpad(self.cast(hours, dt.string), 2, "0") + padded_minute = self.f.lpad(self.cast(minutes, dt.string), 2, "0") + padded_second = self.f.lpad(self.cast(seconds, dt.string), 2, "0") + return self.cast( + self.f.concat(padded_hour, ":", padded_minute, ":", padded_second), op.dtype + ) + + @visit_node.register(ops.DateFromYMD) + def visit_DateFromYMD(self, op, *, year, month, day): + padded_year = self.f.lpad(self.cast(year, dt.string), 4, "0") + padded_month = self.f.lpad(self.cast(month, dt.string), 2, "0") + padded_day = self.f.lpad(self.cast(day, dt.string), 2, "0") + return self.cast( + self.f.concat(padded_year, "-", padded_month, "-", padded_day), op.dtype + ) + + @visit_node.register(ops.TimestampFromYMDHMS) + def visit_TimestampFromYMDHMS( + self, op, *, year, month, day, hours, minutes, seconds + ): + padded_year = self.f.lpad(self.cast(year, dt.string), 4, "0") + padded_month = self.f.lpad(self.cast(month, dt.string), 2, "0") + padded_day = self.f.lpad(self.cast(day, dt.string), 2, "0") + padded_hour = self.f.lpad(self.cast(hours, dt.string), 2, "0") + padded_minute = self.f.lpad(self.cast(minutes, dt.string), 2, "0") + padded_second = self.f.lpad(self.cast(seconds, dt.string), 2, "0") + return self.cast( + self.f.concat( + padded_year, + "-", + padded_month, + "-", + padded_day, + " ", + padded_hour, + ":", + padded_minute, + ":", + padded_second, + ), + op.dtype, + ) + + @visit_node.register(ops.ExtractEpochSeconds) + def visit_ExtractEpochSeconds(self, op, *, arg): + return self.f.unix_timestamp(self.cast(arg, dt.string)) + + @visit_node.register(ops.TimestampDelta) + def visit_TimestampDelta(self, op, *, left, right, part): + return self.f.anon.timestampdiff(self.v[part], right, left) + + @visit_node.register(ops.Cast) + def visit_Cast(self, op, *, arg, to): + from_ = op.arg.dtype + if to.is_timestamp(): + if from_.is_numeric(): + arg = self.f.from_unixtime(arg) + if (tz := to.timezone) is not None: + return self.f.to_timestamp( + self.f.convert_tz(self.cast(arg, dt.string), "UTC+0", tz) + ) + else: + return self.f.to_timestamp(arg, "yyyy-MM-dd HH:mm:ss.SSS") + elif to.is_json(): + return arg + elif from_.is_temporal() and to.is_int64(): + return 1_000_000 * self.f.unix_timestamp(arg) + else: + return self.cast(arg, to) + + @visit_node.register(ops.IfElse) + def visit_IfElse(self, op, *, bool_expr, true_expr, false_null_expr): + return self.if_( + bool_expr, + true_expr if true_expr != sge.NULL else self.cast(true_expr, op.dtype), + ( + false_null_expr + if false_null_expr != sge.NULL + else self.cast(false_null_expr, op.dtype) + ), + ) + + @visit_node.register(ops.Log10) + def visit_Log10(self, op, *, arg): + return self.f.anon.log(10, arg) + + @visit_node.register(ops.ExtractMillisecond) + def visit_ExtractMillisecond(self, op, *, arg): + return self.f.extract(self.v.millisecond, arg) + + @visit_node.register(ops.ExtractMicrosecond) + def visit_ExtractMicrosecond(self, op, *, arg): + return self.f.extract(self.v.microsecond, arg) + + @visit_node.register(ops.DayOfWeekIndex) + def visit_DayOfWeekIndex(self, op, *, arg): + return (self.f.dayofweek(arg) + 5) % 7 + + @visit_node.register(ops.DayOfWeekName) + def visit_DayOfWeekName(self, op, *, arg): + index = self.cast(self.f.dayofweek(self.cast(arg, dt.date)), op.dtype) + lookup_table = self.f.str_to_map( + "1=Sunday,2=Monday,3=Tuesday,4=Wednesday,5=Thursday,6=Friday,7=Saturday" + ) + return lookup_table[index] + + @visit_node.register(ops.TimestampNow) + def visit_TimestampNow(self, op): + return self.v.current_timestamp + + @visit_node.register(ops.TimeDelta) + @visit_node.register(ops.TimestampDelta) + @visit_node.register(ops.DateDelta) + def visit_TemporalDelta(self, op, *, part, left, right): + return self.f.anon.timestampdiff( + self.v[part.this], + self.cast(right, dt.timestamp), + self.cast(left, dt.timestamp), + ) + + @visit_node.register(ops.TimestampBucket) + def visit_TimestampBucket(self, op, *, arg, interval, offset): + unit = op.interval.dtype.unit.name + unit_var = self.v[unit] + + if offset is None: + offset = 0 + else: + offset = op.offset.value + + bucket_width = op.interval.value + unit_func = self.f["dayofmonth" if unit.upper() == "DAY" else unit] + + arg = self.f.anon.timestampadd(unit_var, -paren(offset), arg) + mod = unit_func(arg) % bucket_width + + return self.f.anon.timestampadd( + unit_var, + -paren(mod) + offset, + self.v[f"FLOOR({arg.sql(self.dialect)} TO {unit_var.sql(self.dialect)})"], + ) + + @visit_node.register(ops.ApproxMedian) + @visit_node.register(ops.Arbitrary) + @visit_node.register(ops.ArgMax) + @visit_node.register(ops.ArgMin) + @visit_node.register(ops.ArrayCollect) + @visit_node.register(ops.ArrayFlatten) + @visit_node.register(ops.ArraySort) + @visit_node.register(ops.ArrayStringJoin) + @visit_node.register(ops.Correlation) + @visit_node.register(ops.CountDistinctStar) + @visit_node.register(ops.Covariance) + @visit_node.register(ops.DateDiff) + @visit_node.register(ops.ExtractURLField) + @visit_node.register(ops.FindInSet) + @visit_node.register(ops.IsInf) + @visit_node.register(ops.IsNan) + @visit_node.register(ops.Levenshtein) + @visit_node.register(ops.MapMerge) + @visit_node.register(ops.Median) + @visit_node.register(ops.MultiQuantile) + @visit_node.register(ops.Quantile) + @visit_node.register(ops.RegexSplit) + @visit_node.register(ops.RowID) + @visit_node.register(ops.ScalarUDF) + @visit_node.register(ops.StringSplit) + @visit_node.register(ops.TimestampTruncate) + @visit_node.register(ops.Translate) + @visit_node.register(ops.Unnest) + @visit_node.register(ops.ReductionVectorizedUDF) + @visit_node.register(ops.AnalyticVectorizedUDF) + def visit_Undefined(self, op, **_): + raise com.OperationNotDefinedError(type(op).__name__) + + @staticmethod + def _generate_groups(groups): + return groups + + +_SIMPLE_OPS = { + ops.All: "min", + ops.Any: "max", + ops.ArrayLength: "cardinality", + ops.ArrayPosition: "array_position", + ops.ArrayRemove: "array_remove", + ops.ArrayUnion: "array_union", + ops.ArrayContains: "array_contains", + ops.ArrayDistinct: "array_distinct", + ops.First: "first_value", + ops.Last: "last_value", + ops.Map: "map_from_arrays", + ops.Power: "power", + ops.StringLength: "char_length", + ops.Degrees: "degrees", + ops.Radians: "radians", + ops.RandomScalar: "rand", + ops.ApproxCountDistinct: "approx_count_distinct", + ops.RegexSearch: "regexp", + ops.StrRight: "right", + ops.StringToTimestamp: "to_timestamp", +} + + +for _op, _name in _SIMPLE_OPS.items(): + assert isinstance(type(_op), type), type(_op) + if issubclass(_op, ops.Reduction): + + @FlinkCompiler.visit_node.register(_op) + def _fmt(self, op, *, _name: str = _name, where, **kw): + return self.agg[_name](*kw.values(), where=where) + + else: + + @FlinkCompiler.visit_node.register(_op) + def _fmt(self, op, *, _name: str = _name, **kw): + return self.f[_name](*kw.values()) + + setattr(FlinkCompiler, f"visit_{_op.__name__}", _fmt) + + +del _op, _name, _fmt diff --git a/ibis/backends/flink/compiler/__init__.py b/ibis/backends/flink/compiler/__init__.py deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/ibis/backends/flink/compiler/core.py b/ibis/backends/flink/compiler/core.py deleted file mode 100644 index be187ebdb664a..0000000000000 --- a/ibis/backends/flink/compiler/core.py +++ /dev/null @@ -1,170 +0,0 @@ -"""Flink ibis expression to SQL string compiler.""" - -from __future__ import annotations - -import functools - -import ibis.common.exceptions as com -import ibis.expr.operations as ops -import ibis.expr.types as ir -from ibis.backends.base.sql.compiler import ( - Compiler, - Select, - SelectBuilder, - TableSetFormatter, -) -from ibis.backends.base.sql.registry import quote_identifier -from ibis.backends.base.sqlglot.dialects import Flink -from ibis.backends.flink.translator import FlinkExprTranslator - - -class FlinkTableSetFormatter(TableSetFormatter): - def _quote_identifier(self, name): - return quote_identifier(name, force=True) - - def _format_in_memory_table(self, op): - names = op.schema.names - raw_rows = [] - for row in op.data.to_frame().itertuples(index=False): - raw_row = [] - for val, name in zip(row, names): - lit = ops.Literal(val, dtype=op.schema[name]) - raw_row.append(self._translate(lit)) - raw_rows.append(", ".join(raw_row)) - rows = ", ".join(f"({raw_row})" for raw_row in raw_rows) - return f"(VALUES {rows})" - - def _format_window_tvf(self, op) -> str: - if isinstance(op, ops.TumbleWindowingTVF): - function_type = "TUMBLE" - elif isinstance(op, ops.HopWindowingTVF): - function_type = "HOP" - elif isinstance(op, ops.CumulateWindowingTVF): - function_type = "CUMULATE" - return f"TABLE({function_type}({format_windowing_tvf_params(op, self)}))" - - def _format_table(self, op) -> str: - ctx = self.context - if isinstance(op, ops.WindowingTVF): - formatted_table = self._format_window_tvf(op) - return f"{formatted_table} {ctx.get_ref(op)}" - else: - result = super()._format_table(op) - - ref_op = op - if isinstance(op, ops.SelfReference): - ref_op = op.table - - if isinstance(ref_op, ops.InMemoryTable): - names = op.schema.names - result += f"({', '.join(self._quote_identifier(name) for name in names)})" - - return result - - -class FlinkSelectBuilder(SelectBuilder): - def _convert_group_by(self, exprs): - return exprs - - -class FlinkSelect(Select): - def format_group_by(self) -> str: - if not len(self.group_by): - # There is no aggregation, nothing to see here - return None - - lines = [] - if len(self.group_by) > 0: - group_keys = map(self._translate, self.group_by) - clause = "GROUP BY {}".format(", ".join(list(group_keys))) - lines.append(clause) - - if len(self.having) > 0: - trans_exprs = [] - for expr in self.having: - translated = self._translate(expr) - trans_exprs.append(translated) - lines.append("HAVING {}".format(" AND ".join(trans_exprs))) - - return "\n".join(lines) - - -class FlinkCompiler(Compiler): - translator_class = FlinkExprTranslator - table_set_formatter_class = FlinkTableSetFormatter - select_builder_class = FlinkSelectBuilder - select_class = FlinkSelect - - cheap_in_memory_tables = True - - dialect = Flink - - @classmethod - def to_sql(cls, node, context=None, params=None): - if isinstance(node, ir.Expr): - node = node.op() - - if isinstance(node, ops.Literal): - from ibis.backends.flink.utils import translate_literal - - return translate_literal(node) - - return super().to_sql(node, context, params) - - -@functools.singledispatch -def format_windowing_tvf_params( - op: ops.WindowingTVF, formatter: TableSetFormatter -) -> str: - raise com.OperationNotDefinedError(f"No formatting rule for {type(op)}") - - -@format_windowing_tvf_params.register(ops.TumbleWindowingTVF) -def _tumble_window_params( - op: ops.TumbleWindowingTVF, formatter: TableSetFormatter -) -> str: - return ", ".join( - filter( - None, - [ - f"TABLE {formatter._quote_identifier(op.table.name)}", - f"DESCRIPTOR({formatter._translate(op.time_col)})", - formatter._translate(op.window_size), - formatter._translate(op.offset) if op.offset else None, - ], - ) - ) - - -@format_windowing_tvf_params.register(ops.HopWindowingTVF) -def _hop_window_params(op: ops.HopWindowingTVF, formatter: TableSetFormatter) -> str: - return ", ".join( - filter( - None, - [ - f"TABLE {formatter._quote_identifier(op.table.name)}", - f"DESCRIPTOR({formatter._translate(op.time_col)})", - formatter._translate(op.window_slide), - formatter._translate(op.window_size), - formatter._translate(op.offset) if op.offset else None, - ], - ) - ) - - -@format_windowing_tvf_params.register(ops.CumulateWindowingTVF) -def _cumulate_window_params( - op: ops.CumulateWindowingTVF, formatter: TableSetFormatter -) -> str: - return ", ".join( - filter( - None, - [ - f"TABLE {formatter._quote_identifier(op.table.name)}", - f"DESCRIPTOR({formatter._translate(op.time_col)})", - formatter._translate(op.window_step), - formatter._translate(op.window_size), - formatter._translate(op.offset) if op.offset else None, - ], - ) - ) diff --git a/ibis/backends/flink/ddl.py b/ibis/backends/flink/ddl.py index c48f5063defda..f392b47fb901d 100644 --- a/ibis/backends/flink/ddl.py +++ b/ibis/backends/flink/ddl.py @@ -19,7 +19,7 @@ is_fully_qualified, ) from ibis.backends.base.sql.registry import quote_identifier -from ibis.backends.flink.registry import type_to_sql_string +from ibis.backends.base.sqlglot.datatypes import FlinkType from ibis.util import promote_list if TYPE_CHECKING: @@ -41,13 +41,11 @@ def _format_schema_element(name, t): def type_to_flink_sql_string(tval): + sql_string = FlinkType.from_ibis(tval) if tval.is_timestamp(): - return f"timestamp({tval.scale})" if tval.scale is not None else "timestamp" + return f"TIMESTAMP({tval.scale})" if tval.scale is not None else "TIMESTAMP" else: - sql_string = type_to_sql_string(tval) - if not tval.nullable: - sql_string += " NOT NULL" - return sql_string + return sql_string.sql("flink") + " NOT NULL" * (not tval.nullable) def _format_watermark_strategy(watermark: Watermark) -> str: @@ -364,7 +362,7 @@ def compile(self): else: partition = "" - select_query = self.select.compile() + select_query = self.select scoped_name = self._get_scoped_name( self.table_name, self.database, self.catalog ) diff --git a/ibis/backends/flink/registry.py b/ibis/backends/flink/registry.py index 3fc518de536d3..b5ec4914ab60f 100644 --- a/ibis/backends/flink/registry.py +++ b/ibis/backends/flink/registry.py @@ -4,18 +4,9 @@ import ibis.common.exceptions as com import ibis.expr.operations as ops -from ibis.backends.base.sql.registry import ( - aggregate, - fixed_arity, - helpers, - quote_identifier, - unary, -) from ibis.backends.base.sql.registry import ( operation_registry as base_operation_registry, ) -from ibis.backends.base.sql.registry.main import varargs -from ibis.common.temporal import TimestampUnit if TYPE_CHECKING: from ibis.backends.base.sql.compiler import ExprTranslator @@ -23,33 +14,6 @@ operation_registry = base_operation_registry.copy() -def type_to_sql_string(tval): - if tval.is_array(): - return f"array<{helpers.type_to_sql_string(tval.value_type)}>" - return helpers.type_to_sql_string(tval) - - -def _not(translator: ExprTranslator, op: ops.Node) -> str: - formatted_arg = translator.translate(op.arg) - if helpers.needs_parens(op.arg): - formatted_arg = helpers.parenthesize(formatted_arg) - return f"NOT CAST({formatted_arg} AS boolean)" - - -def _count_star(translator: ExprTranslator, op: ops.Node) -> str: - if (where := op.where) is not None: - condition = f" FILTER (WHERE {translator.translate(where)})" - else: - condition = "" - - return f"COUNT(*){condition}" - - -def _string_concat(translator: ExprTranslator, op: ops.StringConcat) -> str: - joined_args = ", ".join(map(translator.translate, op.arg)) - return f"CONCAT({joined_args})" - - def _strftime(translator: ExprTranslator, op: ops.Strftime) -> str: import sqlglot as sg @@ -67,42 +31,6 @@ def _strftime(translator: ExprTranslator, op: ops.Strftime) -> str: return f"FROM_UNIXTIME(UNIX_TIMESTAMP({arg}), {transformed_format_str})" -def _date(translator: ExprTranslator, op: ops.Node) -> str: - (arg,) = op.args - return f"CAST({translator.translate(arg)} AS DATE)" - - -def _extract_field(sql_attr: str) -> str: - def extract_field_formatter(translator: ExprTranslator, op: ops.Node) -> str: - arg = translator.translate(op.args[0]) - return f"EXTRACT({sql_attr} from {arg})" - - return extract_field_formatter - - -def _cast(translator: ExprTranslator, op: ops.generic.Cast) -> str: - arg, to = op.arg, op.to - arg_translated = translator.translate(arg) - if to.is_timestamp(): - if arg.dtype.is_numeric(): - arg_translated = f"FROM_UNIXTIME({arg_translated})" - - if to.timezone: - return f"TO_TIMESTAMP(CONVERT_TZ(CAST({arg_translated} AS STRING), 'UTC+0', '{to.timezone}'))" - else: - return f"TO_TIMESTAMP({arg_translated}, 'yyyy-MM-dd HH:mm:ss.SSS')" - - elif to.is_date(): - return f"CAST({arg_translated} AS date)" - elif to.is_json(): - return arg_translated - elif op.arg.dtype.is_temporal() and op.to.is_int64(): - return f"1000000 * unix_timestamp({arg_translated})" - else: - sql_type = type_to_sql_string(op.to) - return f"CAST({arg_translated} AS {sql_type})" - - def _left_op_right(translator: ExprTranslator, op_node: ops.Node, op_sign: str) -> str: """Utility to be used in operators that perform '{op.left} {op_sign} {op.right}'.""" return f"{translator.translate(op_node.left)} {op_sign} {translator.translate(op_node.right)}" @@ -118,43 +46,6 @@ def _interval_subtract( return _left_op_right(translator=translator, op_node=op, op_sign="-") -def _literal(translator: ExprTranslator, op: ops.Literal) -> str: - from ibis.backends.flink.utils import translate_literal - - return translate_literal(op) - - -def _try_cast(translator: ExprTranslator, op: ops.Node) -> str: - arg_formatted = translator.translate(op.arg) - - if op.arg.dtype.is_temporal() and op.to.is_numeric(): - # The cast from TIMESTAMP type to NUMERIC type is not allowed. - # It's recommended to use UNIX_TIMESTAMP(CAST(timestamp_col AS STRING)) instead. - return f"UNIX_TIMESTAMP(TRY_CAST({arg_formatted} AS STRING))" - else: - sql_type = type_to_sql_string(op.to) - return f"TRY_CAST({arg_formatted} AS {sql_type})" - - -def _filter(translator: ExprTranslator, op: ops.Node) -> str: - bool_expr = translator.translate(op.bool_expr) - true_expr = translator.translate(op.true_expr) - false_null_expr = translator.translate(op.false_null_expr) - - # [TODO](chloeh13q): It's preferable to use the FILTER syntax instead of CASE WHEN - # to let the planner do more optimizations to reduce the state size; besides, FILTER - # is more compliant with the SQL standard. - # For example, - # ``` - # COUNT(DISTINCT CASE WHEN flag = 'app' THEN user_id ELSE NULL END) AS app_uv - # ``` - # is equivalent to - # ``` - # COUNT(DISTINCT) FILTER (WHERE flag = 'app') AS app_uv - # ``` - return f"CASE WHEN {bool_expr} THEN {true_expr} ELSE {false_null_expr} END" - - def _format_window_start(translator: ExprTranslator, boundary): if boundary is None: return "UNBOUNDED PRECEDING" @@ -272,176 +163,14 @@ def _clip(translator: ExprTranslator, op: ops.Node) -> str: return f"CAST({arg} AS {FlinkType.from_ibis(op.dtype)!s})" -def _ntile(translator: ExprTranslator, op: ops.NTile) -> str: - return f"NTILE({op.buckets.value})" - - -def _floor_divide(translator: ExprTranslator, op: ops.Node) -> str: - left = translator.translate(op.left) - right = translator.translate(op.right) - return f"FLOOR(({left}) / ({right}))" - - -def _array(translator: ExprTranslator, op: ops.Array) -> str: - return f"ARRAY[{', '.join(map(translator.translate, op.exprs))}]" - - -def _array_index(translator: ExprTranslator, op: ops.ArrayIndex): - table_column = op.arg - index = op.index - - table_column_translated = translator.translate(table_column) - index_translated = translator.translate(index) - - return f"{table_column_translated} [ {index_translated} + 1 ]" - - -def _array_length(translator: ExprTranslator, op: ops.ArrayLength) -> str: - return f"CARDINALITY({translator.translate(op.arg)})" - - -def _array_position(translator: ExprTranslator, op: ops.ArrayPosition) -> str: - arg = translator.translate(op.arg) - other = translator.translate(op.other) - return f"ARRAY_POSITION({arg}, {other}) - 1" - - -def _array_slice(translator: ExprTranslator, op: ops.ArraySlice) -> str: - array = translator.translate(op.arg) - start = op.start.value - # The offsets are 1-based for ARRAY_SLICE. - # Ref: https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/functions/systemfunctions - if start >= 0: - start += 1 - - if op.stop is None: - return f"ARRAY_SLICE({array}, {start})" - - stop = op.stop.value - if stop >= 0: - return f"ARRAY_SLICE({array}, {start}, {stop})" - else: - # To imitate the behavior of pandas array slicing. - return f"ARRAY_SLICE({array}, {start}, CARDINALITY({array}) - {abs(stop)})" - - -def _json_get_item(translator: ExprTranslator, op: ops.json.JSONGetItem) -> str: - arg_translated = translator.translate(op.arg) - if op.index.dtype.is_integer(): - query_path = f"$[{op.index.value}]" - else: # is string - query_path = f"$.{op.index.value}" - - return ( - f"JSON_QUERY({arg_translated}, '{query_path}' WITH CONDITIONAL ARRAY WRAPPER)" - ) - - -def _map(translator: ExprTranslator, op: ops.maps.Map) -> str: - key_array = translator.translate(op.keys) - value_array = translator.translate(op.values) - - return f"MAP_FROM_ARRAYS({key_array}, {value_array})" - - -def _map_get(translator: ExprTranslator, op: ops.maps.MapGet) -> str: - map_ = translator.translate(op.arg) - key = translator.translate(op.key) - return f"{map_} [ {key} ]" - - -def _struct_field(translator: ExprTranslator, op: ops.StructField) -> str: - arg = translator.translate(op.arg) - return f"{arg}.`{op.field}`" - - -def _day_of_week_index( - translator: ExprTranslator, op: ops.temporal.DayOfWeekIndex -) -> str: - arg = translator.translate(op.arg) - return f"MOD(DAYOFWEEK({arg}) + 5, 7)" - - -def _day_of_week_name( - translator: ExprTranslator, op: ops.temporal.DayOfWeekName -) -> str: - arg = translator.translate(op.arg) - map_str = "1=Sunday,2=Monday,3=Tuesday,4=Wednesday,5=Thursday,6=Friday,7=Saturday" - return f"STR_TO_MAP('{map_str}')[CAST(DAYOFWEEK(CAST({arg} AS DATE)) AS STRING)]" - - def _date_add(translator: ExprTranslator, op: ops.temporal.DateAdd) -> str: return _left_op_right(translator=translator, op_node=op, op_sign="+") -def _date_delta(translator: ExprTranslator, op: ops.temporal.DateDelta) -> str: - left = translator.translate(op.left) - right = translator.translate(op.right) - unit = op.part.value.upper() - - return ( - f"TIMESTAMPDIFF({unit}, CAST({right} AS TIMESTAMP), CAST({left} AS TIMESTAMP))" - ) - - -def _date_diff(translator: ExprTranslator, op: ops.temporal.DateDiff) -> str: - raise com.UnsupportedOperationError("DATE_DIFF is not supported in Flink.") - - -def _date_from_ymd(translator: ExprTranslator, op: ops.temporal.DateFromYMD) -> str: - year, month, day = ( - f"CAST({translator.translate(e)} AS STRING)" - for e in [op.year, op.month, op.day] - ) - concat_string = f"CONCAT({year}, '-', {month}, '-', {day})" - return f"CAST({concat_string} AS DATE)" - - def _date_sub(translator: ExprTranslator, op: ops.temporal.DateSub) -> str: return _left_op_right(translator=translator, op_node=op, op_sign="-") -def _extract_epoch_seconds(translator: ExprTranslator, op: ops.Node) -> str: - arg = translator.translate(op.arg) - return f"UNIX_TIMESTAMP(CAST({arg} AS STRING))" - - -def _string_to_timestamp( - translator: ExprTranslator, op: ops.temporal.StringToTimestamp -) -> str: - arg = translator.translate(op.arg) - format_string = translator.translate(op.format_str) - return f"TO_TIMESTAMP({arg}, {format_string})" - - -def _time(translator: ExprTranslator, op: ops.temporal.Time) -> str: - if op.arg.dtype.is_timestamp(): - datetime = op.arg.value - return f"TIME '{datetime.hour}:{datetime.minute}:{datetime.second}.{datetime.microsecond}'" - - else: - raise com.UnsupportedOperationError(f"Does NOT support dtype= {op.arg.dtype}") - - -def _time_delta(translator: ExprTranslator, op: ops.temporal.TimeDiff) -> str: - left = translator.translate(op.left) - right = translator.translate(op.right) - unit = op.part.value.upper() - - return ( - f"TIMESTAMPDIFF({unit}, CAST({right} AS TIMESTAMP), CAST({left} AS TIMESTAMP))" - ) - - -def _time_from_hms(translator: ExprTranslator, op: ops.temporal.TimeFromHMS) -> str: - hours, minutes, seconds = ( - f"CAST({translator.translate(e)} AS STRING)" - for e in [op.hours, op.minutes, op.seconds] - ) - concat_string = f"CONCAT({hours}, ':', {minutes}, ':', {seconds})" - return f"CAST({concat_string} AS TIME)" - - def _timestamp_add(translator: ExprTranslator, op: ops.temporal.TimestampAdd) -> str: return _left_op_right(translator=translator, op_node=op, op_sign="+") @@ -463,152 +192,25 @@ def _timestamp_bucket( return f"TIMESTAMPADD({unit}, -({mod}) + {offset}, FLOOR({arg_offset} TO {unit}))" -def _timestamp_delta( - translator: ExprTranslator, op: ops.temporal.TimestampDelta -) -> str: - left = translator.translate(op.left) - right = translator.translate(op.right) - unit = op.part.value.upper() - - return f"TIMESTAMPDIFF({unit}, {right}, {left})" - - def _timestamp_diff(translator: ExprTranslator, op: ops.temporal.TimestampDiff) -> str: return _left_op_right(translator=translator, op_node=op, op_sign="-") -def _timestamp_sub(translator: ExprTranslator, op: ops.temporal.TimestampSub) -> str: - table_column = op.left - interval = op.right - - table_column_translated = translator.translate(table_column) - interval_translated = translator.translate(interval) - return f"{table_column_translated} - {interval_translated}" - - -def _timestamp_from_unix(translator: ExprTranslator, op: ops.TimestampFromUNIX) -> str: - arg, unit = op.arg, op.unit - - if unit == TimestampUnit.MILLISECOND: - precision = 3 - elif unit == TimestampUnit.SECOND: - precision = 0 - else: - raise ValueError(f"{unit!r} unit is not supported!") - - arg = translator.translate(op.arg) - return f"CAST(TO_TIMESTAMP_LTZ({arg}, {precision}) AS TIMESTAMP)" - - -def _timestamp_from_ymdhms( - translator: ExprTranslator, op: ops.temporal.TimestampFromYMDHMS -) -> str: - year, month, day, hours, minutes, seconds = ( - f"CAST({translator.translate(e)} AS STRING)" - for e in [op.year, op.month, op.day, op.hours, op.minutes, op.seconds] - ) - concat_string = f"CONCAT({year}, '-', {month}, '-', {day}, ' ', {hours}, ':', {minutes}, ':', {seconds})" - return f"CAST({concat_string} AS TIMESTAMP)" - - -def _struct_field(translator, op): - arg = translator.translate(op.arg) - return f"{arg}.{quote_identifier(op.field, force=True)}" - - operation_registry.update( { - # Unary operations - ops.Not: _not, - ops.NullIf: fixed_arity("nullif", 2), - ops.RandomScalar: lambda *_: "rand()", - ops.Degrees: unary("degrees"), - ops.Radians: unary("radians"), # Unary aggregates - ops.ApproxCountDistinct: aggregate.reduction("approx_count_distinct"), - ops.CountStar: _count_star, # String operations - ops.RegexSearch: fixed_arity("regexp", 2), - ops.StringConcat: _string_concat, ops.Strftime: _strftime, - ops.StringLength: unary("char_length"), - ops.StrRight: fixed_arity("right", 2), - # Timestamp operations - ops.Date: _date, - ops.ExtractEpochSeconds: _extract_epoch_seconds, - ops.ExtractYear: _extract_field("year"), # equivalent to YEAR(date) - ops.ExtractMonth: _extract_field("month"), # equivalent to MONTH(date) - ops.ExtractDay: _extract_field("day"), # equivalent to DAYOFMONTH(date) - ops.ExtractQuarter: _extract_field("quarter"), # equivalent to QUARTER(date) - ops.ExtractWeekOfYear: _extract_field("week"), # equivalent to WEEK(date) - ops.ExtractDayOfYear: _extract_field("doy"), # equivalent to DAYOFYEAR(date) - ops.ExtractHour: _extract_field("hour"), # equivalent to HOUR(timestamp) - ops.ExtractMinute: _extract_field("minute"), # equivalent to MINUTE(timestamp) - ops.ExtractSecond: _extract_field("second"), # equivalent to SECOND(timestamp) - ops.ExtractMillisecond: _extract_field("millisecond"), - ops.ExtractMicrosecond: _extract_field("microsecond"), # Other operations - ops.Cast: _cast, - ops.Coalesce: varargs("coalesce"), ops.IntervalAdd: _interval_add, ops.IntervalSubtract: _interval_subtract, - ops.Literal: _literal, - ops.TryCast: _try_cast, - ops.IfElse: _filter, ops.Window: _window, ops.Clip: _clip, - ops.NTile: _ntile, - # Binary operations - ops.Power: fixed_arity("power", 2), - ops.FloorDivide: _floor_divide, - # Collection operations - ops.Array: _array, - ops.ArrayContains: fixed_arity("ARRAY_CONTAINS", 2), - ops.ArrayDistinct: fixed_arity("ARRAY_DISTINCT", 1), - ops.ArrayIndex: _array_index, - ops.ArrayLength: _array_length, - ops.ArrayPosition: _array_position, - ops.ArrayRemove: fixed_arity("ARRAY_REMOVE", 2), - ops.ArraySlice: _array_slice, - ops.ArrayUnion: fixed_arity("ARRAY_UNION", 2), - ops.JSONGetItem: _json_get_item, - ops.Map: _map, - ops.MapGet: _map_get, - ops.StructField: _struct_field, # Temporal functions ops.DateAdd: _date_add, - ops.DateDelta: _date_delta, - ops.DateDiff: _date_diff, - ops.DateFromYMD: _date_from_ymd, ops.DateSub: _date_sub, - ops.DayOfWeekIndex: _day_of_week_index, - ops.DayOfWeekName: _day_of_week_name, - ops.StringToTimestamp: _string_to_timestamp, - ops.Time: _time, - ops.TimeDelta: _time_delta, - ops.TimeFromHMS: _time_from_hms, ops.TimestampAdd: _timestamp_add, ops.TimestampBucket: _timestamp_bucket, - ops.TimestampDelta: _timestamp_delta, ops.TimestampDiff: _timestamp_diff, - ops.TimestampFromUNIX: _timestamp_from_unix, - ops.TimestampFromYMDHMS: _timestamp_from_ymdhms, - ops.TimestampSub: _timestamp_sub, - ops.StructField: _struct_field, } ) - -_invalid_operations = { - # ibis.expr.operations.numeric - ops.IsNan, - ops.IsInf, - # ibis.expr.operations.reductions - ops.ApproxMedian, - # ibis.expr.operations.strings - ops.Translate, - ops.FindInSet, -} - -operation_registry = { - k: v for k, v in operation_registry.items() if k not in _invalid_operations -} diff --git a/ibis/backends/flink/tests/conftest.py b/ibis/backends/flink/tests/conftest.py index 5fa2078432e41..db420b8f55dc0 100644 --- a/ibis/backends/flink/tests/conftest.py +++ b/ibis/backends/flink/tests/conftest.py @@ -11,6 +11,7 @@ class TestConf(BackendTest): force_sort = True + stateful = False deps = "pandas", "pyflink" @staticmethod @@ -110,16 +111,6 @@ def con(tmp_path_factory, data_dir, worker_id): ).connection -@pytest.fixture(scope="session") -def db(con): - return con.database() - - -@pytest.fixture(scope="session") -def alltypes(con): - return con.tables.functional_alltypes - - @pytest.fixture def awards_players_schema(): return TEST_TABLES["awards_players"] @@ -163,39 +154,3 @@ def generate_csv_configs(csv_file): } return generate_csv_configs - - -@pytest.fixture -def temp_view(con) -> str: - """Return a temporary view name. - - Parameters - ---------- - con : backend connection - - Yields - ------ - name : string - Random view name for a temporary usage. - - Note (mehmet): Added this here because the fixture - ibis/ibis/backends/conftest.py::temp_view() - leads to docker related errors through its parameter `ddl_con`. - """ - from ibis import util - - name = util.gen_name("view") - yield name - - con.drop_view(name, force=True) - - -@pytest.fixture(autouse=True) -def reset_con(con): - yield - tables_to_drop = list(set(con.list_tables()) - set(TEST_TABLES.keys())) - for table in tables_to_drop: - con.drop_table(table, force=True) - views_to_drop = list(set(con.list_views()) - set(TEST_TABLES.keys())) - for view in views_to_drop: - con.drop_view(view, temp=True, force=True) diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_complex_filtered_agg/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_complex_filtered_agg/out.sql index 5b9990fe38920..e3cdeee88ea52 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_complex_filtered_agg/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_complex_filtered_agg/out.sql @@ -1,5 +1,11 @@ -SELECT t0.`b`, COUNT(*) AS `total`, avg(t0.`a`) AS `avg_a`, - avg(CASE WHEN t0.`g` = 'A' THEN t0.`a` ELSE NULL END) AS `avg_a_A`, - avg(CASE WHEN t0.`g` = 'B' THEN t0.`a` ELSE NULL END) AS `avg_a_B` -FROM table t0 -GROUP BY t0.`b` \ No newline at end of file +SELECT + `t0`.`b`, + COUNT(*) AS `total`, + AVG(`t0`.`a`) AS `avg_a`, + AVG(`t0`.`a`) FILTER(WHERE + `t0`.`g` = 'A') AS `avg_a_A`, + AVG(`t0`.`a`) FILTER(WHERE + `t0`.`g` = 'B') AS `avg_a_B` +FROM `table` AS `t0` +GROUP BY + `t0`.`b` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_complex_groupby_aggregation/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_complex_groupby_aggregation/out.sql index 1ce799579cd5a..e86b81ec26ee9 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_complex_groupby_aggregation/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_complex_groupby_aggregation/out.sql @@ -1,5 +1,9 @@ -SELECT EXTRACT(year from t0.`i`) AS `year`, - EXTRACT(month from t0.`i`) AS `month`, COUNT(*) AS `total`, - count(DISTINCT t0.`b`) AS `b_unique` -FROM table t0 -GROUP BY EXTRACT(year from t0.`i`), EXTRACT(month from t0.`i`) \ No newline at end of file +SELECT + EXTRACT(year FROM `t0`.`i`) AS `year`, + EXTRACT(month FROM `t0`.`i`) AS `month`, + COUNT(*) AS `total`, + COUNT(DISTINCT `t0`.`b`) AS `b_unique` +FROM `table` AS `t0` +GROUP BY + EXTRACT(year FROM `t0`.`i`), + EXTRACT(month FROM `t0`.`i`) \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_complex_projections/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_complex_projections/out.sql index c883cc8ab63a8..f9d32940529d7 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_complex_projections/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_complex_projections/out.sql @@ -1,7 +1,15 @@ -SELECT t0.`a`, avg(abs(t0.`the_sum`)) AS `mad` +SELECT + `t1`.`a`, + AVG(ABS(`t1`.`the_sum`)) AS `mad` FROM ( - SELECT t1.`a`, t1.`c`, sum(t1.`b`) AS `the_sum` - FROM table t1 - GROUP BY t1.`a`, t1.`c` -) t0 -GROUP BY t0.`a` \ No newline at end of file + SELECT + `t0`.`a`, + `t0`.`c`, + SUM(`t0`.`b`) AS `the_sum` + FROM `table` AS `t0` + GROUP BY + `t0`.`a`, + `t0`.`c` +) AS `t1` +GROUP BY + `t1`.`a` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_count_star/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_count_star/out.sql index e3e3e49089cda..b94c832490c0b 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_count_star/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_count_star/out.sql @@ -1,3 +1,6 @@ -SELECT t0.`i`, COUNT(*) AS `CountStar(table)` -FROM table t0 -GROUP BY t0.`i` \ No newline at end of file +SELECT + `t0`.`i`, + COUNT(*) AS `CountStar(table)` +FROM `table` AS `t0` +GROUP BY + `t0`.`i` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/day/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/day/out.sql index 14d96a04c89ca..1f9573020c9dd 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/day/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/day/out.sql @@ -1,2 +1,3 @@ -SELECT EXTRACT(day from t0.`i`) AS `tmp` -FROM table t0 \ No newline at end of file +SELECT + EXTRACT(day FROM `t0`.`i`) AS `tmp` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/day_of_year/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/day_of_year/out.sql index 9774a20af63a9..5ca96f75f2b39 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/day_of_year/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/day_of_year/out.sql @@ -1,2 +1,3 @@ -SELECT EXTRACT(doy from t0.`i`) AS `tmp` -FROM table t0 \ No newline at end of file +SELECT + EXTRACT(dayofyear FROM `t0`.`i`) AS `tmp` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/hour/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/hour/out.sql index e19999b7b1f1d..c91eccac3d062 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/hour/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/hour/out.sql @@ -1,2 +1,3 @@ -SELECT EXTRACT(hour from t0.`i`) AS `tmp` -FROM table t0 \ No newline at end of file +SELECT + EXTRACT(hour FROM `t0`.`i`) AS `tmp` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/minute/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/minute/out.sql index aeed550bdab21..a621ade98023c 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/minute/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/minute/out.sql @@ -1,2 +1,3 @@ -SELECT EXTRACT(minute from t0.`i`) AS `tmp` -FROM table t0 \ No newline at end of file +SELECT + EXTRACT(minute FROM `t0`.`i`) AS `tmp` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/month/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/month/out.sql index 57e3d1c6fa45e..e5596983477c0 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/month/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/month/out.sql @@ -1,2 +1,3 @@ -SELECT EXTRACT(month from t0.`i`) AS `tmp` -FROM table t0 \ No newline at end of file +SELECT + EXTRACT(month FROM `t0`.`i`) AS `tmp` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/quarter/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/quarter/out.sql index c6170172bde11..6afeb5e1b26f4 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/quarter/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/quarter/out.sql @@ -1,2 +1,3 @@ -SELECT EXTRACT(quarter from t0.`i`) AS `tmp` -FROM table t0 \ No newline at end of file +SELECT + EXTRACT(quarter FROM `t0`.`i`) AS `tmp` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/second/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/second/out.sql index 0c32dade798e5..0478f739a6c7f 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/second/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/second/out.sql @@ -1,2 +1,3 @@ -SELECT EXTRACT(second from t0.`i`) AS `tmp` -FROM table t0 \ No newline at end of file +SELECT + EXTRACT(second FROM `t0`.`i`) AS `tmp` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/week_of_year/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/week_of_year/out.sql index 3f6ca60700ebc..aac465a39a88c 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/week_of_year/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/week_of_year/out.sql @@ -1,2 +1,3 @@ -SELECT EXTRACT(week from t0.`i`) AS `tmp` -FROM table t0 \ No newline at end of file +SELECT + EXTRACT(week FROM `t0`.`i`) AS `tmp` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/year/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/year/out.sql index ab354a7fd8612..09d387170f993 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/year/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_extract_fields/year/out.sql @@ -1,2 +1,3 @@ -SELECT EXTRACT(year from t0.`i`) AS `tmp` -FROM table t0 \ No newline at end of file +SELECT + EXTRACT(year FROM `t0`.`i`) AS `tmp` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_filter/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_filter/out.sql index a74d83d269b56..079893708b237 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_filter/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_filter/out.sql @@ -1,4 +1,21 @@ -SELECT t0.* -FROM table t0 -WHERE ((t0.`c` > CAST(0 AS TINYINT)) OR (t0.`c` < CAST(0 AS TINYINT))) AND - (t0.`g` IN ('A', 'B')) \ No newline at end of file +SELECT + `t0`.`a`, + `t0`.`b`, + `t0`.`c`, + `t0`.`d`, + `t0`.`e`, + `t0`.`f`, + `t0`.`g`, + `t0`.`h`, + `t0`.`i`, + `t0`.`j`, + `t0`.`k` +FROM `table` AS `t0` +WHERE + ( + ( + `t0`.`c` > 0 + ) OR ( + `t0`.`c` < 0 + ) + ) AND `t0`.`g` IN ('A', 'B') \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_having/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_having/out.sql index 3744dd045f0de..4dc04f8e5686a 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_having/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_having/out.sql @@ -1,4 +1,14 @@ -SELECT t0.`g`, sum(t0.`b`) AS `b_sum` -FROM table t0 -GROUP BY t0.`g` -HAVING COUNT(*) >= CAST(1000 AS SMALLINT) \ No newline at end of file +SELECT + `t1`.`g`, + `t1`.`b_sum` +FROM ( + SELECT + `t0`.`g`, + SUM(`t0`.`b`) AS `b_sum`, + COUNT(*) AS `CountStar(table)` + FROM `table` AS `t0` + GROUP BY + `t0`.`g` +) AS `t1` +WHERE + `t1`.`CountStar(table)` >= 1000 \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_simple_filtered_agg/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_simple_filtered_agg/out.sql index 19afa7a54cdf1..564a245016ab9 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_simple_filtered_agg/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_simple_filtered_agg/out.sql @@ -1,2 +1,4 @@ -SELECT count(DISTINCT CASE WHEN t0.`g` = 'A' THEN t0.`b` ELSE NULL END) AS `CountDistinct(b, Equals(g, 'A'))` -FROM table t0 \ No newline at end of file +SELECT + COUNT(DISTINCT `t0`.`b`) FILTER(WHERE + `t0`.`g` = 'A') AS `CountDistinct(b, Equals(g, 'A'))` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_sum/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_sum/out.sql index 4cd6d4fde779f..3701dafb94c45 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_sum/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_sum/out.sql @@ -1,2 +1,3 @@ -SELECT sum(t0.`a`) AS `Sum(a)` -FROM table t0 \ No newline at end of file +SELECT + SUM(`t0`.`a`) AS `Sum(a)` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_timestamp_from_unix/timestamp_ms/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_timestamp_from_unix/timestamp_ms/out.sql index c2424f7e63e2d..940b6016a3365 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_timestamp_from_unix/timestamp_ms/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_timestamp_from_unix/timestamp_ms/out.sql @@ -1,2 +1,3 @@ -SELECT CAST(TO_TIMESTAMP_LTZ(t0.`d`, 3) AS TIMESTAMP) AS `TimestampFromUNIX(d)` -FROM table t0 \ No newline at end of file +SELECT + CAST(TO_TIMESTAMP_LTZ(`t0`.`d`, 3) AS TIMESTAMP) AS `TimestampFromUNIX(d, MILLISECOND)` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_timestamp_from_unix/timestamp_s/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_timestamp_from_unix/timestamp_s/out.sql index 31766957381dc..91c783b17492e 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_timestamp_from_unix/timestamp_s/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_timestamp_from_unix/timestamp_s/out.sql @@ -1,2 +1,3 @@ -SELECT CAST(TO_TIMESTAMP_LTZ(t0.`d`, 0) AS TIMESTAMP) AS `TimestampFromUNIX(d)` -FROM table t0 \ No newline at end of file +SELECT + CAST(TO_TIMESTAMP_LTZ(`t0`.`d`, 0) AS TIMESTAMP) AS `TimestampFromUNIX(d, SECOND)` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_value_counts/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_value_counts/out.sql index 53e792c45392a..5645f83ecec1d 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_value_counts/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_value_counts/out.sql @@ -1,6 +1,10 @@ -SELECT t0.`ExtractYear(i)`, COUNT(*) AS `ExtractYear(i)_count` +SELECT + `t1`.`ExtractYear(i)`, + COUNT(*) AS `ExtractYear(i)_count` FROM ( - SELECT EXTRACT(year from t1.`i`) AS `ExtractYear(i)` - FROM table t1 -) t0 -GROUP BY t0.`ExtractYear(i)` \ No newline at end of file + SELECT + EXTRACT(year FROM `t0`.`i`) AS `ExtractYear(i)` + FROM `table` AS `t0` +) AS `t1` +GROUP BY + `t1`.`ExtractYear(i)` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_window_aggregation/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_window_aggregation/out.sql index 9867d27155f14..7c1d4b2455cc0 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_window_aggregation/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_window_aggregation/out.sql @@ -1,3 +1,14 @@ -SELECT t0.`window_start`, t0.`window_end`, t0.`g`, avg(t0.`d`) AS `mean` -FROM TABLE(TUMBLE(TABLE `table`, DESCRIPTOR(`i`), INTERVAL '15' MINUTE)) t0 -GROUP BY t0.`window_start`, t0.`window_end`, t0.`g` \ No newline at end of file +SELECT + `t1`.`window_start`, + `t1`.`window_end`, + `t1`.`g`, + AVG(`t1`.`d`) AS `mean` +FROM ( + SELECT + `t0`.* + FROM TABLE(TUMBLE(TABLE `table`, DESCRIPTOR(`t0`.`i`), INTERVAL '15' MINUTE)) AS `t0` +) AS `t1` +GROUP BY + `t1`.`window_start`, + `t1`.`window_end`, + `t1`.`g` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_window_topn/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_window_topn/out.sql index 92d4e692424ff..6724d7ef0bfaa 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_window_topn/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_window_topn/out.sql @@ -1,12 +1,37 @@ -WITH t0 AS ( - SELECT t2.`a`, t2.`b`, t2.`c`, t2.`d`, t2.`g`, t2.`window_start`, - t2.`window_end` - FROM TABLE(TUMBLE(TABLE `table`, DESCRIPTOR(`i`), INTERVAL '10' MINUTE)) t2 -) -SELECT t1.* +SELECT + `t3`.`a`, + `t3`.`b`, + `t3`.`c`, + `t3`.`d`, + `t3`.`g`, + `t3`.`window_start`, + `t3`.`window_end`, + `t3`.`rownum` FROM ( - SELECT t0.*, - (row_number() OVER (PARTITION BY t0.`window_start`, t0.`window_end` ORDER BY t0.`g` DESC) - 1) AS `rownum` - FROM t0 -) t1 -WHERE t1.`rownum` <= CAST(3 AS TINYINT) \ No newline at end of file + SELECT + `t2`.`a`, + `t2`.`b`, + `t2`.`c`, + `t2`.`d`, + `t2`.`g`, + `t2`.`window_start`, + `t2`.`window_end`, + ROW_NUMBER() OVER (PARTITION BY `t2`.`window_start`, `t2`.`window_end` ORDER BY `t2`.`g` DESC ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) - 1 AS `rownum` + FROM ( + SELECT + `t1`.`a`, + `t1`.`b`, + `t1`.`c`, + `t1`.`d`, + `t1`.`g`, + `t1`.`window_start`, + `t1`.`window_end` + FROM ( + SELECT + `t0`.* + FROM TABLE(TUMBLE(TABLE `table`, DESCRIPTOR(`t0`.`i`), INTERVAL '600' SECOND)) AS `t0` + ) AS `t1` + ) AS `t2` +) AS `t3` +WHERE + `t3`.`rownum` <= 3 \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_windowing_tvf/cumulate_window/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_windowing_tvf/cumulate_window/out.sql index 522c6d576e1ea..d94117e139164 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_windowing_tvf/cumulate_window/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_windowing_tvf/cumulate_window/out.sql @@ -1,2 +1,5 @@ -SELECT t0.* -FROM TABLE(CUMULATE(TABLE `table`, DESCRIPTOR(`i`), INTERVAL '10' SECOND, INTERVAL '1' MINUTE)) t0 \ No newline at end of file +SELECT + `t0`.* +FROM TABLE( + CUMULATE(TABLE `table`, DESCRIPTOR(`t0`.`i`), INTERVAL '10' SECOND, INTERVAL '1' MINUTE) +) AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_windowing_tvf/hop_window/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_windowing_tvf/hop_window/out.sql index 38376568cba92..50f00672e4058 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_windowing_tvf/hop_window/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_windowing_tvf/hop_window/out.sql @@ -1,2 +1,5 @@ -SELECT t0.* -FROM TABLE(HOP(TABLE `table`, DESCRIPTOR(`i`), INTERVAL '1' MINUTE, INTERVAL '15' MINUTE)) t0 \ No newline at end of file +SELECT + `t0`.* +FROM TABLE( + HOP(TABLE `table`, DESCRIPTOR(`t0`.`i`), INTERVAL '1' MINUTE, INTERVAL '15' MINUTE) +) AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_compiler/test_windowing_tvf/tumble_window/out.sql b/ibis/backends/flink/tests/snapshots/test_compiler/test_windowing_tvf/tumble_window/out.sql index d46ca32d3d239..95d55e1c41dc0 100644 --- a/ibis/backends/flink/tests/snapshots/test_compiler/test_windowing_tvf/tumble_window/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_compiler/test_windowing_tvf/tumble_window/out.sql @@ -1,2 +1,3 @@ -SELECT t0.* -FROM TABLE(TUMBLE(TABLE `table`, DESCRIPTOR(`i`), INTERVAL '15' MINUTE)) t0 \ No newline at end of file +SELECT + `t0`.* +FROM TABLE(TUMBLE(TABLE `table`, DESCRIPTOR(`t0`.`i`), INTERVAL '15' MINUTE)) AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/datetime/out.sql b/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/datetime/out.sql index 8f93e34a40ddc..bb447abd40a1d 100644 --- a/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/datetime/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/datetime/out.sql @@ -1 +1,2 @@ -TIMESTAMP '2017-01-01 04:55:59' \ No newline at end of file +SELECT + CAST('2017-01-01 04:55:59' AS TIMESTAMP) AS `datetime.datetime(2017, 1, 1, 4, 55, 59)` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/datetime_with_microseconds/out.sql b/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/datetime_with_microseconds/out.sql index 9260f37f43e61..d5c8151660fe6 100644 --- a/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/datetime_with_microseconds/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/datetime_with_microseconds/out.sql @@ -1 +1,2 @@ -TIMESTAMP '2017-01-01 04:55:59.001122' \ No newline at end of file +SELECT + CAST('2017-01-01 04:55:59.001122' AS TIMESTAMP) AS `datetime.datetime(2017, 1, 1, 4, 55, 59, 1122)` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/string_time/out.sql b/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/string_time/out.sql index b3976577078cb..8b62e47adfdbc 100644 --- a/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/string_time/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/string_time/out.sql @@ -1 +1,2 @@ -TIME '04:55:59' \ No newline at end of file +SELECT + CAST('04:55:59' AS TIMESTAMP) AS `datetime.time(4, 55, 59)` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/string_timestamp/out.sql b/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/string_timestamp/out.sql index 8f93e34a40ddc..bb447abd40a1d 100644 --- a/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/string_timestamp/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/string_timestamp/out.sql @@ -1 +1,2 @@ -TIMESTAMP '2017-01-01 04:55:59' \ No newline at end of file +SELECT + CAST('2017-01-01 04:55:59' AS TIMESTAMP) AS `datetime.datetime(2017, 1, 1, 4, 55, 59)` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/time/out.sql b/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/time/out.sql index b3976577078cb..8b62e47adfdbc 100644 --- a/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/time/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/time/out.sql @@ -1 +1,2 @@ -TIME '04:55:59' \ No newline at end of file +SELECT + CAST('04:55:59' AS TIMESTAMP) AS `datetime.time(4, 55, 59)` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/timestamp/out.sql b/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/timestamp/out.sql index 8f93e34a40ddc..bb447abd40a1d 100644 --- a/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/timestamp/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_literals/test_literal_timestamp_or_time/timestamp/out.sql @@ -1 +1,2 @@ -TIMESTAMP '2017-01-01 04:55:59' \ No newline at end of file +SELECT + CAST('2017-01-01 04:55:59' AS TIMESTAMP) AS `datetime.datetime(2017, 1, 1, 4, 55, 59)` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_window/test_range_window/out.sql b/ibis/backends/flink/tests/snapshots/test_window/test_range_window/out.sql index b43021ac30d0b..a739ff47435ab 100644 --- a/ibis/backends/flink/tests/snapshots/test_window/test_range_window/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_window/test_range_window/out.sql @@ -1,2 +1,3 @@ -SELECT sum(t0.`f`) OVER (ORDER BY t0.`f` ASC RANGE BETWEEN INTERVAL '00 08:20:00.000000' DAY TO SECOND PRECEDING AND CURRENT ROW) AS `Sum(f)` -FROM table t0 \ No newline at end of file +SELECT + SUM(`t0`.`f`) OVER (ORDER BY `t0`.`f` ASC NULLS LAST RANGE BETWEEN INTERVAL '500' MINUTE preceding AND CAST(0 AS INTERVAL MINUTE) following) AS `Sum(f)` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/snapshots/test_window/test_rows_window/out.sql b/ibis/backends/flink/tests/snapshots/test_window/test_rows_window/out.sql index 3173072eafb64..7b5bb83d8c921 100644 --- a/ibis/backends/flink/tests/snapshots/test_window/test_rows_window/out.sql +++ b/ibis/backends/flink/tests/snapshots/test_window/test_rows_window/out.sql @@ -1,2 +1,3 @@ -SELECT sum(t0.`f`) OVER (ORDER BY t0.`f` ASC ROWS BETWEEN CAST(1000 AS SMALLINT) PRECEDING AND CURRENT ROW) AS `Sum(f)` -FROM table t0 \ No newline at end of file +SELECT + SUM(`t0`.`f`) OVER (ORDER BY `t0`.`f` ASC NULLS LAST ROWS BETWEEN 1000 preceding AND CAST(0 AS SMALLINT) following) AS `Sum(f)` +FROM `table` AS `t0` \ No newline at end of file diff --git a/ibis/backends/flink/tests/test_ddl.py b/ibis/backends/flink/tests/test_ddl.py index d1e741e6eb095..2ea8e7e53454e 100644 --- a/ibis/backends/flink/tests/test_ddl.py +++ b/ibis/backends/flink/tests/test_ddl.py @@ -279,14 +279,16 @@ def test_create_table_failure_with_invalid_primary_keys( assert temp_table not in con.list_tables() +@pytest.fixture +def temp_view(con): + name = ibis.util.gen_name("view") + yield name + con.drop_view(name, force=True) + + @pytest.mark.parametrize("temp", [True, False]) def test_create_view( - con, - temp_table: str, - awards_players_schema: sch.Schema, - csv_source_configs, - temp_view: str, - temp, + con, temp_table, awards_players_schema, csv_source_configs, temp_view, temp ): table = con.create_table( name=temp_table, @@ -497,11 +499,6 @@ def test_read_json(con, data_dir, tmp_path, table_name, functional_alltypes_sche assert table_name not in con.list_tables() -@pytest.fixture(scope="module") -def functional_alltypes(con): - return con.table("functional_alltypes") - - @pytest.mark.parametrize( "table_name", [ diff --git a/ibis/backends/flink/tests/test_literals.py b/ibis/backends/flink/tests/test_literals.py deleted file mode 100644 index 091405d142d02..0000000000000 --- a/ibis/backends/flink/tests/test_literals.py +++ /dev/null @@ -1,80 +0,0 @@ -from __future__ import annotations - -import datetime - -import pandas as pd -import pytest -from pytest import param - -import ibis -import ibis.expr.datatypes as dt - - -@pytest.mark.parametrize( - "value,expected", - [ - param(5, "CAST(5 AS TINYINT)", id="int"), - param(1.5, "CAST(1.5 AS DOUBLE)", id="float"), - param(True, "TRUE", id="true"), - param(False, "FALSE", id="false"), - ], -) -def test_simple_literals(con, value, expected): - expr = ibis.literal(value) - result = con.compile(expr) - assert result == expected - - -@pytest.mark.parametrize( - "value,expected", - [ - param("simple", "'simple'", id="simple"), - param("I can't", "'I can''t'", id="nested_quote"), - param('An "escape"', """'An "escape"'""", id="nested_token"), - ], -) -def test_string_literals(con, value, expected): - expr = ibis.literal(value) - result = con.compile(expr) - assert result == expected - - -@pytest.mark.parametrize( - "value,expected", - [ - param( - datetime.timedelta(seconds=70), - "INTERVAL '00 00:01:10.000000' DAY TO SECOND", - id="70seconds", - ), - param( - ibis.interval(months=50), "INTERVAL '04-02' YEAR TO MONTH", id="50months" - ), - param(ibis.interval(seconds=5), "INTERVAL '5' SECOND", id="5seconds"), - ], -) -def test_translate_interval_literal(con, value, expected): - expr = ibis.literal(value) - result = con.compile(expr) - assert result == expected - - -@pytest.mark.parametrize( - ("case", "dtype"), - [ - param(datetime.datetime(2017, 1, 1, 4, 55, 59), dt.timestamp, id="datetime"), - param( - datetime.datetime(2017, 1, 1, 4, 55, 59, 1122), - dt.timestamp, - id="datetime_with_microseconds", - ), - param("2017-01-01 04:55:59", dt.timestamp, id="string_timestamp"), - param(pd.Timestamp("2017-01-01 04:55:59"), dt.timestamp, id="timestamp"), - param(datetime.time(4, 55, 59), dt.time, id="time"), - param("04:55:59", dt.time, id="string_time"), - ], -) -def test_literal_timestamp_or_time(con, snapshot, case, dtype): - expr = ibis.literal(case, type=dtype) - result = con.compile(expr) - snapshot.assert_match(result, "out.sql") diff --git a/ibis/backends/flink/tests/test_window.py b/ibis/backends/flink/tests/test_window.py index ca67a317fa2eb..efb459bcf147b 100644 --- a/ibis/backends/flink/tests/test_window.py +++ b/ibis/backends/flink/tests/test_window.py @@ -1,18 +1,17 @@ from __future__ import annotations import pytest +from pyflink.util.exceptions import TableException from pytest import param import ibis +from ibis.backends.tests.errors import Py4JJavaError from ibis.common.exceptions import UnsupportedOperationError def test_window_requires_order_by(con, simple_table): expr = simple_table.mutate(simple_table.c - simple_table.c.mean()) - with pytest.raises( - UnsupportedOperationError, - match="Flink engine does not support generic window clause with no order by", - ): + with pytest.raises(UnsupportedOperationError): con.compile(expr) @@ -22,42 +21,39 @@ def test_window_does_not_support_multiple_order_by(con, simple_table): group_by=[simple_table.g, simple_table.a], order_by=[simple_table.f, simple_table.d], ) - with pytest.raises( - UnsupportedOperationError, - match="Windows in Flink can only be ordered by a single time column", - ): + with pytest.raises(UnsupportedOperationError): con.compile(expr) @pytest.mark.parametrize( - ("window", "err"), + "window", [ param( {"rows": (-1, 1)}, - "OVER RANGE FOLLOWING windows are not supported in Flink yet", id="bounded_rows_following", + marks=[pytest.mark.xfail(raises=TableException)], ), param( {"rows": (-1, None)}, - "OVER RANGE FOLLOWING windows are not supported in Flink yet", id="unbounded_rows_following", + marks=[pytest.mark.xfail(raises=TableException)], ), param( {"rows": (-500, 1)}, - "OVER RANGE FOLLOWING windows are not supported in Flink yet", id="casted_bounded_rows_following", + marks=[pytest.mark.xfail(raises=TableException)], ), param( {"range": (-1000, 0)}, - "Data Type mismatch between ORDER BY and RANGE clause", id="int_range", + marks=[pytest.mark.xfail(raises=Py4JJavaError)], ), ], ) -def test_window_invalid_start_end(con, simple_table, window, err): - expr = simple_table.f.sum().over(**window, order_by=simple_table.f) - with pytest.raises(UnsupportedOperationError, match=err): - con.compile(expr) +def test_window_invalid_start_end(con, window): + t = con.tables.functional_alltypes + expr = t.int_col.sum().over(**window, order_by=t.timestamp_col) + con.execute(expr) def test_range_window(con, snapshot, simple_table): diff --git a/ibis/backends/flink/translator.py b/ibis/backends/flink/translator.py deleted file mode 100644 index 37bbcc0170ab7..0000000000000 --- a/ibis/backends/flink/translator.py +++ /dev/null @@ -1,16 +0,0 @@ -from __future__ import annotations - -import ibis.expr.operations as ops -from ibis.backends.base.sql.compiler import ExprTranslator -from ibis.backends.flink.registry import operation_registry - - -class FlinkExprTranslator(ExprTranslator): - _dialect_name = "hive" # TODO: make a custom sqlglot dialect for Flink - _registry = operation_registry - _bool_aggs_need_cast_to_int32 = True - - -@FlinkExprTranslator.rewrites(ops.Clip) -def _clip_no_op(op): - return op diff --git a/ibis/backends/tests/snapshots/test_generic/test_many_subqueries/flink/out.sql b/ibis/backends/tests/snapshots/test_generic/test_many_subqueries/flink/out.sql new file mode 100644 index 0000000000000..831c5245665ad --- /dev/null +++ b/ibis/backends/tests/snapshots/test_generic/test_many_subqueries/flink/out.sql @@ -0,0 +1,42 @@ +WITH `t6` AS ( + SELECT + `t5`.`street`, + ROW_NUMBER() OVER (ORDER BY `t5`.`street` ASC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) - 1 AS `key` + FROM ( + SELECT + `t2`.`street`, + `t2`.`key` + FROM ( + SELECT + `t0`.`street`, + ROW_NUMBER() OVER (ORDER BY `t0`.`street` ASC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) - 1 AS `key` + FROM `data` AS `t0` + ) AS `t2` + INNER JOIN ( + SELECT + `t1`.`key` + FROM ( + SELECT + `t0`.`street`, + ROW_NUMBER() OVER (ORDER BY `t0`.`street` ASC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) - 1 AS `key` + FROM `data` AS `t0` + ) AS `t1` + ) AS `t4` + ON `t2`.`key` = `t4`.`key` + ) AS `t5` +), `t1` AS ( + SELECT + `t0`.`street`, + ROW_NUMBER() OVER (ORDER BY `t0`.`street` ASC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) - 1 AS `key` + FROM `data` AS `t0` +) +SELECT + `t8`.`street`, + `t8`.`key` +FROM `t6` AS `t8` +INNER JOIN ( + SELECT + `t7`.`key` + FROM `t6` AS `t7` +) AS `t10` + ON `t8`.`key` = `t10`.`key` \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_interactive/test_default_limit/flink/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_default_limit/flink/out.sql new file mode 100644 index 0000000000000..f63de03c314af --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_default_limit/flink/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/flink/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_disable_query_limit/flink/out.sql new file mode 100644 index 0000000000000..f63de03c314af --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_disable_query_limit/flink/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/flink/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_interactive_execute_on_repr/flink/out.sql new file mode 100644 index 0000000000000..d8a9c4090dc11 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_interactive_execute_on_repr/flink/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/flink/out.sql b/ibis/backends/tests/snapshots/test_interactive/test_respect_set_limit/flink/out.sql new file mode 100644 index 0000000000000..d4b1b19815b09 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_interactive/test_respect_set_limit/flink/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_cte_refs_in_topo_order/flink/out.sql b/ibis/backends/tests/snapshots/test_sql/test_cte_refs_in_topo_order/flink/out.sql new file mode 100644 index 0000000000000..8d13c9ddda1be --- /dev/null +++ b/ibis/backends/tests/snapshots/test_sql/test_cte_refs_in_topo_order/flink/out.sql @@ -0,0 +1,20 @@ +WITH `t1` AS ( + SELECT + `t0`.`key` + FROM `leaf` AS `t0` + WHERE + TRUE +) +SELECT + `t3`.`key` +FROM `t1` AS `t3` +INNER JOIN `t1` AS `t4` + ON `t3`.`key` = `t4`.`key` +INNER JOIN ( + SELECT + `t3`.`key` + FROM `t1` AS `t3` + INNER JOIN `t1` AS `t4` + ON `t3`.`key` = `t4`.`key` +) AS `t6` + ON `t3`.`key` = `t6`.`key` \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/flink/out.sql b/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/flink/out.sql new file mode 100644 index 0000000000000..98e8ba8a8c2f0 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_sql/test_group_by_has_index/flink/out.sql @@ -0,0 +1,38 @@ +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 + 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 \ No newline at end of file diff --git a/ibis/backends/tests/snapshots/test_sql/test_isin_bug/flink/out.sql b/ibis/backends/tests/snapshots/test_sql/test_isin_bug/flink/out.sql new file mode 100644 index 0000000000000..db5ddb124e868 --- /dev/null +++ b/ibis/backends/tests/snapshots/test_sql/test_isin_bug/flink/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/test_aggregation.py b/ibis/backends/tests/test_aggregation.py index 66988e15bb633..b1decb2df0f20 100644 --- a/ibis/backends/tests/test_aggregation.py +++ b/ibis/backends/tests/test_aggregation.py @@ -481,25 +481,37 @@ def mean_and_std(v): lambda t, where: t.double_col.std(how="sample", where=where), lambda t, where: t.double_col[where].std(ddof=1), id="std", - marks=[pytest.mark.notimpl(["druid"], raises=com.OperationNotDefinedError)], + marks=[ + pytest.mark.notimpl(["druid"], raises=com.OperationNotDefinedError), + pytest.mark.notyet(["flink"], raises=AssertionError, strict=False), + ], ), param( lambda t, where: t.double_col.var(how="sample", where=where), lambda t, where: t.double_col[where].var(ddof=1), id="var", - marks=[pytest.mark.notimpl(["druid"], raises=com.OperationNotDefinedError)], + marks=[ + pytest.mark.notimpl(["druid"], raises=com.OperationNotDefinedError), + pytest.mark.notyet(["flink"], raises=AssertionError, strict=False), + ], ), param( lambda t, where: t.double_col.std(how="pop", where=where), lambda t, where: t.double_col[where].std(ddof=0), id="std_pop", - marks=[pytest.mark.notimpl(["druid"], raises=com.OperationNotDefinedError)], + marks=[ + pytest.mark.notimpl(["druid"], raises=com.OperationNotDefinedError), + pytest.mark.notyet(["flink"], raises=AssertionError, strict=False), + ], ), param( lambda t, where: t.double_col.var(how="pop", where=where), lambda t, where: t.double_col[where].var(ddof=0), id="var_pop", - marks=[pytest.mark.notimpl(["druid"], raises=com.OperationNotDefinedError)], + marks=[ + pytest.mark.notimpl(["druid"], raises=com.OperationNotDefinedError), + pytest.mark.notyet(["flink"], raises=AssertionError, strict=False), + ], ), param( lambda t, where: t.string_col.approx_nunique(where=where), @@ -637,13 +649,10 @@ def mean_and_std(v): id="first", marks=[ pytest.mark.notimpl( - ["druid", "impala", "mssql", "mysql", "oracle", "flink"], + ["dask", "druid", "impala", "mssql", "mysql", "oracle"], raises=com.OperationNotDefinedError, ), - pytest.mark.notimpl( - ["risingwave"], - raises=PsycoPg2InternalError, - ), + pytest.mark.notimpl(["risingwave"], raises=PsycoPg2InternalError), ], ), param( @@ -652,13 +661,10 @@ def mean_and_std(v): id="last", marks=[ pytest.mark.notimpl( - ["druid", "impala", "mssql", "mysql", "oracle", "flink"], + ["dask", "druid", "impala", "mssql", "mysql", "oracle"], raises=com.OperationNotDefinedError, ), - pytest.mark.notimpl( - ["risingwave"], - raises=PsycoPg2InternalError, - ), + pytest.mark.notimpl(["risingwave"], raises=PsycoPg2InternalError), ], ), param( @@ -1048,7 +1054,7 @@ def test_quantile( reason="backend only implements population correlation coefficient", ), pytest.mark.notyet( - ["impala", "mysql", "sqlite"], + ["impala", "mysql", "sqlite", "flink"], raises=com.OperationNotDefinedError, ), pytest.mark.notyet( @@ -1573,7 +1579,12 @@ def test_grouped_case(backend, con): @pytest.mark.notimpl(["datafusion", "polars"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl(["exasol"], raises=ExaQueryError) -@pytest.mark.notyet(["flink"], raises=com.UnsupportedOperationError) +@pytest.mark.broken( + ["dask"], + reason="Dask does not windowize this operation correctly", + raises=AssertionError, +) +@pytest.mark.notyet(["flink"], raises=Py4JError) @pytest.mark.notyet(["impala"], raises=ImpalaHiveServer2Error) @pytest.mark.notyet(["clickhouse"], raises=ClickHouseDatabaseError) @pytest.mark.notyet(["druid"], raises=PyDruidProgrammingError) diff --git a/ibis/backends/tests/test_array.py b/ibis/backends/tests/test_array.py index 2d203c2c9f926..df9c0a973ceae 100644 --- a/ibis/backends/tests/test_array.py +++ b/ibis/backends/tests/test_array.py @@ -116,8 +116,6 @@ def test_array_repeat(con): assert np.array_equal(result, expected) -# Issues #2370 -@pytest.mark.notimpl(["flink"], raises=com.OperationNotDefinedError) def test_array_concat(con): left = ibis.literal([1, 2, 3]) right = ibis.literal([2, 1]) @@ -126,8 +124,6 @@ def test_array_concat(con): assert sorted(result) == sorted([1, 2, 3, 2, 1]) -# Issues #2370 -@pytest.mark.notimpl(["flink"], raises=com.OperationNotDefinedError) def test_array_concat_variadic(con): left = ibis.literal([1, 2, 3]) right = ibis.literal([2, 1]) @@ -138,7 +134,7 @@ def test_array_concat_variadic(con): # Issues #2370 -@pytest.mark.notimpl(["flink"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl(["flink"], raises=Py4JJavaError) @pytest.mark.notyet(["trino"], raises=TrinoUserError) def test_array_concat_some_empty(con): left = ibis.literal([]) @@ -149,7 +145,6 @@ def test_array_concat_some_empty(con): assert np.array_equal(result, expected) -@pytest.mark.notimpl(["flink"], raises=com.OperationNotDefinedError) def test_array_radd_concat(con): left = [1] right = ibis.literal([2]) @@ -250,7 +245,7 @@ def test_array_discovery(backend): reason="BigQuery doesn't support casting array to array", raises=GoogleBadRequest, ) -@pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl(["datafusion", "flink"], raises=com.OperationNotDefinedError) def test_unnest_simple(backend): array_types = backend.array_types expected = ( @@ -361,7 +356,7 @@ def test_unnest_no_nulls(backend): raises=ValueError, reason="all the input arrays must have same number of dimensions", ) -@pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl(["datafusion", "flink"], raises=com.OperationNotDefinedError) def test_unnest_default_name(backend): array_types = backend.array_types df = array_types.execute() @@ -547,6 +542,9 @@ def test_array_filter(con, input, output): @builtin_array @pytest.mark.notimpl(["polars"], raises=com.OperationNotDefinedError) +@pytest.mark.broken( + ["flink"], raises=Py4JJavaError, reason="NullPointerException raised on execution" +) def test_array_contains(backend, con): t = backend.array_types expr = t.x.contains(1) @@ -643,9 +641,10 @@ def test_array_remove(con, a): reason="bigquery doesn't support null elements in arrays", ) @pytest.mark.broken( - ["risingwave"], - raises=AssertionError, - reason="TODO(Kexiang): seems a bug", + ["risingwave"], raises=AssertionError, reason="TODO(Kexiang): seems a bug" +) +@pytest.mark.notyet( + ["flink"], raises=Py4JJavaError, reason="empty arrays not supported" ) @pytest.mark.parametrize( ("input", "expected"), @@ -662,9 +661,6 @@ def test_array_remove(con, a): ), ], ) -@pytest.mark.notimpl( - ["flink"], raises=NotImplementedError, reason="`from_ibis()` is not implemented" -) def test_array_unique(con, input, expected): t = ibis.memtable(input) expr = t.a.unique() @@ -789,7 +785,7 @@ def test_array_intersect(con, data): ) @pytest.mark.notimpl(["postgres"], raises=PsycoPg2SyntaxError) @pytest.mark.notimpl(["risingwave"], raises=PsycoPg2InternalError) -@pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl(["datafusion", "flink"], raises=com.OperationNotDefinedError) @pytest.mark.broken( ["trino"], reason="inserting maps into structs doesn't work", raises=TrinoUserError ) @@ -815,6 +811,7 @@ def test_unnest_struct(con): "polars", "postgres", "risingwave", + "flink", ], raises=com.OperationNotDefinedError, ) @@ -844,7 +841,7 @@ def test_zip(backend): ) @pytest.mark.notimpl(["postgres"], raises=PsycoPg2SyntaxError) @pytest.mark.notimpl(["risingwave"], raises=PsycoPg2ProgrammingError) -@pytest.mark.notimpl(["datafusion"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl(["datafusion", "flink"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl( ["polars"], raises=com.OperationNotDefinedError, diff --git a/ibis/backends/tests/test_asof_join.py b/ibis/backends/tests/test_asof_join.py index 2a1901efc520b..5f2d1ac067c26 100644 --- a/ibis/backends/tests/test_asof_join.py +++ b/ibis/backends/tests/test_asof_join.py @@ -96,6 +96,7 @@ def time_keyed_right(time_keyed_df2): "mssql", "sqlite", "risingwave", + "flink", ] ) def test_asof_join(con, time_left, time_right, time_df1, time_df2, direction, op): @@ -137,6 +138,7 @@ def test_asof_join(con, time_left, time_right, time_df1, time_df2, direction, op "mssql", "sqlite", "risingwave", + "flink", ] ) def test_keyed_asof_join_with_tolerance( diff --git a/ibis/backends/tests/test_client.py b/ibis/backends/tests/test_client.py index 3339a2c250505..bd7f1cb1e2ac1 100644 --- a/ibis/backends/tests/test_client.py +++ b/ibis/backends/tests/test_client.py @@ -30,7 +30,6 @@ OracleDatabaseError, PsycoPg2InternalError, PsycoPg2UndefinedObject, - Py4JJavaError, PyODBCProgrammingError, SnowflakeProgrammingError, TrinoUserError, @@ -49,6 +48,10 @@ def new_schema(): def _create_temp_table_with_schema(backend, con, temp_table_name, schema, data=None): if con.name == "druid": pytest.xfail("druid doesn't implement create_table") + elif con.name == "flink": + pytest.xfail( + "flink doesn't implement create_table from schema without additional arguments" + ) temporary = con.create_table(temp_table_name, schema=schema) assert temporary.to_pandas().empty @@ -128,14 +131,7 @@ def test_create_table(backend, con, temp_table, func, sch): ], ), param( - False, - True, - marks=[ - pytest.mark.notyet( - ["polars"], raises=com.IbisError, reason="all tables are ephemeral" - ) - ], - id="no temp, overwrite", + False, True, id="no temp, overwrite", marks=pytest.mark.notyet(["flink"]) ), param( True, @@ -182,6 +178,7 @@ def test_create_table_overwrite_temp(backend, con, temp_table, temp, overwrite): ids=["dataframe", "pyarrow table"], ) @pytest.mark.notyet(["druid"], raises=NotImplementedError) +@pytest.mark.notyet(["flink"], raises=com.IbisError) def test_load_data(backend, con, temp_table, lamduh): sch = ibis.schema( [ @@ -221,14 +218,6 @@ def test_load_data(backend, con, temp_table, lamduh): ), ], ) -@pytest.mark.broken( - ["flink"], - raises=Py4JJavaError, - reason=( - "org.apache.flink.table.api.ValidationException: " - "Table `default_catalog`.`default_database`.`functional_alltypes` was not found." - ), -) def test_query_schema(ddl_backend, expr_fn, expected): expr = expr_fn(ddl_backend.functional_alltypes) @@ -250,9 +239,6 @@ def test_query_schema(ddl_backend, expr_fn, expected): @pytest.mark.notimpl(["datafusion", "mssql"]) @pytest.mark.never(["dask", "pandas"], reason="dask and pandas do not support SQL") -@pytest.mark.notimpl( - ["flink"], raises=AttributeError, reason="'Backend' object has no attribute 'sql'" -) def test_sql(backend, con): # execute the expression using SQL query table = backend.format_table("functional_alltypes") @@ -316,6 +302,11 @@ def test_create_table_from_schema(con, new_schema, temp_table): raises=PsycoPg2InternalError, reason="truncate not supported upstream", ) +@pytest.mark.notimpl( + ["flink"], + raises=com.IbisError, + reason="`tbl_properties` is required when creating table with schema", +) def test_create_temporary_table_from_schema(con_no_data, new_schema): temp_table = gen_name(f"test_{con_no_data.name}_tmp") table = con_no_data.create_table(temp_table, schema=new_schema, temp=True) @@ -398,6 +389,7 @@ def test_nullable_input_output(con, temp_table): @pytest.mark.broken( ["flink"], raises=ValueError, + strict=False, reason=( "table `FUNCTIONAL_ALLTYPES` does not exist" "Note (mehmet): Not raised when only this test function is executed, " @@ -609,6 +601,11 @@ def _emp(a, b, c, d): reason="`insert` method not implemented", ) @pytest.mark.notyet(["druid"], raises=NotImplementedError) +@pytest.mark.notimpl( + ["flink"], + raises=com.IbisError, + reason="`tbl_properties` is required when creating table with schema", +) def test_insert_from_memtable(con, temp_table): df = pd.DataFrame({"x": range(3)}) table_name = temp_table @@ -644,6 +641,7 @@ def test_list_databases(con): "pyspark": set(), "sqlite": {"main"}, "trino": {"memory"}, + "flink": set(), } result = set(con.list_databases()) assert test_databases[con.name] <= result @@ -672,6 +670,11 @@ def test_list_databases(con): raises=PsycoPg2InternalError, reason="unsigned integers are not supported", ) +@pytest.mark.notimpl( + ["flink"], + raises=com.IbisError, + reason="`tbl_properties` is required when creating table with schema", +) def test_unsigned_integer_type(con, temp_table): con.create_table( temp_table, diff --git a/ibis/backends/tests/test_dot_sql.py b/ibis/backends/tests/test_dot_sql.py index 26d3936e47561..c8f95f5a5d7d8 100644 --- a/ibis/backends/tests/test_dot_sql.py +++ b/ibis/backends/tests/test_dot_sql.py @@ -30,7 +30,6 @@ PolarsComputeError, ) -dot_sql_notimpl = pytest.mark.notimpl(["flink"]) dot_sql_never = pytest.mark.never( ["dask", "pandas"], reason="dask and pandas do not accept SQL" ) @@ -41,7 +40,6 @@ } -@pytest.mark.notimpl(["flink"]) @pytest.mark.notyet(["oracle"], reason="table quoting behavior") @dot_sql_never @pytest.mark.parametrize( @@ -94,7 +92,6 @@ def test_con_dot_sql(backend, con, schema): @pytest.mark.notyet( ["druid"], raises=com.IbisTypeError, reason="druid does not preserve case" ) -@dot_sql_notimpl @dot_sql_never def test_table_dot_sql(backend): alltypes = backend.functional_alltypes @@ -142,7 +139,6 @@ def test_table_dot_sql(backend): OracleDatabaseError, reason="oracle doesn't know which of the tables in the join to sort from", ) -@dot_sql_notimpl @dot_sql_never def test_table_dot_sql_with_join(backend): alltypes = backend.functional_alltypes @@ -194,7 +190,6 @@ def test_table_dot_sql_with_join(backend): @pytest.mark.notyet( ["bigquery"], raises=GoogleBadRequest, reason="requires a qualified name" ) -@dot_sql_notimpl @dot_sql_never def test_table_dot_sql_repr(backend): alltypes = backend.functional_alltypes @@ -220,7 +215,6 @@ def test_table_dot_sql_repr(backend): assert repr(t) -@dot_sql_notimpl @dot_sql_never def test_dot_sql_alias_with_params(backend, alltypes, df): t = alltypes @@ -230,7 +224,6 @@ def test_dot_sql_alias_with_params(backend, alltypes, df): backend.assert_series_equal(result.x, expected) -@dot_sql_notimpl @dot_sql_never def test_dot_sql_reuse_alias_with_different_types(backend, alltypes, df): foo1 = alltypes.select(x=alltypes.string_col).alias("foo") @@ -241,7 +234,7 @@ def test_dot_sql_reuse_alias_with_different_types(backend, alltypes, df): backend.assert_series_equal(foo2.x.execute(), expected2) -_NO_SQLGLOT_DIALECT = {"pandas", "dask", "druid", "flink"} +_NO_SQLGLOT_DIALECT = {"pandas", "dask", "druid"} no_sqlglot_dialect = sorted( # TODO(cpcloud): remove the strict=False hack once backends are ported to # sqlglot @@ -255,7 +248,6 @@ def test_dot_sql_reuse_alias_with_different_types(backend, alltypes, df): [*sorted(_get_backend_names() - _NO_SQLGLOT_DIALECT), *no_sqlglot_dialect], ) @pytest.mark.notyet(["polars"], raises=PolarsComputeError) -@dot_sql_notimpl @dot_sql_never @pytest.mark.notyet(["druid"], reason="druid doesn't respect column name case") def test_table_dot_sql_transpile(backend, alltypes, dialect, df): @@ -280,7 +272,6 @@ def test_table_dot_sql_transpile(backend, alltypes, dialect, df): ["druid"], raises=AttributeError, reason="druid doesn't respect column names" ) @pytest.mark.notyet(["bigquery"]) -@dot_sql_notimpl @dot_sql_never def test_con_dot_sql_transpile(backend, con, dialect, df): t = sg.table("functional_alltypes", quoted=True) @@ -294,9 +285,8 @@ def test_con_dot_sql_transpile(backend, con, dialect, df): backend.assert_series_equal(result.x, expected) -@dot_sql_notimpl @dot_sql_never -@pytest.mark.notimpl(["druid", "flink", "polars", "exasol"]) +@pytest.mark.notimpl(["druid", "polars"]) def test_order_by_no_projection(backend): con = backend.connection expr = ( @@ -309,7 +299,6 @@ def test_order_by_no_projection(backend): assert set(result) == {"Ross, Jerry L.", "Chang-Diaz, Franklin R."} -@dot_sql_notimpl @dot_sql_never @pytest.mark.notyet(["polars"], raises=PolarsComputeError) def test_dot_sql_limit(con): @@ -328,13 +317,15 @@ def mem_t(con): pytest.xfail("druid does not support create_table") name = ibis.util.gen_name(con.name) - con.create_table(name, ibis.memtable({"a": list("def")})) + + # flink only supports memtables if `temp` is True, seems like we should + # address that for users + con.create_table(name, ibis.memtable({"a": list("def")}), temp=con.name == "flink") yield name with contextlib.suppress(NotImplementedError): con.drop_table(name, force=True) -@dot_sql_notimpl @dot_sql_never @pytest.mark.notyet(["polars"], raises=PolarsComputeError) def test_cte(con, mem_t): diff --git a/ibis/backends/tests/test_examples.py b/ibis/backends/tests/test_examples.py index 845a20e3b3df5..cfcd893cfb1de 100644 --- a/ibis/backends/tests/test_examples.py +++ b/ibis/backends/tests/test_examples.py @@ -15,7 +15,7 @@ (LINUX or MACOS) and SANDBOXED, reason="nix on linux cannot download duckdb extensions or data due to sandboxing", ) -@pytest.mark.notimpl(["pyspark", "flink", "exasol"]) +@pytest.mark.notimpl(["pyspark", "exasol"]) @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 02eefb296c3a8..03ee2ccebe01f 100644 --- a/ibis/backends/tests/test_export.py +++ b/ibis/backends/tests/test_export.py @@ -29,16 +29,8 @@ param( 42, id="limit", - marks=[ - pytest.mark.notimpl( - [ - # limit not implemented for flink and pandas backend execution - "dask", - "pandas", - "flink", - ] - ), - ], + # limit not implemented for pandas-family backends + marks=[pytest.mark.notimpl(["dask", "pandas"])], ), ] @@ -301,7 +293,6 @@ def test_memtable_to_file(tmp_path, con, ftype, monkeypatch): assert outfile.is_file() -@pytest.mark.notimpl(["flink"]) def test_table_to_csv(tmp_path, backend, awards_players): outcsv = tmp_path / "out.csv" @@ -315,7 +306,6 @@ def test_table_to_csv(tmp_path, backend, awards_players): backend.assert_frame_equal(awards_players.to_pandas(), df) -@pytest.mark.notimpl(["flink"]) @pytest.mark.notimpl( ["duckdb"], reason="cannot inline WriteOptions objects", @@ -339,10 +329,7 @@ def test_table_to_csv_writer_kwargs(delimiter, tmp_path, awards_players): dt.Decimal(38, 9), pa.Decimal128Type, id="decimal128", - marks=[ - pytest.mark.notyet(["flink"], raises=NotImplementedError), - pytest.mark.notyet(["exasol"], raises=ExaQueryError), - ], + marks=[pytest.mark.notyet(["exasol"], raises=ExaQueryError)], ), param( dt.Decimal(76, 38), @@ -361,7 +348,6 @@ def test_table_to_csv_writer_kwargs(delimiter, tmp_path, awards_players): raises=(PySparkParseException, PySparkArithmeticException), reason="precision is out of range", ), - pytest.mark.notyet(["flink"], raises=NotImplementedError), pytest.mark.notyet(["exasol"], raises=ExaQueryError), ], ), @@ -480,14 +466,7 @@ def test_to_pandas_batches_empty_table(backend, con): assert sum(map(len, t.to_pandas_batches())) == n -@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() @@ -496,16 +475,7 @@ def test_to_pandas_batches_nonempty_table(backend, con, n): assert sum(map(len, t.to_pandas_batches())) == 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 ab353c2acfa27..02ee64b5057e1 100644 --- a/ibis/backends/tests/test_generic.py +++ b/ibis/backends/tests/test_generic.py @@ -26,7 +26,6 @@ MySQLProgrammingError, OracleDatabaseError, PsycoPg2InternalError, - Py4JJavaError, PyDruidProgrammingError, PyODBCDataError, PyODBCProgrammingError, @@ -120,6 +119,11 @@ def test_scalar_fillna_nullif(con, expr, expected): raises=ExaQueryError, reason="no way to test for nan-ness", ), + pytest.mark.notyet( + ["flink"], + "NaN is not supported in Flink SQL", + raises=NotImplementedError, + ), ], id="nan_col", ), @@ -128,7 +132,6 @@ def test_scalar_fillna_nullif(con, expr, expected): ), ], ) -@pytest.mark.notyet(["flink"], "NaN is not supported in Flink SQL", raises=ValueError) def test_isna(backend, alltypes, col, value, filt): table = alltypes.select(**{col: value}) df = table.execute() @@ -168,7 +171,9 @@ def test_isna(backend, alltypes, col, value, filt): reason="NaN != NULL for these backends", ), pytest.mark.notyet( - ["flink"], "NaN is not supported in Flink SQL", raises=ValueError + ["flink"], + "NaN is not supported in Flink SQL", + raises=NotImplementedError, ), ], id="nan_col", @@ -373,7 +378,9 @@ def test_case_where(backend, alltypes, df): # TODO: some of these are notimpl (datafusion) others are probably never @pytest.mark.notimpl(["mysql", "sqlite", "mssql", "druid", "exasol"]) -@pytest.mark.notyet(["flink"], "NaN is not supported in Flink SQL", raises=ValueError) +@pytest.mark.notyet( + ["flink"], "NaN is not supported in Flink SQL", raises=NotImplementedError +) def test_select_filter_mutate(backend, alltypes, df): """Test that select, filter and mutate are executed in right order. @@ -484,14 +491,15 @@ def test_dropna_invalid(alltypes): @pytest.mark.parametrize( "subset", [ - None, + param(None, id="none"), param( [], marks=pytest.mark.notimpl(["exasol"], raises=ExaQueryError, strict=False), + id="empty", ), - "col_1", - ["col_1", "col_2"], - ["col_1", "col_3"], + param("col_1", id="single"), + param(["col_1", "col_2"], id="one-and-two"), + param(["col_1", "col_3"], id="one-and-three"), ], ) def test_dropna_table(backend, alltypes, how, subset): @@ -748,11 +756,6 @@ def test_between(backend, alltypes, df): @pytest.mark.notimpl(["druid"]) -@pytest.mark.notimpl( - ["flink"], - raises=Py4JJavaError, - reason="Flink does not support now() - t.`timestamp_col`", -) def test_interactive(alltypes, monkeypatch): monkeypatch.setattr(ibis.options, "interactive", True) @@ -831,6 +834,7 @@ def test_exists(batting, awards_players, method_name): "druid", "oracle", "exasol", + "flink", ], raises=com.OperationNotDefinedError, ) @@ -995,7 +999,6 @@ def test_memtable_column_naming_mismatch(backend, con, monkeypatch, df, columns) @pytest.mark.notimpl( ["dask", "pandas", "polars"], raises=NotImplementedError, reason="not a SQL backend" ) -@pytest.mark.notimpl(["flink"], reason="no sqlglot dialect", raises=ValueError) 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))) @@ -1390,7 +1393,7 @@ def test_try_cast(con, from_val, to_type, expected): "int", marks=[ pytest.mark.never( - ["clickhouse", "pyspark"], reason="casts to 1672531200" + ["clickhouse", "pyspark", "flink"], reason="casts to 1672531200" ), pytest.mark.notyet(["bigquery"], raises=GoogleBadRequest), pytest.mark.notyet(["trino"], raises=TrinoUserError), @@ -1750,7 +1753,7 @@ def test_dynamic_table_slice_with_computed_offset(backend): backend.assert_frame_equal(result, expected) -@pytest.mark.notimpl(["druid", "flink", "polars", "snowflake"]) +@pytest.mark.notimpl(["druid", "polars", "snowflake"]) @pytest.mark.notimpl( ["risingwave"], raises=PsycoPg2InternalError, @@ -1771,7 +1774,7 @@ def test_sample(backend): backend.assert_frame_equal(empty, df.iloc[:0]) -@pytest.mark.notimpl(["druid", "flink", "polars", "snowflake"]) +@pytest.mark.notimpl(["druid", "polars", "snowflake"]) @pytest.mark.notimpl( ["risingwave"], raises=PsycoPg2InternalError, @@ -1829,7 +1832,6 @@ def test_substitute(backend): @pytest.mark.notimpl( ["dask", "pandas", "polars"], raises=NotImplementedError, reason="not a SQL backend" ) -@pytest.mark.notimpl(["flink"], reason="no sqlglot dialect", raises=ValueError) def test_simple_memtable_construct(con): t = ibis.memtable({"a": [1, 2]}) expr = t.a diff --git a/ibis/backends/tests/test_join.py b/ibis/backends/tests/test_join.py index fa36dea7ab0ce..14d3263a7a2b9 100644 --- a/ibis/backends/tests/test_join.py +++ b/ibis/backends/tests/test_join.py @@ -289,7 +289,6 @@ def test_join_with_trivial_predicate(awards_players, predicate, how, pandas_valu @pytest.mark.notimpl(["druid"], raises=PyDruidProgrammingError) -@pytest.mark.notimpl(["flink"], reason="`win` table isn't loaded") @pytest.mark.parametrize( ("how", "nrows", "gen_right", "keys"), [ diff --git a/ibis/backends/tests/test_map.py b/ibis/backends/tests/test_map.py index 441332cc8f6d1..4efc9fd0527c1 100644 --- a/ibis/backends/tests/test_map.py +++ b/ibis/backends/tests/test_map.py @@ -207,11 +207,6 @@ def test_literal_map_merge(con): assert con.execute(expr) == {"a": 1, "b": 2, "c": 3} -@pytest.mark.notimpl( - ["flink"], - raises=NotImplementedError, - reason="No translation rule for map", -) @pytest.mark.notimpl( ["risingwave"], raises=PsycoPg2InternalError, @@ -229,11 +224,6 @@ def test_literal_map_getitem_broadcast(backend, alltypes, df): backend.assert_series_equal(result, expected) -@pytest.mark.notimpl( - ["flink"], - raises=NotImplementedError, - reason="No translation rule for map", -) @pytest.mark.notimpl( ["risingwave"], raises=PsycoPg2InternalError, @@ -296,11 +286,6 @@ def test_map_construct_array_column(con, alltypes, df): @pytest.mark.notyet( ["postgres", "risingwave"], reason="only support maps of string -> string" ) -@pytest.mark.notimpl( - ["flink"], - raises=NotImplementedError, - reason="No translation rule for map", -) def test_map_get_with_compatible_value_smaller(con): value = ibis.literal({"A": 1000, "B": 2000}) expr = value.get("C", 3) @@ -310,11 +295,6 @@ def test_map_get_with_compatible_value_smaller(con): @pytest.mark.notyet( ["postgres", "risingwave"], reason="only support maps of string -> string" ) -@pytest.mark.notimpl( - ["flink"], - raises=NotImplementedError, - reason="No translation rule for map", -) def test_map_get_with_compatible_value_bigger(con): value = ibis.literal({"A": 1, "B": 2}) expr = value.get("C", 3000) @@ -324,11 +304,6 @@ def test_map_get_with_compatible_value_bigger(con): @pytest.mark.notyet( ["postgres", "risingwave"], reason="only support maps of string -> string" ) -@pytest.mark.notimpl( - ["flink"], - raises=NotImplementedError, - reason="NotImplementedError: No translation rule for map", -) def test_map_get_with_incompatible_value_different_kind(con): value = ibis.literal({"A": 1000, "B": 2000}) expr = value.get("C", 3.0) @@ -339,11 +314,6 @@ def test_map_get_with_incompatible_value_different_kind(con): @pytest.mark.notyet( ["postgres", "risingwave"], reason="only support maps of string -> string" ) -@pytest.mark.notimpl( - ["flink"], - raises=NotImplementedError, - reason="No translation rule for map", -) def test_map_get_with_null_on_not_nullable(con, null_value): map_type = dt.Map(dt.string, dt.Int16(nullable=False)) value = ibis.literal({"A": 1000, "B": 2000}).cast(map_type) @@ -353,10 +323,8 @@ def test_map_get_with_null_on_not_nullable(con, null_value): @pytest.mark.parametrize("null_value", [None, ibis.NA]) -@pytest.mark.notimpl( - ["flink"], - raises=NotImplementedError, - reason="No translation rule for map", +@pytest.mark.notyet( + ["flink"], raises=Py4JJavaError, reason="Flink cannot handle typeless nulls" ) @pytest.mark.notimpl( ["risingwave"], @@ -373,10 +341,8 @@ def test_map_get_with_null_on_null_type_with_null(con, null_value): @pytest.mark.notyet( ["postgres", "risingwave"], reason="only support maps of string -> string" ) -@pytest.mark.notimpl( - ["flink"], - raises=NotImplementedError, - reason="No translation rule for map", +@pytest.mark.notyet( + ["flink"], raises=Py4JJavaError, reason="Flink cannot handle typeless nulls" ) def test_map_get_with_null_on_null_type_with_non_null(con): value = ibis.literal({"A": None, "B": None}) diff --git a/ibis/backends/tests/test_network.py b/ibis/backends/tests/test_network.py index dca5815c68554..0947ecf6f4d20 100644 --- a/ibis/backends/tests/test_network.py +++ b/ibis/backends/tests/test_network.py @@ -25,7 +25,7 @@ } -@pytest.mark.notimpl(["flink", "polars"], raises=NotImplementedError) +@pytest.mark.notimpl(["polars"], raises=NotImplementedError) def test_macaddr_literal(con, backend): test_macaddr = "00:00:0A:BB:28:FC" expr = ibis.literal(test_macaddr, type=dt.macaddr) @@ -110,7 +110,7 @@ def test_macaddr_literal(con, backend): ), ], ) -@pytest.mark.notimpl(["flink", "polars"], raises=NotImplementedError) +@pytest.mark.notimpl(["polars"], raises=NotImplementedError) @pytest.mark.notimpl(["druid", "oracle"], raises=KeyError) @pytest.mark.notimpl(["exasol"], raises=(ExaQueryError, KeyError)) def test_inet_literal(con, backend, test_value, expected_values, expected_types): diff --git a/ibis/backends/tests/test_numeric.py b/ibis/backends/tests/test_numeric.py index 79989561ae883..95acac0eb7248 100644 --- a/ibis/backends/tests/test_numeric.py +++ b/ibis/backends/tests/test_numeric.py @@ -26,6 +26,7 @@ PsycoPg2DivisionByZero, PsycoPg2InternalError, Py4JError, + Py4JJavaError, PyDruidProgrammingError, PyODBCDataError, PyODBCProgrammingError, @@ -373,7 +374,7 @@ def test_numeric_literal(con, backend, expr, expected_types): pytest.mark.notyet( ["flink"], "The precision can be up to 38 in Flink", - raises=ValueError, + raises=Py4JJavaError, ), pytest.mark.notyet(["mssql"], raises=PyODBCProgrammingError), ], @@ -431,7 +432,7 @@ def test_numeric_literal(con, backend, expr, expected_types): pytest.mark.notyet( ["flink"], "Infinity is not supported in Flink SQL", - raises=ValueError, + raises=Py4JJavaError, ), pytest.mark.notyet( ["snowflake"], @@ -490,7 +491,7 @@ def test_numeric_literal(con, backend, expr, expected_types): pytest.mark.notyet( ["flink"], "Infinity is not supported in Flink SQL", - raises=ValueError, + raises=Py4JJavaError, ), pytest.mark.notyet( ["snowflake"], @@ -557,7 +558,7 @@ def test_numeric_literal(con, backend, expr, expected_types): pytest.mark.notyet( ["flink"], "NaN is not supported in Flink SQL", - raises=ValueError, + raises=Py4JJavaError, ), pytest.mark.notyet( ["snowflake"], @@ -659,8 +660,9 @@ def test_decimal_literal(con, backend, expr, expected_types, expected_result): ), ], ) +@pytest.mark.notimpl(["sqlite", "mssql", "druid"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl( - ["sqlite", "mssql", "flink", "druid"], raises=com.OperationNotDefinedError + ["flink"], raises=(com.OperationNotDefinedError, NotImplementedError) ) @pytest.mark.notimpl(["mysql"], raises=(MySQLOperationalError, NotImplementedError)) def test_isnan_isinf( @@ -716,16 +718,7 @@ def test_isnan_isinf( param(L(5.556).exp(), math.exp(5.556), id="exp"), param(L(5.556).sign(), 1, id="sign-pos"), param(L(-5.556).sign(), -1, id="sign-neg"), - param( - L(0).sign(), - 0, - id="sign-zero", - marks=pytest.mark.broken( - ["flink"], - "An error occurred while calling z:org.apache.flink.table.runtime.arrow.ArrowUtils.collectAsPandasDataFrame.", - raises=Py4JError, - ), - ), + param(L(0).sign(), 0, id="sign-zero"), param(L(5.556).sqrt(), math.sqrt(5.556), id="sqrt"), param( L(5.556).log(2), @@ -1130,7 +1123,7 @@ def test_floating_mod(backend, alltypes, df): ), pytest.mark.notyet( "flink", - raises=Py4JError, + raises=Py4JJavaError, reason="Flink doesn't do integer division by zero", ), ], @@ -1146,7 +1139,7 @@ def test_floating_mod(backend, alltypes, df): ), pytest.mark.notyet( "flink", - raises=Py4JError, + raises=Py4JJavaError, reason="Flink doesn't do integer division by zero", ), ], @@ -1162,7 +1155,7 @@ def test_floating_mod(backend, alltypes, df): ), pytest.mark.notyet( "flink", - raises=Py4JError, + raises=Py4JJavaError, reason="Flink doesn't do integer division by zero", ), ], @@ -1178,7 +1171,7 @@ def test_floating_mod(backend, alltypes, df): ), pytest.mark.notyet( "flink", - raises=Py4JError, + raises=Py4JJavaError, reason="Flink doesn't do integer division by zero", ), ], @@ -1195,6 +1188,11 @@ def test_floating_mod(backend, alltypes, df): reason="Oracle doesn't do integer division by zero", ), pytest.mark.never(["impala"], reason="doesn't allow divide by zero"), + pytest.mark.notyet( + "flink", + raises=Py4JJavaError, + reason="Flink doesn't do integer division by zero", + ), ], ), param( @@ -1207,6 +1205,11 @@ def test_floating_mod(backend, alltypes, df): reason="Oracle doesn't do integer division by zero", ), pytest.mark.never(["impala"], reason="doesn't allow divide by zero"), + pytest.mark.notyet( + "flink", + raises=Py4JJavaError, + reason="Flink doesn't do integer division by zero", + ), ], ), param( @@ -1219,6 +1222,11 @@ def test_floating_mod(backend, alltypes, df): reason="Oracle doesn't do integer division by zero", ), pytest.mark.never(["impala"], reason="doesn't allow divide by zero"), + pytest.mark.notyet( + "flink", + raises=Py4JJavaError, + reason="Flink doesn't do integer division by zero", + ), ], ), param( @@ -1231,6 +1239,11 @@ def test_floating_mod(backend, alltypes, df): reason="Oracle doesn't do integer division by zero", ), pytest.mark.never(["impala"], reason="doesn't allow divide by zero"), + pytest.mark.notyet( + "flink", + raises=Py4JJavaError, + reason="Flink doesn't do integer division by zero", + ), ], ), param( @@ -1343,11 +1356,6 @@ def test_clip(backend, alltypes, df, ibis_func, pandas_func): raises=PyDruidProgrammingError, reason="SQL query requires 'MIN' operator that is not supported.", ) -@pytest.mark.never( - ["flink"], - raises=com.UnsupportedOperationError, - reason="Flink does not support 'MIN' or 'MAX' operation without specifying window.", -) def test_histogram(con, alltypes): n = 10 hist = con.execute(alltypes.int_col.histogram(n).name("hist")) diff --git a/ibis/backends/tests/test_param.py b/ibis/backends/tests/test_param.py index e1f59c1fcbf90..3be9222cee342 100644 --- a/ibis/backends/tests/test_param.py +++ b/ibis/backends/tests/test_param.py @@ -11,11 +11,7 @@ import ibis import ibis.expr.datatypes as dt from ibis import _ -from ibis.backends.tests.errors import ( - OracleDatabaseError, - PsycoPg2InternalError, - Py4JJavaError, -) +from ibis.backends.tests.errors import OracleDatabaseError, PsycoPg2InternalError @pytest.mark.parametrize( @@ -103,15 +99,6 @@ def test_scalar_param_struct(con): reason="mysql and sqlite will never implement map types", ) @pytest.mark.notyet(["bigquery"]) -@pytest.mark.notimpl( - ["flink"], - "WIP", - raises=Py4JJavaError, - reason=( - "SqlParseException: Expecting alias, found character literal" - "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, diff --git a/ibis/backends/tests/test_sql.py b/ibis/backends/tests/test_sql.py index a9d526c30146f..a5edcf3c5db81 100644 --- a/ibis/backends/tests/test_sql.py +++ b/ibis/backends/tests/test_sql.py @@ -10,8 +10,6 @@ sg = pytest.importorskip("sqlglot") -pytestmark = pytest.mark.notimpl(["flink"]) - simple_literal = param(ibis.literal(1), id="simple_literal") array_literal = param( ibis.array([1]), diff --git a/ibis/backends/tests/test_string.py b/ibis/backends/tests/test_string.py index ea954243f505c..2ef5ed61cd28b 100644 --- a/ibis/backends/tests/test_string.py +++ b/ibis/backends/tests/test_string.py @@ -14,6 +14,7 @@ ClickHouseDatabaseError, OracleDatabaseError, PsycoPg2InternalError, + Py4JJavaError, PyDruidProgrammingError, PyODBCProgrammingError, ) @@ -65,6 +66,9 @@ raises=PsycoPg2InternalError, reason='sql parser error: Expected end of statement, found: "NG\'" at line:1, column:31 Near "SELECT \'STRI"NG\' AS "\'STRI""', ), + pytest.mark.broken( + ["flink"], raises=Py4JJavaError, reason="parse error" + ), ], ), param( @@ -135,6 +139,9 @@ def uses_java_re(t): pytest.mark.broken( ["mssql"], raises=PyODBCProgrammingError, reason="incorrect syntax" ), + pytest.mark.broken( + ["flink"], raises=Py4JJavaError, reason="incorrect syntax" + ), ], ), param( @@ -399,7 +406,12 @@ def uses_java_re(t): lambda t: t.string_col.find("a"), lambda t: t.string_col.str.find("a"), id="find", - marks=pytest.mark.notimpl(["polars"], raises=com.OperationNotDefinedError), + marks=[ + pytest.mark.notimpl(["polars"], raises=com.OperationNotDefinedError), + pytest.mark.broken( + ["flink"], raises=Py4JJavaError, reason="incorrect code" + ), + ], ), param( lambda t: t.date_string_col.find("13", 3), @@ -408,6 +420,9 @@ def uses_java_re(t): marks=[ pytest.mark.notimpl(["polars"], raises=com.OperationNotDefinedError), pytest.mark.notyet(["bigquery"], raises=NotImplementedError), + pytest.mark.broken( + ["flink"], raises=Py4JJavaError, reason="incorrect code" + ), ], ), param( @@ -514,6 +529,9 @@ def uses_java_re(t): id="startswith", marks=[ pytest.mark.notimpl(["mssql"], raises=com.OperationNotDefinedError), + pytest.mark.broken( + ["flink"], raises=Py4JJavaError, reason="incorrect code" + ), ], ), param( @@ -524,7 +542,8 @@ def uses_java_re(t): id="endswith", marks=[ pytest.mark.notimpl( - ["datafusion", "mssql"], raises=com.OperationNotDefinedError + ["datafusion", "mssql", "flink"], + raises=com.OperationNotDefinedError, ), ], ), @@ -534,6 +553,9 @@ def uses_java_re(t): id="startswith-simple", marks=[ pytest.mark.notimpl(["mssql"], raises=com.OperationNotDefinedError), + pytest.mark.broken( + ["flink"], raises=Py4JJavaError, reason="incorrect code" + ), ], ), param( @@ -542,7 +564,8 @@ def uses_java_re(t): id="endswith-simple", marks=[ pytest.mark.notimpl( - ["datafusion", "mssql"], raises=com.OperationNotDefinedError + ["datafusion", "mssql", "flink"], + raises=com.OperationNotDefinedError, ), ], ), @@ -550,6 +573,11 @@ def uses_java_re(t): lambda t: t.string_col.strip(), lambda t: t.string_col.str.strip(), id="strip", + marks=[ + pytest.mark.broken( + ["flink"], raises=Py4JJavaError, reason="incorrect code" + ), + ], ), param( lambda t: t.string_col.lstrip(), @@ -602,10 +630,7 @@ def uses_java_re(t): lambda t: t.date_string_col[-2], lambda t: t.date_string_col.str[-2], id="negative-index", - marks=[ - pytest.mark.broken(["druid"], raises=PyDruidProgrammingError), - pytest.mark.broken(["flink"], raises=AssertionError), - ], + marks=[pytest.mark.broken(["druid"], raises=PyDruidProgrammingError)], ), param( lambda t: t.date_string_col[t.date_string_col.length() - 1 :], @@ -836,6 +861,7 @@ def test_substr_with_null_values(backend, alltypes, df): "pyspark", "druid", "oracle", + "flink", ], raises=com.OperationNotDefinedError, ) diff --git a/ibis/backends/tests/test_temporal.py b/ibis/backends/tests/test_temporal.py index 0987e991d4fc8..f750e2c4059e2 100644 --- a/ibis/backends/tests/test_temporal.py +++ b/ibis/backends/tests/test_temporal.py @@ -69,7 +69,7 @@ def test_date_extract(backend, alltypes, df, attr, expr_fn): "day_of_year", marks=[ pytest.mark.notimpl( - ["exasol", "impala"], raises=com.OperationNotDefinedError + ["exasol", "impala", "flink"], raises=com.OperationNotDefinedError ), pytest.mark.notyet(["oracle"], raises=com.OperationNotDefinedError), ], @@ -164,7 +164,8 @@ def test_timestamp_extract(backend, alltypes, df, attr): ) def test_timestamp_extract_literal(con, func, expected): value = ibis.timestamp("2015-09-01 14:48:05.359") - assert con.execute(func(value).name("tmp")) == expected + expr = func(value).name("tmp") + assert con.execute(expr) == expected @pytest.mark.notimpl(["oracle"], raises=com.OperationNotDefinedError) @@ -274,14 +275,6 @@ def test_timestamp_extract_week_of_year(backend, alltypes, df): raises=AssertionError, reason="numpy array are different", ), - pytest.mark.never( - ["flink"], - raises=Py4JJavaError, - reason=( - "CalciteContextException: No match found for function signature trunc(, )" - "Timestamp truncation is not supported in Flink" - ), - ), ], ), param( @@ -292,14 +285,6 @@ def test_timestamp_extract_week_of_year(backend, alltypes, df): raises=AssertionError, reason="numpy array are different", ), - pytest.mark.never( - ["flink"], - raises=Py4JJavaError, - reason=( - "CalciteContextException: No match found for function signature trunc(, )" - "Timestamp truncation is not supported in Flink" - ), - ), ], ), param( @@ -310,14 +295,6 @@ def test_timestamp_extract_week_of_year(backend, alltypes, df): raises=AssertionError, reason="numpy array are different", ), - pytest.mark.never( - ["flink"], - raises=Py4JJavaError, - reason=( - "CalciteContextException: No match found for function signature trunc(, )" - "Timestamp truncation is not supported in Flink" - ), - ), ], ), param( @@ -329,14 +306,6 @@ def test_timestamp_extract_week_of_year(backend, alltypes, df): raises=AssertionError, reason="numpy array are different", ), - pytest.mark.never( - ["flink"], - raises=Py4JJavaError, - reason=( - "CalciteContextException: No match found for function signature trunc(, )" - "Timestamp truncation is not supported in Flink" - ), - ), ], ), param( @@ -348,14 +317,6 @@ def test_timestamp_extract_week_of_year(backend, alltypes, df): raises=AssertionError, reason="numpy array are different", ), - pytest.mark.never( - ["flink"], - raises=Py4JJavaError, - reason=( - "CalciteContextException: No match found for function signature trunc(, )" - "Timestamp truncation is not supported in Flink" - ), - ), ], ), param( @@ -367,14 +328,6 @@ def test_timestamp_extract_week_of_year(backend, alltypes, df): raises=AssertionError, reason="numpy array are different", ), - pytest.mark.never( - ["flink"], - raises=Py4JJavaError, - reason=( - "CalciteContextException: No match found for function signature trunc(, )" - "Timestamp truncation is not supported in Flink" - ), - ), ], ), param( @@ -386,11 +339,6 @@ def test_timestamp_extract_week_of_year(backend, alltypes, df): raises=AssertionError, reason="numpy array are different", ), - pytest.mark.never( - ["flink"], - raises=com.UnsupportedOperationError, - reason=" unit is not supported in timestamp truncate", - ), ], ), param( @@ -405,11 +353,6 @@ def test_timestamp_extract_week_of_year(backend, alltypes, df): raises=AssertionError, reason="numpy array are different", ), - pytest.mark.notimpl( - ["flink"], - raises=com.UnsupportedOperationError, - reason=" unit is not supported in timestamp truncate", - ), ], ), param( @@ -424,11 +367,6 @@ def test_timestamp_extract_week_of_year(backend, alltypes, df): raises=AssertionError, reason="numpy array are different", ), - pytest.mark.notimpl( - ["flink"], - raises=com.UnsupportedOperationError, - reason=" unit is not supported in timestamp truncate", - ), ], ), param( @@ -457,11 +395,6 @@ def test_timestamp_extract_week_of_year(backend, alltypes, df): raises=PolarsPanicException, reason="attempt to calculate the remainder with a divisor of zero", ), - pytest.mark.notimpl( - ["flink"], - raises=com.UnsupportedOperationError, - reason=" unit is not supported in timestamp truncate", - ), ], ), ], @@ -471,7 +404,7 @@ def test_timestamp_extract_week_of_year(backend, alltypes, df): raises=AttributeError, reason="AttributeError: 'StringColumn' object has no attribute 'truncate'", ) -@pytest.mark.notimpl(["exasol"], raises=com.OperationNotDefinedError) +@pytest.mark.notimpl(["exasol", "flink"], raises=com.OperationNotDefinedError) def test_timestamp_truncate(backend, alltypes, df, unit): expr = alltypes.timestamp_col.truncate(unit).name("tmp") @@ -984,9 +917,9 @@ def convert_to_offset(x): raises=AttributeError, reason="'StringColumn' object has no attribute 'date'", ), - pytest.mark.broken( + pytest.mark.notimpl( ["flink"], - raises=com.UnsupportedOperationError, + raises=com.OperationNotDefinedError, reason="DATE_DIFF is not supported in Flink", ), pytest.mark.broken( @@ -1207,26 +1140,7 @@ def test_temporal_binop_pandas_timedelta( backend.assert_series_equal(result, expected.astype(result.dtype)) -@pytest.mark.parametrize( - "func_name", - [ - "gt", - "ge", - "lt", - "le", - "eq", - param( - "ne", - marks=[ - pytest.mark.notimpl( - ["flink"], - raises=Py4JJavaError, - reason="SqlParseException: Bang equal '!=' is not allowed under the current SQL conformance level", - ), - ], - ), - ], -) +@pytest.mark.parametrize("func_name", ["gt", "ge", "lt", "le", "eq", "ne"]) @pytest.mark.notimpl( ["polars"], raises=TypeError, @@ -1316,16 +1230,7 @@ def test_timestamp_comparison_filter(backend, con, alltypes, df, func_name): ], ), "eq", - param( - "ne", - marks=[ - pytest.mark.notimpl( - ["flink"], - raises=Py4JJavaError, - reason="SqlParseException: Bang equal '!=' is not allowed under the current SQL conformance level", - ), - ], - ), + "ne", ], ) @pytest.mark.broken( @@ -1370,11 +1275,6 @@ def test_timestamp_comparison_filter_numpy(backend, con, alltypes, df, func_name raises=AttributeError, reason="'StringColumn' object has no attribute 'date'", ) -@pytest.mark.broken( - ["flink"], - raises=Py4JJavaError, - reason="ParseException: Encountered '+ INTERVAL CAST'", -) def test_interval_add_cast_scalar(backend, alltypes): timestamp_date = alltypes.timestamp_col.date() delta = ibis.literal(10).cast("interval('D')") @@ -1393,6 +1293,7 @@ def test_interval_add_cast_scalar(backend, alltypes): raises=AttributeError, reason="'StringColumn' object has no attribute 'date'", ) +@pytest.mark.broken(["flink"], raises=AssertionError, reason="incorrect results") def test_interval_add_cast_column(backend, alltypes, df): timestamp_date = alltypes.timestamp_col.date() delta = alltypes.bigint_col.cast("interval('D')") @@ -1438,10 +1339,6 @@ def test_interval_add_cast_column(backend, alltypes, df): raises=AttributeError, reason="'StringColumn' object has no attribute 'strftime'", ), - pytest.mark.notimpl( - ["flink"], - raises=AssertionError, - ), ], id="column_format_str", ), @@ -1638,11 +1535,6 @@ def test_string_to_timestamp(alltypes, fmt): ], ) @pytest.mark.notimpl(["druid", "oracle"], raises=com.OperationNotDefinedError) -@pytest.mark.notimpl( - ["flink"], - raises=Py4JJavaError, - reason="DayOfWeekName is not supported in Flink", -) @pytest.mark.notimpl(["exasol"], raises=com.OperationNotDefinedError) @pytest.mark.broken( ["risingwave"], @@ -1702,20 +1594,6 @@ def test_day_of_week_column(backend, alltypes, df): raises=AssertionError, reason="Refer to https://github.com/risingwavelabs/risingwave/issues/14670", ), - pytest.mark.never( - ["flink"], - raises=Py4JJavaError, - reason=( - "SqlValidatorException: No match found for function signature dayname()" - "`day_of_week_name` is not supported in Flink" - "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", - ), ], ), ], @@ -1918,6 +1796,7 @@ def test_timestamp_with_timezone_literal(con, timezone, expected): @pytest.mark.notyet( ["clickhouse", "impala", "exasol"], raises=com.OperationNotDefinedError ) +@pytest.mark.notyet(["flink"], raises=Py4JJavaError) @pytest.mark.notimpl(["druid"], raises=com.OperationNotDefinedError) @pytest.mark.notimpl( ["risingwave"], @@ -1949,7 +1828,7 @@ def test_time_literal(con, backend): @pytest.mark.parametrize( "microsecond", [ - 0, + param(0, marks=[pytest.mark.notyet(["flink"], raises=Py4JJavaError)]), param( 561021, marks=[ @@ -1959,14 +1838,7 @@ def test_time_literal(con, backend): reason="doesn't have enough precision to capture microseconds", ), pytest.mark.notyet(["trino"], raises=AssertionError), - pytest.mark.notimpl( - ["flink"], - raises=AssertionError, - reason=( - "Flink does not support microsecond precision in time." - "assert datetime.time(13, 20, 5) == datetime.time(13, 20, 5, 561021)" - ), - ), + pytest.mark.notyet(["flink"], raises=Py4JJavaError), ], ), ], @@ -2176,7 +2048,6 @@ def test_integer_cast_to_timestamp_scalar(alltypes, df): ) @pytest.mark.broken( ["flink"], - reason="Casting from timestamp[s] to timestamp[ns] would result in out of bounds timestamp: 81953424000", raises=ArrowInvalid, ) @pytest.mark.notyet(["polars"], raises=PolarsComputeError) @@ -2272,15 +2143,10 @@ def test_large_timestamp(con): id="us", marks=[ pytest.mark.notyet( - ["sqlite"], + ["sqlite", "flink"], reason="doesn't support microseconds", raises=AssertionError, ), - pytest.mark.broken( - ["flink"], - reason="assert Timestamp('2023-01-07 13:20:05.561000') == Timestamp('2023-01-07 13:20:05.561021')", - raises=AssertionError, - ), pytest.mark.notyet( ["druid"], reason="time_parse truncates to milliseconds", @@ -2378,6 +2244,7 @@ def test_timestamp_precision_output(con, ts, scale, unit): 22, id="time", marks=[ + pytest.mark.notimpl(["flink"], raises=Py4JJavaError), pytest.mark.notimpl( ["clickhouse"], raises=com.OperationNotDefinedError, diff --git a/ibis/backends/tests/test_timecontext.py b/ibis/backends/tests/test_timecontext.py index a974bc9ee2961..5b335fe2b8f3d 100644 --- a/ibis/backends/tests/test_timecontext.py +++ b/ibis/backends/tests/test_timecontext.py @@ -122,7 +122,7 @@ def test_context_adjustment_filter_before_window( @pytest.mark.notimpl(["duckdb"]) @pytest.mark.notimpl( ["flink"], - raises=com.UnsupportedOperationError, + raises=com.OperationNotDefinedError, reason="Flink engine does not support generic window clause with no order by", ) def test_context_adjustment_multi_col_udf_non_grouped( diff --git a/ibis/backends/tests/test_window.py b/ibis/backends/tests/test_window.py index 768872f76f468..1c941c8c08a47 100644 --- a/ibis/backends/tests/test_window.py +++ b/ibis/backends/tests/test_window.py @@ -178,11 +178,6 @@ def calc_zscore(s): raises=AssertionError, reason="Results are shifted + 1", ), - pytest.mark.broken( - ["flink"], - raises=com.UnsupportedOperationError, - reason="Windows in Flink can only be ordered by a single time column", - ), pytest.mark.notimpl( ["risingwave"], raises=PsycoPg2InternalError, @@ -434,11 +429,6 @@ def test_ungrouped_bounded_expanding_window( ], ) @pytest.mark.notimpl(["polars"], raises=com.OperationNotDefinedError) -@pytest.mark.notimpl( - ["flink"], - raises=com.UnsupportedOperationError, - reason="OVER RANGE FOLLOWING windows are not supported in Flink yet", -) def test_grouped_bounded_following_window(backend, alltypes, df, preceding, following): window = ibis.window( preceding=preceding, @@ -569,7 +559,6 @@ def test_grouped_bounded_preceding_window(backend, alltypes, df, window_fn): False, id="unordered", marks=[ - pytest.mark.notimpl(["flink"], raises=com.UnsupportedOperationError), pytest.mark.broken( ["mssql"], raises=PyODBCProgrammingError, @@ -621,7 +610,6 @@ def test_grouped_unbounded_window( @pytest.mark.broken(["dask"], raises=AssertionError) @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, @@ -674,7 +662,7 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): id="ordered-mean", marks=[ pytest.mark.broken( - ["flink", "impala"], + ["impala"], reason="default window semantics are different", raises=AssertionError, ), @@ -691,11 +679,6 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): False, id="unordered-mean", marks=[ - pytest.mark.notimpl( - ["flink"], - raises=com.UnsupportedOperationError, - reason="Flink engine does not support generic window clause with no order by", - ), pytest.mark.broken( ["mssql"], raises=PyODBCProgrammingError, @@ -712,11 +695,6 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): pytest.mark.notimpl( ["pandas", "dask"], raises=com.OperationNotDefinedError ), - pytest.mark.notimpl( - ["flink"], - raises=Py4JJavaError, - reason="CalciteContextException: Argument to function 'NTILE' must be a literal", - ), pytest.mark.notimpl( ["risingwave"], raises=PsycoPg2InternalError, @@ -774,14 +752,10 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): "trino", "datafusion", "exasol", + "flink", ], raises=com.OperationNotDefinedError, ), - pytest.mark.notimpl( - ["flink"], - raises=com.UnsupportedOperationError, - reason="Flink engine does not support generic window clause with no order by", - ), ], ), # Analytic ops @@ -926,14 +900,10 @@ def test_simple_ungrouped_window_with_scalar_order_by(alltypes): "trino", "datafusion", "exasol", + "flink", ], raises=com.OperationNotDefinedError, ), - pytest.mark.notimpl( - ["flink"], - raises=com.UnsupportedOperationError, - reason="Flink engine does not support generic window clause with no order by", - ), ], ), ], @@ -970,11 +940,6 @@ def test_ungrouped_unbounded_window( @pytest.mark.notimpl( ["impala"], raises=ImpalaHiveServer2Error, reason="limited RANGE support" ) -@pytest.mark.notimpl( - ["flink"], - raises=com.UnsupportedOperationError, - reason="Data Type mismatch between ORDER BY and RANGE clause", -) @pytest.mark.notyet( ["clickhouse"], reason="RANGE OFFSET frame for 'DB::ColumnNullable' ORDER BY column is not implemented", @@ -1115,11 +1080,6 @@ def test_mutate_window_filter(backend, alltypes): @pytest.mark.notimpl(["polars", "exasol"], raises=com.OperationNotDefinedError) -@pytest.mark.notimpl( - ["flink"], - raises=Exception, - reason="KeyError: Table with name win doesn't exist.", -) def test_first_last(backend): t = backend.win w = ibis.window(group_by=t.g, order_by=[t.x, t.y], preceding=1, following=0) diff --git a/poetry-overrides.nix b/poetry-overrides.nix index 176ce22e308f9..212b84c5da3c7 100644 --- a/poetry-overrides.nix +++ b/poetry-overrides.nix @@ -18,9 +18,28 @@ self: super: { }) ]; }); + pyodbc = super.pyodbc.overridePythonAttrs (attrs: { preFixup = attrs.preFixup or "" + '' addAutoPatchelfSearchPath ${self.pkgs.unixODBC} ''; }); + + avro-python3 = super.avro-python3.overridePythonAttrs (attrs: { + nativeBuildInputs = attrs.nativeBuildInputs or [ ] ++ [ + self.pycodestyle + self.isort + ]; + }); + + apache-flink-libraries = super.apache-flink-libraries.overridePythonAttrs (attrs: { + buildInputs = attrs.nativeBuildInputs or [ ] ++ [ self.setuptools ]; + # apache-flink and apache-flink-libraries both install version.py into the + # pyflink output derivation, which is invalid: whichever gets installed + # last will be used + postInstall = '' + rm $out/${self.python.sitePackages}/pyflink/version.py + rm $out/${self.python.sitePackages}/pyflink/__pycache__/version.*.pyc + ''; + }); } diff --git a/poetry.lock b/poetry.lock index e1270ef4ff52d..c3df211ab2d27 100644 --- a/poetry.lock +++ b/poetry.lock @@ -146,24 +146,127 @@ files = [ ] [[package]] -name = "anywidget" -version = "0.7.1" -description = "custom jupyter widgets made easy" -optional = false +name = "apache-beam" +version = "2.48.0" +description = "Apache Beam SDK for Python" +optional = true python-versions = ">=3.7" files = [ - {file = "anywidget-0.7.1-py3-none-any.whl", hash = "sha256:17828c6e255a7066c960636cbda045d55b08c996a2fc4e886e4c8507db6e9407"}, - {file = "anywidget-0.7.1.tar.gz", hash = "sha256:1c4cb51063f8ec5172b8c8efb1bfb79861d340c3491220d56b8c2a6aa8db3b3d"}, -] - -[package.dependencies] -ipywidgets = ">=7.6.0" -psygnal = ">=0.8.1" -typing-extensions = ">=4.2.0" + {file = "apache-beam-2.48.0.zip", hash = "sha256:611b9e0015e9d1d2ca34b91453117ee5b54ca7446de505b95b2c5a5c4d9f4b1e"}, + {file = "apache_beam-2.48.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:4efbd2b808ecd371c0a827986b89da8cea9f073a0114eede9d2674b34d7044ab"}, + {file = "apache_beam-2.48.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f55df2d5d5e6036b7987bf468dbd35b6494f1998733a1b74acfd7feee80096fb"}, + {file = "apache_beam-2.48.0-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d52beeac61eea96fdab61bcc1a94e72b9ce0bef65b949a19b50aba383c70ae0b"}, + {file = "apache_beam-2.48.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:916ee34a3665e664f6790ff01c65e2332b956c8a47c4f7137b898fbf99dca664"}, + {file = "apache_beam-2.48.0-cp310-cp310-win32.whl", hash = "sha256:5ca78411e49a33daa48b62880b7cd9fdb7811540d74bdbc89c4e98c6be725813"}, + {file = "apache_beam-2.48.0-cp310-cp310-win_amd64.whl", hash = "sha256:5458e3f0ac2de29f209879cea712a443765ec6ed78779c900236fe796067aee8"}, + {file = "apache_beam-2.48.0-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:3d9223b21001bfedd460218237e8f87a3c57886781755be60d5511a7b483fae9"}, + {file = "apache_beam-2.48.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:454c3babc19d0dafab7bb38899f617ee0b73f9a991108f293fcfd30e779619f5"}, + {file = "apache_beam-2.48.0-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3bb9979ccd7ab5885f790f06a43d3a456e0422274a5361142f670199d2631263"}, + {file = "apache_beam-2.48.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:054fc78d1a4916006e347538ad570c7126ff681bd013493d444a762ead60743d"}, + {file = "apache_beam-2.48.0-cp311-cp311-win32.whl", hash = "sha256:57342f524dd8da3a0aa0c4d17aa21d3b68a1ef2821e759a99e69b6b843ef0438"}, + {file = "apache_beam-2.48.0-cp311-cp311-win_amd64.whl", hash = "sha256:70ac24ac3b4cbd4d447bf216f614fb68ba61e5068992763a75885a3392c292db"}, + {file = "apache_beam-2.48.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:40314e0be2ae0775cc361e4afdf20715c74a9aa8d2432943c5263b0598f980cb"}, + {file = "apache_beam-2.48.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f48737ac565df8fe954e4d7f9d54cf402620ba4289cfd9a2441f2fb102a432ab"}, + {file = "apache_beam-2.48.0-cp37-cp37m-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:09ad28418a79f50e1efd36e43bf4cb549ff6da17fc0e6de91af9930a4c828e66"}, + {file = "apache_beam-2.48.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c7d1b489de63435eace113acf7ac5417a2ea0428ec2c6887a50454be3872cfc8"}, + {file = "apache_beam-2.48.0-cp37-cp37m-win32.whl", hash = "sha256:3021f34b10f04b6244a03c2ae6c9ae32e3d17003d5a63f6edb25eb0e7bf92788"}, + {file = "apache_beam-2.48.0-cp37-cp37m-win_amd64.whl", hash = "sha256:4a8b51ec381d97191ba35bcfd0ee773c7daf356e08aa05ba294ed52b17460718"}, + {file = "apache_beam-2.48.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:83f5986006247791614165b81a44476b23b474dcd6b6caa5efcf9510b4d26099"}, + {file = "apache_beam-2.48.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6cc25c575e1a3e03257b4e9108b94f1ea15929ef0d6b9cad43c16376b830dede"}, + {file = "apache_beam-2.48.0-cp38-cp38-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:85275ea190ae5388d86e18e1c16a17e5dcd1264273cea478a6ea6c392305111f"}, + {file = "apache_beam-2.48.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:64cdefb72b0c71563156c23fb68037c977bb7cf6c59ab78d7a72c78d588d055e"}, + {file = "apache_beam-2.48.0-cp38-cp38-win32.whl", hash = "sha256:5cbda66f07a87d696edd266995fd90a8bba9f055451bbc26fa5296aebc657179"}, + {file = "apache_beam-2.48.0-cp38-cp38-win_amd64.whl", hash = "sha256:ce35acde3ad140b3d0cc590dd3dc03eea179c79aa6ee1b0a60188dfd367afb73"}, + {file = "apache_beam-2.48.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:764ad48b87b7c72b6482b540c8edc5dd78bb59b2c58a84f85a3edaadf9686608"}, + {file = "apache_beam-2.48.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:dd2bd7b6718304549961236ef8f0303cc0ac925977e15f09061fb85578de890b"}, + {file = "apache_beam-2.48.0-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:6ef5c1f3731b06264e27a457b8079056cc45e17df215d1943a2112f2914dee16"}, + {file = "apache_beam-2.48.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f54387f561315779783652ebb657e7893779917e7fb0e67d602c513786c8b190"}, + {file = "apache_beam-2.48.0-cp39-cp39-win32.whl", hash = "sha256:a4165c163041b2151e29569096b613f81c23c577c2072734f4cf4781e4236490"}, + {file = "apache_beam-2.48.0-cp39-cp39-win_amd64.whl", hash = "sha256:0fed22965e1bd593229c00c372dca2aa088b7d6b25b31d595303018fcffb0c37"}, +] + +[package.dependencies] +cloudpickle = ">=2.2.1,<2.3.0" +crcmod = ">=1.7,<2.0" +dill = ">=0.3.1.1,<0.3.2" +fastavro = ">=0.23.6,<2" +fasteners = ">=0.3,<1.0" +grpcio = ">=1.33.1,<1.48.0 || >1.48.0,<2" +hdfs = ">=2.1.0,<3.0.0" +httplib2 = ">=0.8,<0.23.0" +numpy = ">=1.14.3,<1.25.0" +objsize = ">=0.6.1,<0.7.0" +orjson = "<4.0" +proto-plus = ">=1.7.1,<2" +protobuf = ">=3.20.3,<4.24.0" +pyarrow = ">=3.0.0,<12.0.0" +pydot = ">=1.2.0,<2" +pymongo = ">=3.8.0,<5.0.0" +python-dateutil = ">=2.8.0,<3" +pytz = ">=2018.3" +regex = ">=2020.6.8" +requests = ">=2.24.0,<3.0.0" +typing-extensions = ">=3.7.0" +zstandard = ">=0.18.0,<1" + +[package.extras] +aws = ["boto3 (>=1.9,<2)"] +azure = ["azure-core (>=1.7.0,<2)", "azure-identity (>=1.12.0,<2)", "azure-storage-blob (>=12.3.2,<13)"] +dask = ["dask (>=2022.6)", "distributed (>=2022.6)"] +dataframe = ["pandas (<1.6.0)", "pandas (>=1.4.3,!=1.5.0,!=1.5.1,<1.6)"] +docs = ["Sphinx (>=1.5.2,<2.0)", "docutils (==0.17.1)", "pandas (<2.0.0)"] +gcp = ["cachetools (>=3.1.0,<6)", "google-apitools (>=0.5.31,<0.5.32)", "google-auth (>=1.18.0,<3)", "google-auth-httplib2 (>=0.1.0,<0.2.0)", "google-cloud-bigquery (>=2.0.0,<4)", "google-cloud-bigquery-storage (>=2.6.3,<3)", "google-cloud-bigtable (>=2.0.0,<2.18.0)", "google-cloud-core (>=2.0.0,<3)", "google-cloud-datastore (>=2.0.0,<3)", "google-cloud-dlp (>=3.0.0,<4)", "google-cloud-language (>=2.0,<3)", "google-cloud-pubsub (>=2.1.0,<3)", "google-cloud-pubsublite (>=1.2.0,<2)", "google-cloud-recommendations-ai (>=0.1.0,<0.11.0)", "google-cloud-spanner (>=3.0.0,<4)", "google-cloud-videointelligence (>=2.0,<3)", "google-cloud-vision (>=2,<4)"] +interactive = ["facets-overview (>=1.1.0,<2)", "google-cloud-dataproc (>=5.0.0,<6)", "ipykernel (>=6,<7)", "ipython (>=7,<8)", "ipython (>=8,<9)", "ipywidgets (>=8,<9)", "jupyter-client (>=6.1.11,!=6.1.13,<8.2.1)", "pandas (<1.6.0)", "pandas (>=1.4.3,!=1.5.0,!=1.5.1,<1.6)", "timeloop (>=1.0.2,<2)"] +interactive-test = ["chromedriver-binary (>=100,<114)", "nbconvert (>=6.2.0,<8)", "nbformat (>=5.0.5,<6)", "needle (>=0.5.0,<1)", "pillow (>=7.1.1,<10)"] +test = ["cryptography (>=36.0.0)", "freezegun (>=0.3.12)", "hypothesis (>5.0.0,<=7.0.0)", "joblib (>=1.0.1)", "mock (>=1.0.1,<6.0.0)", "pandas (<2.0.0)", "parameterized (>=0.7.1,<0.10.0)", "psycopg2-binary (>=2.8.5,<3.0.0)", "pyhamcrest (>=1.9,!=1.10.0,<3.0.0)", "pytest (>=7.1.2,<8.0)", "pytest-timeout (>=2.1.0,<3)", "pytest-xdist (>=2.5.0,<4)", "pyyaml (>=3.12,<7.0.0)", "requests-mock (>=1.7,<2.0)", "scikit-learn (>=0.20.0)", "sqlalchemy (>=1.3,<2.0)", "tenacity (>=8.0.0,<9)", "testcontainers[mysql] (>=3.0.3,<4.0.0)"] + +[[package]] +name = "apache-flink" +version = "1.18.1" +description = "Apache Flink Python API" +optional = true +python-versions = ">=3.7" +files = [ + {file = "apache-flink-1.18.1.tar.gz", hash = "sha256:5c9457234fe22e828e1b8faf0d1b0b94c05faee4593972eb73459018a94472c8"}, + {file = "apache_flink-1.18.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:e74e89b9e9d0c1cf59f2b1456650da4f14659913e938477c62a1b88245100330"}, + {file = "apache_flink-1.18.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:a8d52c9c3e24829ca4235171857c94cea10fdeaf6896fe66b8b12d3e858b8aeb"}, + {file = "apache_flink-1.18.1-cp310-cp310-manylinux1_x86_64.whl", hash = "sha256:f93cbc4d88e39135dd59d7d7cc2e2252dec4017c4943dd8ab62e117e953a2d15"}, + {file = "apache_flink-1.18.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:6ca2887e7aa6b2fb88c8caf08b994cbdf7bd80bfae43734f539f5962f9c7c488"}, + {file = "apache_flink-1.18.1-cp37-cp37m-manylinux1_x86_64.whl", hash = "sha256:0746d0742105bb3b1f7fce01c63b7071f66b0711a777814c99461debaa3015bd"}, + {file = "apache_flink-1.18.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:0fd51ce47abaf56c12df9f0cda36d823fb6c9998672f81c0e582a3b58453a7c9"}, + {file = "apache_flink-1.18.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:e763006f57b2eda4e0b3b657462796f5f514ef3de5556c7b7905021a2401feaa"}, + {file = "apache_flink-1.18.1-cp38-cp38-manylinux1_x86_64.whl", hash = "sha256:dec50f88ec55e6255746ac940c1a0cd55d7baf689c24d6b174146d072c9e82b6"}, + {file = "apache_flink-1.18.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:21d07d8cf78615d0bfcf4bebd1d1eebc8280b11d0cebe6c403bf0ed76114b9a2"}, + {file = "apache_flink-1.18.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:ecce586aac697f52c7555b1d56c1afd00d460a29b5c69b0de6239798d67271ec"}, + {file = "apache_flink-1.18.1-cp39-cp39-manylinux1_x86_64.whl", hash = "sha256:68c83b80dce53811826c32350c9e534f53dbf62f5e113937dd5bb101e1ab76fb"}, +] + +[package.dependencies] +apache-beam = ">=2.43.0,<2.49.0" +apache-flink-libraries = ">=1.18.1,<1.18.2" +avro-python3 = ">=1.8.1,<1.9.2 || >1.9.2" +cloudpickle = ">=2.2.0" +fastavro = ">=1.1.0,<1.8.0 || >1.8.0" +httplib2 = ">=0.19.0" +numpy = ">=1.21.4" +pandas = ">=1.3.0" +pemja = {version = "0.3.0", markers = "platform_system != \"Windows\""} +protobuf = ">=3.19.0" +py4j = "0.10.9.7" +pyarrow = ">=5.0.0" +python-dateutil = ">=2.8.0,<3" +pytz = ">=2018.3" +requests = ">=2.26.0" -[package.extras] -dev = ["comm (>=0.1.0)", "watchfiles (>=0.18.0)"] -test = ["black[jupyter]", "ipython (<8.13)", "msgspec", "pydantic", "pytest", "pytest-cov", "ruff"] +[[package]] +name = "apache-flink-libraries" +version = "1.18.1" +description = "Apache Flink Libraries" +optional = true +python-versions = ">=3.6" +files = [ + {file = "apache-flink-libraries-1.18.1.tar.gz", hash = "sha256:369ce11ab33e5e15cdd349db2f5412dcc2fad163b6a87aa93b927a3631bcca7f"}, +] [[package]] name = "appdirs" @@ -257,6 +360,20 @@ tests = ["attrs[tests-no-zope]", "zope-interface"] tests-mypy = ["mypy (>=1.6)", "pytest-mypy-plugins"] tests-no-zope = ["attrs[tests-mypy]", "cloudpickle", "hypothesis", "pympler", "pytest (>=4.3.0)", "pytest-xdist[psutil]"] +[[package]] +name = "avro-python3" +version = "1.10.2" +description = "Avro is a serialization and RPC framework." +optional = true +python-versions = ">=3.5" +files = [ + {file = "avro-python3-1.10.2.tar.gz", hash = "sha256:3b63f24e6b04368c3e4a6f923f484be0230d821aad65ac36108edbff29e9aaab"}, +] + +[package.extras] +snappy = ["python-snappy"] +zstandard = ["zstandard"] + [[package]] name = "beartype" version = "0.16.4" @@ -803,7 +920,7 @@ colorama = {version = "*", markers = "platform_system == \"Windows\""} name = "click-plugins" version = "1.1.1" description = "An extension module for click to enable registering CLI commands via setuptools entry-points." -optional = false +optional = true python-versions = "*" files = [ {file = "click-plugins-1.1.1.tar.gz", hash = "sha256:46ab999744a9d831159c3411bb0c79346d94a444df9a3a3742e9ed63645f264b"}, @@ -926,7 +1043,7 @@ sqlalchemy = ["sqlalchemy (>1.3.21,<2.0)"] name = "cligj" version = "0.7.2" description = "Click params for commmand line interfaces to GeoJSON" -optional = false +optional = true python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, <4" files = [ {file = "cligj-0.7.2-py3-none-any.whl", hash = "sha256:c1ca117dbce1fe20a5809dc96f01e1c2840f6dcc939b3ddbb1111bf330ba82df"}, @@ -941,13 +1058,13 @@ test = ["pytest-cov"] [[package]] name = "cloudpickle" -version = "3.0.0" -description = "Pickler class to extend the standard pickle.Pickler functionality" +version = "2.2.1" +description = "Extended pickling support for Python objects" optional = false -python-versions = ">=3.8" +python-versions = ">=3.6" files = [ - {file = "cloudpickle-3.0.0-py3-none-any.whl", hash = "sha256:246ee7d0c295602a036e86369c77fecda4ab17b506496730f2f576d9016fd9c7"}, - {file = "cloudpickle-3.0.0.tar.gz", hash = "sha256:996d9a482c6fb4f33c1a35335cf8afd065d2a56e973270364840712d9131a882"}, + {file = "cloudpickle-2.2.1-py3-none-any.whl", hash = "sha256:61f594d1f4c295fa5cd9014ceb3a1fc4a70b0de1164b94fbc2d854ccba056f9f"}, + {file = "cloudpickle-2.2.1.tar.gz", hash = "sha256:d89684b8de9e34a2a43b3460fbca07d09d6e25ce858df4d5a44240403b6178f5"}, ] [[package]] @@ -1154,6 +1271,16 @@ files = [ {file = "crashtest-0.4.1.tar.gz", hash = "sha256:80d7b1f316ebfbd429f648076d6275c877ba30ba48979de4191714a75266f0ce"}, ] +[[package]] +name = "crcmod" +version = "1.7" +description = "CRC Generator" +optional = true +python-versions = "*" +files = [ + {file = "crcmod-1.7.tar.gz", hash = "sha256:dc7051a0db5f2bd48665a990d3ec1cc305a466a77358ca4492826f41f283601e"}, +] + [[package]] name = "cryptography" version = "41.0.7" @@ -1342,6 +1469,19 @@ devel = ["mypy", "packaging (>=20)", "pytest", "pytest-benchmark", "pytest-cov", pandas = ["pandas"] pyspark = ["delta-spark", "numpy (==1.22.2)", "pyspark"] +[[package]] +name = "dill" +version = "0.3.1.1" +description = "serialize all of python" +optional = true +python-versions = ">=2.6, !=3.0.*" +files = [ + {file = "dill-0.3.1.1.tar.gz", hash = "sha256:42d8ef819367516592a825746a18073ced42ca169ab1f5f4044134703e7a049c"}, +] + +[package.extras] +graph = ["objgraph (>=1.7.2)"] + [[package]] name = "distlib" version = "0.3.8" @@ -1381,6 +1521,36 @@ tornado = ">=6.0.4" urllib3 = ">=1.24.3" zict = ">=3.0.0" +[[package]] +name = "dnspython" +version = "2.5.0" +description = "DNS toolkit" +optional = true +python-versions = ">=3.8" +files = [ + {file = "dnspython-2.5.0-py3-none-any.whl", hash = "sha256:6facdf76b73c742ccf2d07add296f178e629da60be23ce4b0a9c927b1e02c3a6"}, + {file = "dnspython-2.5.0.tar.gz", hash = "sha256:a0034815a59ba9ae888946be7ccca8f7c157b286f8455b379c692efb51022a15"}, +] + +[package.extras] +dev = ["black (>=23.1.0)", "coverage (>=7.0)", "flake8 (>=5.0.3)", "mypy (>=1.0.1)", "pylint (>=2.7)", "pytest (>=6.2.5)", "pytest-cov (>=3.0.0)", "sphinx (>=7.0.0)", "twine (>=4.0.0)", "wheel (>=0.41.0)"] +dnssec = ["cryptography (>=41)"] +doh = ["h2 (>=4.1.0)", "httpcore (>=0.17.3)", "httpx (>=0.25.1)"] +doq = ["aioquic (>=0.9.20)"] +idna = ["idna (>=2.1)"] +trio = ["trio (>=0.14)"] +wmi = ["wmi (>=1.5.1)"] + +[[package]] +name = "docopt" +version = "0.6.2" +description = "Pythonic argument parser, that will make you smile" +optional = true +python-versions = "*" +files = [ + {file = "docopt-0.6.2.tar.gz", hash = "sha256:49b3a825280bd66b3aa83585ef59c4a8c82f2c8a522dbe754a8bc8d08c85c491"}, +] + [[package]] name = "duckdb" version = "0.9.2" @@ -1572,6 +1742,63 @@ files = [ [package.extras] tests = ["asttokens (>=2.1.0)", "coverage", "coverage-enable-subprocess", "ipython", "littleutils", "pytest", "rich"] +[[package]] +name = "fastavro" +version = "1.9.3" +description = "Fast read/write of AVRO files" +optional = true +python-versions = ">=3.8" +files = [ + {file = "fastavro-1.9.3-cp310-cp310-macosx_11_0_x86_64.whl", hash = "sha256:5e9b2e1427fb84c0754bc34923d10cabcf2ed23230201208a1371ab7b6027674"}, + {file = "fastavro-1.9.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c4ef82f86ae276309abc0072598474b6be68105a0b28f8d7cc0398d1d353d7de"}, + {file = "fastavro-1.9.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:280ef7ab7232ecb2097038d6842416ec717d0e1c314b80ff245f85201f3396a4"}, + {file = "fastavro-1.9.3-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:4a36cfc0421ed7576ecb1c22de7bd1dedcce62aebbffcc597379d59171e5d76e"}, + {file = "fastavro-1.9.3-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:d80f2e20199140eb8c036b4393e9bc9eff325543311b958c72318999499d4279"}, + {file = "fastavro-1.9.3-cp310-cp310-win_amd64.whl", hash = "sha256:a435f7edd7c5b52cee3f23ca950cd9373ab35cf2aa3d269b3d6aca7e2fc1372c"}, + {file = "fastavro-1.9.3-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:2a7053ed10194ec53754f5337b57b3273a74b48505edcd6edb79fe3c4cd259c0"}, + {file = "fastavro-1.9.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:853e01f13534d1baa0a3d493a8573e665e93ffa35b4bf1d125e21764d343af8e"}, + {file = "fastavro-1.9.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a5a279cda25d876e6f120950cadf184a307fd8998f9a22a90bb62e6749f88d1e"}, + {file = "fastavro-1.9.3-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:63d6f928840f3fb1f2e1fe20bc8b7d0e1a51ba4bb0e554ecb837a669fba31288"}, + {file = "fastavro-1.9.3-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:8807046edc78f50b3ea5f55f6a534c87b2a13538e7c56fec3532ef802bcae333"}, + {file = "fastavro-1.9.3-cp311-cp311-win_amd64.whl", hash = "sha256:e502579da4a51c5630eadbd811a1b3d262d6e783bf19998cfb33d2ea0cf6f516"}, + {file = "fastavro-1.9.3-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:6b665efe442061df8d9608c2fb692847df85d52ad825b776c441802f0dfa6571"}, + {file = "fastavro-1.9.3-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5b8c96d81f0115633489d7f1133a03832922629a61ca81c1d47b482ddcda3b94"}, + {file = "fastavro-1.9.3-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:338c7ec94dd2474c4679e44d2560a1922cb6fa99acbb7b18957264baf8eadfc7"}, + {file = "fastavro-1.9.3-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:a509b34c9af71a109c633631ac2f6d2209830e13200d0048f7e9c057fd563f8f"}, + {file = "fastavro-1.9.3-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:967edefab470987c024cd5a1fcd04744a50a91e740c7bdf325181043a47f1083"}, + {file = "fastavro-1.9.3-cp312-cp312-win_amd64.whl", hash = "sha256:033c15e8ed02f80f01d58be1cd880b09fd444faf277263d563a727711d47a98a"}, + {file = "fastavro-1.9.3-cp38-cp38-macosx_11_0_x86_64.whl", hash = "sha256:6b38723327603d77080aec56628e13a739415f8596ca0cc41a905615977c6d6b"}, + {file = "fastavro-1.9.3-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:046d75c4400941fd08f0a6855a34ae63bf02ea01f366b5b749942abe10640056"}, + {file = "fastavro-1.9.3-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:87ab312b8baf0e61ee717878d390022ee1b713d70b244d69efbf3325680f9749"}, + {file = "fastavro-1.9.3-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:c562fcf8f5091a2446aafd0c2a0da590c24e0b53527a0100d33908e32f20eea8"}, + {file = "fastavro-1.9.3-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:2aa0111e7ebd076d2a094862bbdf8ea175cebba148fcce6c89ff46b625e334b4"}, + {file = "fastavro-1.9.3-cp38-cp38-win_amd64.whl", hash = "sha256:652072e0f455ca19a1ee502b527e603389783657c130d81f89df66775979d6f5"}, + {file = "fastavro-1.9.3-cp39-cp39-macosx_11_0_x86_64.whl", hash = "sha256:0a57cdd4edaee36d4216faf801ebc7f53f45e4e1518bdd9832d6f6f1d6e2d88f"}, + {file = "fastavro-1.9.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8b46a18ebed61573b0823c28eda2716485d283258a83659c7fe6ad3aaeacfed4"}, + {file = "fastavro-1.9.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5f756f0723f3bd97db20437d0a8e45712839e6ccd7c82f4d82469533be48b4c7"}, + {file = "fastavro-1.9.3-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:d98d5a08063f5b6d7ac5016a0dfe0698b50d9987cb74686f7dfa8288b7b09e0b"}, + {file = "fastavro-1.9.3-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:00698e60db58a2d52cb709df882d451fb7664ebb2f8cb37d9171697e060dc767"}, + {file = "fastavro-1.9.3-cp39-cp39-win_amd64.whl", hash = "sha256:d021bbc135023194688e88a7431fb0b5e3ce20e27153bf258f2ce08ee1a0106b"}, + {file = "fastavro-1.9.3.tar.gz", hash = "sha256:a30d3d2353f6d3b4f6dcd6a97ae937b3775faddd63f5856fe11ba3b0dbb1756a"}, +] + +[package.extras] +codecs = ["cramjam", "lz4", "zstandard"] +lz4 = ["lz4"] +snappy = ["cramjam"] +zstandard = ["zstandard"] + +[[package]] +name = "fasteners" +version = "0.19" +description = "A python package that provides useful locks" +optional = true +python-versions = ">=3.6" +files = [ + {file = "fasteners-0.19-py3-none-any.whl", hash = "sha256:758819cb5d94cdedf4e836988b74de396ceacb8e2794d21f82d131fd9ee77237"}, + {file = "fasteners-0.19.tar.gz", hash = "sha256:b4f37c3ac52d8a445af3a66bce57b33b5e90b97c696b7b984f530cf8f0ded09c"}, +] + [[package]] name = "fastjsonschema" version = "2.19.1" @@ -1602,11 +1829,21 @@ docs = ["furo (>=2023.9.10)", "sphinx (>=7.2.6)", "sphinx-autodoc-typehints (>=1 testing = ["covdefaults (>=2.3)", "coverage (>=7.3.2)", "diff-cover (>=8)", "pytest (>=7.4.3)", "pytest-cov (>=4.1)", "pytest-mock (>=3.12)", "pytest-timeout (>=2.2)"] typing = ["typing-extensions (>=4.8)"] +[[package]] +name = "find-libpython" +version = "0.3.1" +description = "Finds the libpython associated with your environment, wherever it may be hiding" +optional = true +python-versions = "*" +files = [ + {file = "find_libpython-0.3.1-py3-none-any.whl", hash = "sha256:f63c2c05b9c6077bfafa5c7b283547b918a54c17fd930ceead487d1f220ff9cb"}, +] + [[package]] name = "fiona" version = "1.9.5" description = "Fiona reads and writes spatial data files" -optional = false +optional = true python-versions = ">=3.7" files = [ {file = "fiona-1.9.5-cp310-cp310-macosx_10_15_x86_64.whl", hash = "sha256:5f40a40529ecfca5294260316cf987a0420c77a2f0cf0849f529d1afbccd093e"}, @@ -1915,7 +2152,7 @@ files = [ name = "geopandas" version = "0.14.2" description = "Geographic pandas extensions" -optional = false +optional = true python-versions = ">=3.9" files = [ {file = "geopandas-0.14.2-py3-none-any.whl", hash = "sha256:0efa61235a68862c1c6be89fc3707cdeba67667d5676bb19e24f3c57a8c2f723"}, @@ -2392,6 +2629,40 @@ googleapis-common-protos = ">=1.5.5" grpcio = ">=1.60.0" protobuf = ">=4.21.6" +[[package]] +name = "hdfs" +version = "2.7.3" +description = "HdfsCLI: API and command line interface for HDFS." +optional = true +python-versions = "*" +files = [ + {file = "hdfs-2.7.3.tar.gz", hash = "sha256:752a21e43f82197dce43697c73f454ba490838108c73a57a9247efb66d1c0479"}, +] + +[package.dependencies] +docopt = "*" +requests = ">=2.7.0" +six = ">=1.9.0" + +[package.extras] +avro = ["fastavro (>=0.21.19)"] +dataframe = ["fastavro (>=0.21.19)", "pandas (>=0.14.1)"] +kerberos = ["requests-kerberos (>=0.7.0)"] + +[[package]] +name = "httplib2" +version = "0.22.0" +description = "A comprehensive HTTP client library." +optional = true +python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*" +files = [ + {file = "httplib2-0.22.0-py3-none-any.whl", hash = "sha256:14ae0a53c1ba8f3d37e9e27cf37eabb0fb9980f435ba405d546948b009dd64dc"}, + {file = "httplib2-0.22.0.tar.gz", hash = "sha256:d7a10bc5ef5ab08322488bde8c726eeee5c8618723fdb399597ec58f3d82df81"}, +] + +[package.dependencies] +pyparsing = {version = ">=2.4.2,<3.0.0 || >3.0.0,<3.0.1 || >3.0.1,<3.0.2 || >3.0.2,<3.0.3 || >3.0.3,<4", markers = "python_version > \"3.0\""} + [[package]] name = "humanize" version = "4.9.0" @@ -3074,26 +3345,6 @@ files = [ {file = "locket-1.0.0.tar.gz", hash = "sha256:5c0d4c052a8bbbf750e056a8e65ccd309086f4f0f18a2eac306a8dfa4112a632"}, ] -[[package]] -name = "lonboard" -version = "0.5.0" -description = "Python library for fast, interactive geospatial vector data visualization in Jupyter." -optional = false -python-versions = ">=3.8,<4.0" -files = [ - {file = "lonboard-0.5.0-py3-none-any.whl", hash = "sha256:f62590c5a2562cc21653253d0a167dcbd2e0f2d3a72a6344e112c0f959e7feee"}, - {file = "lonboard-0.5.0.tar.gz", hash = "sha256:70d7d347a3175c53a257175f381072cc633e4cb058534aaff6a5725c22cd0095"}, -] - -[package.dependencies] -anywidget = ">=0.7.1,<0.8.0" -geopandas = ">=0.13" -matplotlib = ">=3.7,<4.0" -palettable = ">=3.3.3,<4.0.0" -pandas = ">=2,<3" -pyarrow = ">=14.0.1" -shapely = ">=2,<3" - [[package]] name = "lz4" version = "4.3.3" @@ -3510,7 +3761,7 @@ files = [ name = "mypy-extensions" version = "1.0.0" description = "Type system extensions for programs checked with the mypy type checker." -optional = false +optional = true python-versions = ">=3.5" files = [ {file = "mypy_extensions-1.0.0-py3-none-any.whl", hash = "sha256:4392f6c0eb8a5668a69e23d168ffa70f0be9ccfd32b5cc2d26a34ae5b844552d"}, @@ -3587,47 +3838,39 @@ setuptools = "*" [[package]] name = "numpy" -version = "1.26.3" +version = "1.24.4" description = "Fundamental package for array computing in Python" optional = false -python-versions = ">=3.9" +python-versions = ">=3.8" files = [ - {file = "numpy-1.26.3-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:806dd64230dbbfaca8a27faa64e2f414bf1c6622ab78cc4264f7f5f028fee3bf"}, - {file = "numpy-1.26.3-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:02f98011ba4ab17f46f80f7f8f1c291ee7d855fcef0a5a98db80767a468c85cd"}, - {file = "numpy-1.26.3-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6d45b3ec2faed4baca41c76617fcdcfa4f684ff7a151ce6fc78ad3b6e85af0a6"}, - {file = "numpy-1.26.3-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:bdd2b45bf079d9ad90377048e2747a0c82351989a2165821f0c96831b4a2a54b"}, - {file = "numpy-1.26.3-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:211ddd1e94817ed2d175b60b6374120244a4dd2287f4ece45d49228b4d529178"}, - {file = "numpy-1.26.3-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:b1240f767f69d7c4c8a29adde2310b871153df9b26b5cb2b54a561ac85146485"}, - {file = "numpy-1.26.3-cp310-cp310-win32.whl", hash = "sha256:21a9484e75ad018974a2fdaa216524d64ed4212e418e0a551a2d83403b0531d3"}, - {file = "numpy-1.26.3-cp310-cp310-win_amd64.whl", hash = "sha256:9e1591f6ae98bcfac2a4bbf9221c0b92ab49762228f38287f6eeb5f3f55905ce"}, - {file = "numpy-1.26.3-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:b831295e5472954104ecb46cd98c08b98b49c69fdb7040483aff799a755a7374"}, - {file = "numpy-1.26.3-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:9e87562b91f68dd8b1c39149d0323b42e0082db7ddb8e934ab4c292094d575d6"}, - {file = "numpy-1.26.3-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8c66d6fec467e8c0f975818c1796d25c53521124b7cfb760114be0abad53a0a2"}, - {file = "numpy-1.26.3-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f25e2811a9c932e43943a2615e65fc487a0b6b49218899e62e426e7f0a57eeda"}, - {file = "numpy-1.26.3-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:af36e0aa45e25c9f57bf684b1175e59ea05d9a7d3e8e87b7ae1a1da246f2767e"}, - {file = "numpy-1.26.3-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:51c7f1b344f302067b02e0f5b5d2daa9ed4a721cf49f070280ac202738ea7f00"}, - {file = "numpy-1.26.3-cp311-cp311-win32.whl", hash = "sha256:7ca4f24341df071877849eb2034948459ce3a07915c2734f1abb4018d9c49d7b"}, - {file = "numpy-1.26.3-cp311-cp311-win_amd64.whl", hash = "sha256:39763aee6dfdd4878032361b30b2b12593fb445ddb66bbac802e2113eb8a6ac4"}, - {file = "numpy-1.26.3-cp312-cp312-macosx_10_9_x86_64.whl", hash = "sha256:a7081fd19a6d573e1a05e600c82a1c421011db7935ed0d5c483e9dd96b99cf13"}, - {file = "numpy-1.26.3-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:12c70ac274b32bc00c7f61b515126c9205323703abb99cd41836e8125ea0043e"}, - {file = "numpy-1.26.3-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:7f784e13e598e9594750b2ef6729bcd5a47f6cfe4a12cca13def35e06d8163e3"}, - {file = "numpy-1.26.3-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5f24750ef94d56ce6e33e4019a8a4d68cfdb1ef661a52cdaee628a56d2437419"}, - {file = "numpy-1.26.3-cp312-cp312-musllinux_1_1_aarch64.whl", hash = "sha256:77810ef29e0fb1d289d225cabb9ee6cf4d11978a00bb99f7f8ec2132a84e0166"}, - {file = "numpy-1.26.3-cp312-cp312-musllinux_1_1_x86_64.whl", hash = "sha256:8ed07a90f5450d99dad60d3799f9c03c6566709bd53b497eb9ccad9a55867f36"}, - {file = "numpy-1.26.3-cp312-cp312-win32.whl", hash = "sha256:f73497e8c38295aaa4741bdfa4fda1a5aedda5473074369eca10626835445511"}, - {file = "numpy-1.26.3-cp312-cp312-win_amd64.whl", hash = "sha256:da4b0c6c699a0ad73c810736303f7fbae483bcb012e38d7eb06a5e3b432c981b"}, - {file = "numpy-1.26.3-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:1666f634cb3c80ccbd77ec97bc17337718f56d6658acf5d3b906ca03e90ce87f"}, - {file = "numpy-1.26.3-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:18c3319a7d39b2c6a9e3bb75aab2304ab79a811ac0168a671a62e6346c29b03f"}, - {file = "numpy-1.26.3-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0b7e807d6888da0db6e7e75838444d62495e2b588b99e90dd80c3459594e857b"}, - {file = "numpy-1.26.3-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:b4d362e17bcb0011738c2d83e0a65ea8ce627057b2fdda37678f4374a382a137"}, - {file = "numpy-1.26.3-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:b8c275f0ae90069496068c714387b4a0eba5d531aace269559ff2b43655edd58"}, - {file = "numpy-1.26.3-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:cc0743f0302b94f397a4a65a660d4cd24267439eb16493fb3caad2e4389bccbb"}, - {file = "numpy-1.26.3-cp39-cp39-win32.whl", hash = "sha256:9bc6d1a7f8cedd519c4b7b1156d98e051b726bf160715b769106661d567b3f03"}, - {file = "numpy-1.26.3-cp39-cp39-win_amd64.whl", hash = "sha256:867e3644e208c8922a3be26fc6bbf112a035f50f0a86497f98f228c50c607bb2"}, - {file = "numpy-1.26.3-pp39-pypy39_pp73-macosx_10_9_x86_64.whl", hash = "sha256:3c67423b3703f8fbd90f5adaa37f85b5794d3366948efe9a5190a5f3a83fc34e"}, - {file = "numpy-1.26.3-pp39-pypy39_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:46f47ee566d98849323f01b349d58f2557f02167ee301e5e28809a8c0e27a2d0"}, - {file = "numpy-1.26.3-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:a8474703bffc65ca15853d5fd4d06b18138ae90c17c8d12169968e998e448bb5"}, - {file = "numpy-1.26.3.tar.gz", hash = "sha256:697df43e2b6310ecc9d95f05d5ef20eacc09c7c4ecc9da3f235d39e71b7da1e4"}, + {file = "numpy-1.24.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c0bfb52d2169d58c1cdb8cc1f16989101639b34c7d3ce60ed70b19c63eba0b64"}, + {file = "numpy-1.24.4-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:ed094d4f0c177b1b8e7aa9cba7d6ceed51c0e569a5318ac0ca9a090680a6a1b1"}, + {file = "numpy-1.24.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:79fc682a374c4a8ed08b331bef9c5f582585d1048fa6d80bc6c35bc384eee9b4"}, + {file = "numpy-1.24.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7ffe43c74893dbf38c2b0a1f5428760a1a9c98285553c89e12d70a96a7f3a4d6"}, + {file = "numpy-1.24.4-cp310-cp310-win32.whl", hash = "sha256:4c21decb6ea94057331e111a5bed9a79d335658c27ce2adb580fb4d54f2ad9bc"}, + {file = "numpy-1.24.4-cp310-cp310-win_amd64.whl", hash = "sha256:b4bea75e47d9586d31e892a7401f76e909712a0fd510f58f5337bea9572c571e"}, + {file = "numpy-1.24.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:f136bab9c2cfd8da131132c2cf6cc27331dd6fae65f95f69dcd4ae3c3639c810"}, + {file = "numpy-1.24.4-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:e2926dac25b313635e4d6cf4dc4e51c8c0ebfed60b801c799ffc4c32bf3d1254"}, + {file = "numpy-1.24.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:222e40d0e2548690405b0b3c7b21d1169117391c2e82c378467ef9ab4c8f0da7"}, + {file = "numpy-1.24.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:7215847ce88a85ce39baf9e89070cb860c98fdddacbaa6c0da3ffb31b3350bd5"}, + {file = "numpy-1.24.4-cp311-cp311-win32.whl", hash = "sha256:4979217d7de511a8d57f4b4b5b2b965f707768440c17cb70fbf254c4b225238d"}, + {file = "numpy-1.24.4-cp311-cp311-win_amd64.whl", hash = "sha256:b7b1fc9864d7d39e28f41d089bfd6353cb5f27ecd9905348c24187a768c79694"}, + {file = "numpy-1.24.4-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:1452241c290f3e2a312c137a9999cdbf63f78864d63c79039bda65ee86943f61"}, + {file = "numpy-1.24.4-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:04640dab83f7c6c85abf9cd729c5b65f1ebd0ccf9de90b270cd61935eef0197f"}, + {file = "numpy-1.24.4-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a5425b114831d1e77e4b5d812b69d11d962e104095a5b9c3b641a218abcc050e"}, + {file = "numpy-1.24.4-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:dd80e219fd4c71fc3699fc1dadac5dcf4fd882bfc6f7ec53d30fa197b8ee22dc"}, + {file = "numpy-1.24.4-cp38-cp38-win32.whl", hash = "sha256:4602244f345453db537be5314d3983dbf5834a9701b7723ec28923e2889e0bb2"}, + {file = "numpy-1.24.4-cp38-cp38-win_amd64.whl", hash = "sha256:692f2e0f55794943c5bfff12b3f56f99af76f902fc47487bdfe97856de51a706"}, + {file = "numpy-1.24.4-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:2541312fbf09977f3b3ad449c4e5f4bb55d0dbf79226d7724211acc905049400"}, + {file = "numpy-1.24.4-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:9667575fb6d13c95f1b36aca12c5ee3356bf001b714fc354eb5465ce1609e62f"}, + {file = "numpy-1.24.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f3a86ed21e4f87050382c7bc96571755193c4c1392490744ac73d660e8f564a9"}, + {file = "numpy-1.24.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d11efb4dbecbdf22508d55e48d9c8384db795e1b7b51ea735289ff96613ff74d"}, + {file = "numpy-1.24.4-cp39-cp39-win32.whl", hash = "sha256:6620c0acd41dbcb368610bb2f4d83145674040025e5536954782467100aa8835"}, + {file = "numpy-1.24.4-cp39-cp39-win_amd64.whl", hash = "sha256:befe2bf740fd8373cf56149a5c23a0f601e82869598d41f8e188a0e9869926f8"}, + {file = "numpy-1.24.4-pp38-pypy38_pp73-macosx_10_9_x86_64.whl", hash = "sha256:31f13e25b4e304632a4619d0e0777662c2ffea99fcae2029556b17d8ff958aef"}, + {file = "numpy-1.24.4-pp38-pypy38_pp73-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:95f7ac6540e95bc440ad77f56e520da5bf877f87dca58bd095288dce8940532a"}, + {file = "numpy-1.24.4-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:e98f220aa76ca2a977fe435f5b04d7b3470c0a2e6312907b37ba6068f26787f2"}, + {file = "numpy-1.24.4.tar.gz", hash = "sha256:80f5e3a4e498641401868df4208b74581206afbee7cf7b8329daae82676d9463"}, ] [[package]] @@ -3646,6 +3889,20 @@ rsa = ["cryptography (>=3.0.0)"] signals = ["blinker (>=1.4.0)"] signedtoken = ["cryptography (>=3.0.0)", "pyjwt (>=2.0.0,<3)"] +[[package]] +name = "objsize" +version = "0.6.1" +description = "Traversal over Python's objects subtree and calculate the total size of the subtree in bytes (deep size)." +optional = true +python-versions = ">=3.7" +files = [ + {file = "objsize-0.6.1-py3-none-any.whl", hash = "sha256:807dde91898f93b2308be4cfca3e863f689f578a81a4d75da6bb184e6c5f105d"}, + {file = "objsize-0.6.1.tar.gz", hash = "sha256:56482d468acdc445c7da05a56e5b05ad38e5c0a27a761b5d8c538df494e19d2a"}, +] + +[package.extras] +dev = ["black", "bumpver", "coveralls", "isort", "pip-tools", "pytest", "pytest-cov"] + [[package]] name = "oracledb" version = "2.0.1" @@ -3689,6 +3946,65 @@ files = [ [package.dependencies] cryptography = ">=3.2.1" +[[package]] +name = "orjson" +version = "3.9.13" +description = "Fast, correct Python JSON library supporting dataclasses, datetimes, and numpy" +optional = true +python-versions = ">=3.8" +files = [ + {file = "orjson-3.9.13-cp310-cp310-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:fa6b67f8bef277c2a4aadd548d58796854e7d760964126c3209b19bccc6a74f1"}, + {file = "orjson-3.9.13-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b812417199eeb169c25f67815cfb66fd8de7ff098bf57d065e8c1943a7ba5c8f"}, + {file = "orjson-3.9.13-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:7ccd5bd222e5041069ad9d9868ab59e6dbc53ecde8d8c82b919954fbba43b46b"}, + {file = "orjson-3.9.13-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:eaaf80957c38e9d3f796f355a80fad945e72cd745e6b64c210e635b7043b673e"}, + {file = "orjson-3.9.13-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:60da7316131185d0110a1848e9ad15311e6c8938ee0b5be8cbd7261e1d80ee8f"}, + {file = "orjson-3.9.13-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5b98cd948372f0eb219bc309dee4633db1278687161e3280d9e693b6076951d2"}, + {file = "orjson-3.9.13-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:3869d65561f10071d3e7f35ae58fd377056f67d7aaed5222f318390c3ad30339"}, + {file = "orjson-3.9.13-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:43fd6036b16bb6742d03dae62f7bdf8214d06dea47e4353cde7e2bd1358d186f"}, + {file = "orjson-3.9.13-cp310-none-win32.whl", hash = "sha256:0d3ba9d88e20765335260d7b25547d7c571eee2b698200f97afa7d8c7cd668fc"}, + {file = "orjson-3.9.13-cp310-none-win_amd64.whl", hash = "sha256:6e47153db080f5e87e8ba638f1a8b18995eede6b0abb93964d58cf11bcea362f"}, + {file = "orjson-3.9.13-cp311-cp311-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:4584e8eb727bc431baaf1bf97e35a1d8a0109c924ec847395673dfd5f4ef6d6f"}, + {file = "orjson-3.9.13-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2f37f0cdd026ef777a4336e599d8194c8357fc14760c2a5ddcfdf1965d45504b"}, + {file = "orjson-3.9.13-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:d714595d81efab11b42bccd119977d94b25d12d3a806851ff6bfd286a4bce960"}, + {file = "orjson-3.9.13-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9171e8e1a1f221953e38e84ae0abffe8759002fd8968106ee379febbb5358b33"}, + {file = "orjson-3.9.13-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1ab9dbdec3f13f3ea6f937564ce21651844cfbf2725099f2f490426acf683c23"}, + {file = "orjson-3.9.13-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:811ac076855e33e931549340288e0761873baf29276ad00f221709933c644330"}, + {file = "orjson-3.9.13-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:860d0f5b42d0c0afd73fa4177709f6e1b966ba691fcd72175affa902052a81d6"}, + {file = "orjson-3.9.13-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:838b898e8c1f26eb6b8d81b180981273f6f5110c76c22c384979aca854194f1b"}, + {file = "orjson-3.9.13-cp311-none-win32.whl", hash = "sha256:d3222db9df629ef3c3673124f2e05fb72bc4a320c117e953fec0d69dde82e36d"}, + {file = "orjson-3.9.13-cp311-none-win_amd64.whl", hash = "sha256:978117122ca4cc59b28af5322253017f6c5fc03dbdda78c7f4b94ae984c8dd43"}, + {file = "orjson-3.9.13-cp312-cp312-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:031df1026c7ea8303332d78711f180231e3ae8b564271fb748a03926587c5546"}, + {file = "orjson-3.9.13-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0fd9a2101d04e85086ea6198786a3f016e45475f800712e6833e14bf9ce2832f"}, + {file = "orjson-3.9.13-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:446d9ad04204e79229ae19502daeea56479e55cbc32634655d886f5a39e91b44"}, + {file = "orjson-3.9.13-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:b57c0954a9fdd2b05b9cec0f5a12a0bdce5bf021a5b3b09323041613972481ab"}, + {file = "orjson-3.9.13-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:266e55c83f81248f63cc93d11c5e3a53df49a5d2598fa9e9db5f99837a802d5d"}, + {file = "orjson-3.9.13-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:31372ba3a9fe8ad118e7d22fba46bbc18e89039e3bfa89db7bc8c18ee722dca8"}, + {file = "orjson-3.9.13-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:e3b0c4da61f39899561e08e571f54472a09fa71717d9797928af558175ae5243"}, + {file = "orjson-3.9.13-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:2cc03a35bfc71c8ebf96ce49b82c2a7be6af4b3cd3ac34166fdb42ac510bbfff"}, + {file = "orjson-3.9.13-cp312-none-win_amd64.whl", hash = "sha256:49b7e3fe861cb246361825d1a238f2584ed8ea21e714bf6bb17cebb86772e61c"}, + {file = "orjson-3.9.13-cp38-cp38-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:62e9a99879c4d5a04926ac2518a992134bfa00d546ea5a4cae4b9be454d35a22"}, + {file = "orjson-3.9.13-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:d92a3e835a5100f1d5b566fff79217eab92223ca31900dba733902a182a35ab0"}, + {file = "orjson-3.9.13-cp38-cp38-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:23f21faf072ed3b60b5954686f98157e073f6a8068eaa58dbde83e87212eda84"}, + {file = "orjson-3.9.13-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:828c502bb261588f7de897e06cb23c4b122997cb039d2014cb78e7dabe92ef0c"}, + {file = "orjson-3.9.13-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:16946d095212a3dec552572c5d9bca7afa40f3116ad49695a397be07d529f1fa"}, + {file = "orjson-3.9.13-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3deadd8dc0e9ff844b5b656fa30a48dbee1c3b332d8278302dd9637f6b09f627"}, + {file = "orjson-3.9.13-cp38-cp38-musllinux_1_2_aarch64.whl", hash = "sha256:9b1b5adc5adf596c59dca57156b71ad301d73956f5bab4039b0e34dbf50b9fa0"}, + {file = "orjson-3.9.13-cp38-cp38-musllinux_1_2_x86_64.whl", hash = "sha256:ddc089315d030c54f0f03fb38286e2667c05009a78d659f108a8efcfbdf2e585"}, + {file = "orjson-3.9.13-cp38-none-win32.whl", hash = "sha256:ae77275a28667d9c82d4522b681504642055efa0368d73108511647c6499b31c"}, + {file = "orjson-3.9.13-cp38-none-win_amd64.whl", hash = "sha256:730385fdb99a21fce9bb84bb7fcbda72c88626facd74956bda712834b480729d"}, + {file = "orjson-3.9.13-cp39-cp39-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:7e8e4a571d958910272af8d53a9cbe6599f9f5fd496a1bc51211183bb2072cbd"}, + {file = "orjson-3.9.13-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cfad553a36548262e7da0f3a7464270e13900b898800fb571a5d4b298c3f8356"}, + {file = "orjson-3.9.13-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:0d691c44604941945b00e0a13b19a7d9c1a19511abadf0080f373e98fdeb6b31"}, + {file = "orjson-3.9.13-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a8c83718346de08d68b3cb1105c5d91e5fc39885d8610fdda16613d4e3941459"}, + {file = "orjson-3.9.13-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:63ef57a53bfc2091a7cd50a640d9ae866bd7d92a5225a1bab6baa60ef62583f2"}, + {file = "orjson-3.9.13-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:9156b96afa38db71344522f5517077eaedf62fcd2c9148392ff93d801128809c"}, + {file = "orjson-3.9.13-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:31fb66b41fb2c4c817d9610f0bc7d31345728d7b5295ac78b63603407432a2b2"}, + {file = "orjson-3.9.13-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:8a730bf07feacb0863974e67b206b7c503a62199de1cece2eb0d4c233ec29c11"}, + {file = "orjson-3.9.13-cp39-none-win32.whl", hash = "sha256:5ef58869f3399acbbe013518d8b374ee9558659eef14bca0984f67cb1fbd3c37"}, + {file = "orjson-3.9.13-cp39-none-win_amd64.whl", hash = "sha256:9bcf56efdb83244cde070e82a69c0f03c47c235f0a5cb6c81d9da23af7fbaae4"}, + {file = "orjson-3.9.13.tar.gz", hash = "sha256:fc6bc65b0cf524ee042e0bc2912b9206ef242edfba7426cf95763e4af01f527a"}, +] + [[package]] name = "packaging" version = "23.2" @@ -3700,17 +4016,6 @@ files = [ {file = "packaging-23.2.tar.gz", hash = "sha256:048fb0e9405036518eaaf48a55953c750c11e1a1b68e0dd1a9d62ed0c092cfc5"}, ] -[[package]] -name = "palettable" -version = "3.3.3" -description = "Color palettes for Python" -optional = false -python-versions = ">=3.7" -files = [ - {file = "palettable-3.3.3-py2.py3-none-any.whl", hash = "sha256:74e9e7d7fe5a9be065e02397558ed1777b2df0b793a6f4ce1a5ee74f74fb0caa"}, - {file = "palettable-3.3.3.tar.gz", hash = "sha256:094dd7d9a5fc1cca4854773e5c1fc6a315b33bd5b3a8f47064928facaf0490a8"}, -] - [[package]] name = "pandas" version = "2.0.3" @@ -3851,6 +4156,30 @@ six = "*" [package.extras] test = ["pytest", "pytest-cov", "scipy"] +[[package]] +name = "pemja" +version = "0.3.0" +description = "PemJa" +optional = true +python-versions = ">=3.7" +files = [ + {file = "pemja-0.3.0-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:9ff2f0dabfb725676c260b1e866a18757756f3700ec46f85e87570713d972dd5"}, + {file = "pemja-0.3.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:2102e913bb1b8f6fcec67a6da0ff0e245ee454ffe2525af7be2d6df58ab71579"}, + {file = "pemja-0.3.0-cp310-cp310-manylinux1_x86_64.whl", hash = "sha256:a1eba60d7cefb633014268acd9024093a4ec588978c9786d71962ebe50e80867"}, + {file = "pemja-0.3.0-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:fe04522c3377b1c902148c9da1c48e3c0c78d3f16e024c68b7aa103a1f04dbef"}, + {file = "pemja-0.3.0-cp37-cp37m-manylinux1_x86_64.whl", hash = "sha256:1e1cdc5975765e06004e7dde82bcdf8cba55bf9c17a4f21ea0035bcd2839f24f"}, + {file = "pemja-0.3.0-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:b1cc5720a3a7710a0b7e8870f24ce009c41ba4562e939c4957aae5d85bd2ab72"}, + {file = "pemja-0.3.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:56916c374da19277060227f38f0eebbe4eedcd1a4222f2dd46b531a590b9a7b9"}, + {file = "pemja-0.3.0-cp38-cp38-manylinux1_x86_64.whl", hash = "sha256:cdf9ad292f354fbf14642a5343a52df14ecafb9ad0fb29409c6742e42eaa12ad"}, + {file = "pemja-0.3.0-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:f6544391a3df38593b89f27c376c19a170e697f7d76c946c777e1459f5cab554"}, + {file = "pemja-0.3.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:7ac0ad95b55d085e4664682a99ad65749dee6f5236303607c0781115efd9be84"}, + {file = "pemja-0.3.0-cp39-cp39-manylinux1_x86_64.whl", hash = "sha256:989bd4e66bd69019969a67638d6b1b44f3de456628fad5f76ed2e9e6137caad0"}, + {file = "pemja-0.3.0.tar.gz", hash = "sha256:8a468de79b31fda84cb1ad1c67df5d7214d27924f1a30aa1133af523656ab5fd"}, +] + +[package.dependencies] +find-libpython = "*" + [[package]] name = "pexpect" version = "4.9.0" @@ -4270,22 +4599,24 @@ testing = ["google-api-core[grpc] (>=1.31.5)"] [[package]] name = "protobuf" -version = "4.25.2" +version = "4.23.4" description = "" optional = false -python-versions = ">=3.8" +python-versions = ">=3.7" files = [ - {file = "protobuf-4.25.2-cp310-abi3-win32.whl", hash = "sha256:b50c949608682b12efb0b2717f53256f03636af5f60ac0c1d900df6213910fd6"}, - {file = "protobuf-4.25.2-cp310-abi3-win_amd64.whl", hash = "sha256:8f62574857ee1de9f770baf04dde4165e30b15ad97ba03ceac65f760ff018ac9"}, - {file = "protobuf-4.25.2-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:2db9f8fa64fbdcdc93767d3cf81e0f2aef176284071507e3ede160811502fd3d"}, - {file = "protobuf-4.25.2-cp37-abi3-manylinux2014_aarch64.whl", hash = "sha256:10894a2885b7175d3984f2be8d9850712c57d5e7587a2410720af8be56cdaf62"}, - {file = "protobuf-4.25.2-cp37-abi3-manylinux2014_x86_64.whl", hash = "sha256:fc381d1dd0516343f1440019cedf08a7405f791cd49eef4ae1ea06520bc1c020"}, - {file = "protobuf-4.25.2-cp38-cp38-win32.whl", hash = "sha256:33a1aeef4b1927431d1be780e87b641e322b88d654203a9e9d93f218ee359e61"}, - {file = "protobuf-4.25.2-cp38-cp38-win_amd64.whl", hash = "sha256:47f3de503fe7c1245f6f03bea7e8d3ec11c6c4a2ea9ef910e3221c8a15516d62"}, - {file = "protobuf-4.25.2-cp39-cp39-win32.whl", hash = "sha256:5e5c933b4c30a988b52e0b7c02641760a5ba046edc5e43d3b94a74c9fc57c1b3"}, - {file = "protobuf-4.25.2-cp39-cp39-win_amd64.whl", hash = "sha256:d66a769b8d687df9024f2985d5137a337f957a0916cf5464d1513eee96a63ff0"}, - {file = "protobuf-4.25.2-py3-none-any.whl", hash = "sha256:a8b7a98d4ce823303145bf3c1a8bdb0f2f4642a414b196f04ad9853ed0c8f830"}, - {file = "protobuf-4.25.2.tar.gz", hash = "sha256:fe599e175cb347efc8ee524bcd4b902d11f7262c0e569ececcb89995c15f0a5e"}, + {file = "protobuf-4.23.4-cp310-abi3-win32.whl", hash = "sha256:5fea3c64d41ea5ecf5697b83e41d09b9589e6f20b677ab3c48e5f242d9b7897b"}, + {file = "protobuf-4.23.4-cp310-abi3-win_amd64.whl", hash = "sha256:7b19b6266d92ca6a2a87effa88ecc4af73ebc5cfde194dc737cf8ef23a9a3b12"}, + {file = "protobuf-4.23.4-cp37-abi3-macosx_10_9_universal2.whl", hash = "sha256:8547bf44fe8cec3c69e3042f5c4fb3e36eb2a7a013bb0a44c018fc1e427aafbd"}, + {file = "protobuf-4.23.4-cp37-abi3-manylinux2014_aarch64.whl", hash = "sha256:fee88269a090ada09ca63551bf2f573eb2424035bcf2cb1b121895b01a46594a"}, + {file = "protobuf-4.23.4-cp37-abi3-manylinux2014_x86_64.whl", hash = "sha256:effeac51ab79332d44fba74660d40ae79985901ac21bca408f8dc335a81aa597"}, + {file = "protobuf-4.23.4-cp37-cp37m-win32.whl", hash = "sha256:c3e0939433c40796ca4cfc0fac08af50b00eb66a40bbbc5dee711998fb0bbc1e"}, + {file = "protobuf-4.23.4-cp37-cp37m-win_amd64.whl", hash = "sha256:9053df6df8e5a76c84339ee4a9f5a2661ceee4a0dab019e8663c50ba324208b0"}, + {file = "protobuf-4.23.4-cp38-cp38-win32.whl", hash = "sha256:e1c915778d8ced71e26fcf43c0866d7499891bca14c4368448a82edc61fdbc70"}, + {file = "protobuf-4.23.4-cp38-cp38-win_amd64.whl", hash = "sha256:351cc90f7d10839c480aeb9b870a211e322bf05f6ab3f55fcb2f51331f80a7d2"}, + {file = "protobuf-4.23.4-cp39-cp39-win32.whl", hash = "sha256:6dd9b9940e3f17077e820b75851126615ee38643c2c5332aa7a359988820c720"}, + {file = "protobuf-4.23.4-cp39-cp39-win_amd64.whl", hash = "sha256:0a5759f5696895de8cc913f084e27fd4125e8fb0914bb729a17816a33819f474"}, + {file = "protobuf-4.23.4-py3-none-any.whl", hash = "sha256:e9d0be5bf34b275b9f87ba7407796556abeeba635455d036c7351f7c183ef8ff"}, + {file = "protobuf-4.23.4.tar.gz", hash = "sha256:ccd9430c0719dce806b93f89c91de7977304729e55377f872a92465d548329a9"}, ] [[package]] @@ -4338,53 +4669,6 @@ files = [ {file = "psycopg2-2.9.9.tar.gz", hash = "sha256:d1454bde93fb1e224166811694d600e746430c006fbb031ea06ecc2ea41bf156"}, ] -[[package]] -name = "psygnal" -version = "0.9.5" -description = "Fast python callback/event system modeled after Qt Signals" -optional = false -python-versions = ">=3.7" -files = [ - {file = "psygnal-0.9.5-cp310-cp310-macosx_10_16_arm64.whl", hash = "sha256:54a55f5193c070f683542c373028084c0a1f8fa247f4500c66a01b17f96f4314"}, - {file = "psygnal-0.9.5-cp310-cp310-macosx_10_16_x86_64.whl", hash = "sha256:b9f6b5cea9b11fd14ed1ac78eef33c4332aebc4862736883960e7e513404b7ba"}, - {file = "psygnal-0.9.5-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0a3ff86f5c20469419e67a00ad913d619afe10a488e2abbb9fff8683f11bbaef"}, - {file = "psygnal-0.9.5-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:39890829750905553776160746a9433ed6c7b9cdf4064b7d5e1d958b808b37d7"}, - {file = "psygnal-0.9.5-cp310-cp310-win_amd64.whl", hash = "sha256:43160c91eef076d83f25fdf38460ad23c6f78a2740988eee335b50aeea566ac2"}, - {file = "psygnal-0.9.5-cp311-cp311-macosx_10_16_arm64.whl", hash = "sha256:5412575ea2db2b0aa783463185cb10d1ccd227fbc29fd971aabacca1fa97113e"}, - {file = "psygnal-0.9.5-cp311-cp311-macosx_10_16_x86_64.whl", hash = "sha256:6d41e16a19e98b79431c33183611e961ecc74610987249e52bae097774f3aa71"}, - {file = "psygnal-0.9.5-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:05cc3b4b253ea6d8efda736de89a6174c1b958a359f78a7842b8049fdfe4ef74"}, - {file = "psygnal-0.9.5-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:d332a874263951e0a0158e3cef82601bce8e1cfe6f4c57a8d20d02f942e8bdb4"}, - {file = "psygnal-0.9.5-cp311-cp311-win_amd64.whl", hash = "sha256:21132b6ae5a7cfc8b2078b24cc3d7dc59fd07d867f649e10ef092976c4b8e33f"}, - {file = "psygnal-0.9.5-cp37-cp37m-macosx_10_16_x86_64.whl", hash = "sha256:a379aaf7b6e4caf107851c5d948694ddf3e1ce4ca83a16ccd0dceee870e2d5f0"}, - {file = "psygnal-0.9.5-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2338ea6e3a952a93a1b524e674ae1176666c0cd887311d8e93716e19f05a9da7"}, - {file = "psygnal-0.9.5-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:db88899c1da064e54fbac9fdb49fa34e73d762918fdb6c541c9f90fb6c232161"}, - {file = "psygnal-0.9.5-cp37-cp37m-win_amd64.whl", hash = "sha256:b5b575dd9b8a8b221104ca51ebc44bc7b06f2425991401690c46633befec2721"}, - {file = "psygnal-0.9.5-cp38-cp38-macosx_10_16_arm64.whl", hash = "sha256:06b79289705888199d63bfa0579be85b15df7df578c27d2bb54f0f51e7faec44"}, - {file = "psygnal-0.9.5-cp38-cp38-macosx_10_16_x86_64.whl", hash = "sha256:bf0d0c9bea1f8c3d379ae6ac52d87c52b4fe62a651db2f06e92b964fb7bc2539"}, - {file = "psygnal-0.9.5-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c8fa4929a66ba776d98f119db8d87f144d102c86c4ee9893a427507bde46d704"}, - {file = "psygnal-0.9.5-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:9b6e1249240177ff1bfc7dd19df983bdaba0a59361613ff9138b8f0349787d9c"}, - {file = "psygnal-0.9.5-cp38-cp38-win_amd64.whl", hash = "sha256:e2f0623a1cbb40791e6e6787f6859fad52e65241d34f516caaf9968663d5405b"}, - {file = "psygnal-0.9.5-cp39-cp39-macosx_10_16_arm64.whl", hash = "sha256:ef576ba8dde2b2b5f786a37bdfce3b4608047c69c55653be071855c32f012f99"}, - {file = "psygnal-0.9.5-cp39-cp39-macosx_10_16_x86_64.whl", hash = "sha256:407ee58cac4b683f58cf0ccae21ad8c0371aff4cb6b55f531ff61aca73fdcf0e"}, - {file = "psygnal-0.9.5-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:42d6168e891050005c7b7474897c35188b7be7f62bb88efe93985d6c6a50a9c3"}, - {file = "psygnal-0.9.5-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:bb85ed3e8d22d97d98f86af714eec0a3458a659bf72296423036acc7acaa29ec"}, - {file = "psygnal-0.9.5-cp39-cp39-win_amd64.whl", hash = "sha256:f511de71282598c30f4a0ce268300d384dc8d4dd536ceb0e4b47759558931282"}, - {file = "psygnal-0.9.5-py3-none-any.whl", hash = "sha256:13b3db8cbf0eea620abd6b611912087ffb23621c8bad20d90b037cc57c3985d5"}, - {file = "psygnal-0.9.5.tar.gz", hash = "sha256:4956ea6c36a75f7fc457558935b67dd8be2594661b4d08eeb3357d69c509c55f"}, -] - -[package.dependencies] -mypy-extensions = "*" -typing-extensions = ">=3.7.4.2" - -[package.extras] -dev = ["black", "cruft", "dask", "ipython", "mypy", "numpy", "pre-commit", "pydantic", "pyqt5", "pytest", "pytest-cov", "pytest-mypy-plugins", "pytest-qt", "qtpy", "rich", "ruff", "wrapt"] -docs = ["griffe (==0.25.5)", "mkdocs (==1.4.2)", "mkdocs-material (==8.5.10)", "mkdocs-minify-plugin", "mkdocs-spellcheck[all]", "mkdocstrings (==0.20.0)", "mkdocstrings-python (==0.8.3)"] -proxy = ["wrapt"] -pydantic = ["pydantic"] -test = ["attrs", "dask", "msgspec", "numpy", "pydantic", "pyinstaller (>=4.0)", "pytest (>=6.0)", "pytest-codspeed", "pytest-cov", "toolz", "wrapt"] -testqt = ["pytest-qt", "qtpy"] - [[package]] name = "ptyprocess" version = "0.7.0" @@ -4448,51 +4732,40 @@ files = [ [[package]] name = "pyarrow" -version = "15.0.0" +version = "11.0.0" description = "Python library for Apache Arrow" optional = false -python-versions = ">=3.8" +python-versions = ">=3.7" files = [ - {file = "pyarrow-15.0.0-cp310-cp310-macosx_10_15_x86_64.whl", hash = "sha256:0a524532fd6dd482edaa563b686d754c70417c2f72742a8c990b322d4c03a15d"}, - {file = "pyarrow-15.0.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:60a6bdb314affa9c2e0d5dddf3d9cbb9ef4a8dddaa68669975287d47ece67642"}, - {file = "pyarrow-15.0.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:66958fd1771a4d4b754cd385835e66a3ef6b12611e001d4e5edfcef5f30391e2"}, - {file = "pyarrow-15.0.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1f500956a49aadd907eaa21d4fff75f73954605eaa41f61cb94fb008cf2e00c6"}, - {file = "pyarrow-15.0.0-cp310-cp310-manylinux_2_28_aarch64.whl", hash = "sha256:6f87d9c4f09e049c2cade559643424da84c43a35068f2a1c4653dc5b1408a929"}, - {file = "pyarrow-15.0.0-cp310-cp310-manylinux_2_28_x86_64.whl", hash = "sha256:85239b9f93278e130d86c0e6bb455dcb66fc3fd891398b9d45ace8799a871a1e"}, - {file = "pyarrow-15.0.0-cp310-cp310-win_amd64.whl", hash = "sha256:5b8d43e31ca16aa6e12402fcb1e14352d0d809de70edd185c7650fe80e0769e3"}, - {file = "pyarrow-15.0.0-cp311-cp311-macosx_10_15_x86_64.whl", hash = "sha256:fa7cd198280dbd0c988df525e50e35b5d16873e2cdae2aaaa6363cdb64e3eec5"}, - {file = "pyarrow-15.0.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:8780b1a29d3c8b21ba6b191305a2a607de2e30dab399776ff0aa09131e266340"}, - {file = "pyarrow-15.0.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fe0ec198ccc680f6c92723fadcb97b74f07c45ff3fdec9dd765deb04955ccf19"}, - {file = "pyarrow-15.0.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:036a7209c235588c2f07477fe75c07e6caced9b7b61bb897c8d4e52c4b5f9555"}, - {file = "pyarrow-15.0.0-cp311-cp311-manylinux_2_28_aarch64.whl", hash = "sha256:2bd8a0e5296797faf9a3294e9fa2dc67aa7f10ae2207920dbebb785c77e9dbe5"}, - {file = "pyarrow-15.0.0-cp311-cp311-manylinux_2_28_x86_64.whl", hash = "sha256:e8ebed6053dbe76883a822d4e8da36860f479d55a762bd9e70d8494aed87113e"}, - {file = "pyarrow-15.0.0-cp311-cp311-win_amd64.whl", hash = "sha256:17d53a9d1b2b5bd7d5e4cd84d018e2a45bc9baaa68f7e6e3ebed45649900ba99"}, - {file = "pyarrow-15.0.0-cp312-cp312-macosx_10_15_x86_64.whl", hash = "sha256:9950a9c9df24090d3d558b43b97753b8f5867fb8e521f29876aa021c52fda351"}, - {file = "pyarrow-15.0.0-cp312-cp312-macosx_11_0_arm64.whl", hash = "sha256:003d680b5e422d0204e7287bb3fa775b332b3fce2996aa69e9adea23f5c8f970"}, - {file = "pyarrow-15.0.0-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f75fce89dad10c95f4bf590b765e3ae98bcc5ba9f6ce75adb828a334e26a3d40"}, - {file = "pyarrow-15.0.0-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:0ca9cb0039923bec49b4fe23803807e4ef39576a2bec59c32b11296464623dc2"}, - {file = "pyarrow-15.0.0-cp312-cp312-manylinux_2_28_aarch64.whl", hash = "sha256:9ed5a78ed29d171d0acc26a305a4b7f83c122d54ff5270810ac23c75813585e4"}, - {file = "pyarrow-15.0.0-cp312-cp312-manylinux_2_28_x86_64.whl", hash = "sha256:6eda9e117f0402dfcd3cd6ec9bfee89ac5071c48fc83a84f3075b60efa96747f"}, - {file = "pyarrow-15.0.0-cp312-cp312-win_amd64.whl", hash = "sha256:9a3a6180c0e8f2727e6f1b1c87c72d3254cac909e609f35f22532e4115461177"}, - {file = "pyarrow-15.0.0-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:19a8918045993349b207de72d4576af0191beef03ea655d8bdb13762f0cd6eac"}, - {file = "pyarrow-15.0.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:d0ec076b32bacb6666e8813a22e6e5a7ef1314c8069d4ff345efa6246bc38593"}, - {file = "pyarrow-15.0.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5db1769e5d0a77eb92344c7382d6543bea1164cca3704f84aa44e26c67e320fb"}, - {file = "pyarrow-15.0.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e2617e3bf9df2a00020dd1c1c6dce5cc343d979efe10bc401c0632b0eef6ef5b"}, - {file = "pyarrow-15.0.0-cp38-cp38-manylinux_2_28_aarch64.whl", hash = "sha256:d31c1d45060180131caf10f0f698e3a782db333a422038bf7fe01dace18b3a31"}, - {file = "pyarrow-15.0.0-cp38-cp38-manylinux_2_28_x86_64.whl", hash = "sha256:c8c287d1d479de8269398b34282e206844abb3208224dbdd7166d580804674b7"}, - {file = "pyarrow-15.0.0-cp38-cp38-win_amd64.whl", hash = "sha256:07eb7f07dc9ecbb8dace0f58f009d3a29ee58682fcdc91337dfeb51ea618a75b"}, - {file = "pyarrow-15.0.0-cp39-cp39-macosx_10_15_x86_64.whl", hash = "sha256:47af7036f64fce990bb8a5948c04722e4e3ea3e13b1007ef52dfe0aa8f23cf7f"}, - {file = "pyarrow-15.0.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:93768ccfff85cf044c418bfeeafce9a8bb0cee091bd8fd19011aff91e58de540"}, - {file = "pyarrow-15.0.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:f6ee87fd6892700960d90abb7b17a72a5abb3b64ee0fe8db6c782bcc2d0dc0b4"}, - {file = "pyarrow-15.0.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:001fca027738c5f6be0b7a3159cc7ba16a5c52486db18160909a0831b063c4e4"}, - {file = "pyarrow-15.0.0-cp39-cp39-manylinux_2_28_aarch64.whl", hash = "sha256:d1c48648f64aec09accf44140dccb92f4f94394b8d79976c426a5b79b11d4fa7"}, - {file = "pyarrow-15.0.0-cp39-cp39-manylinux_2_28_x86_64.whl", hash = "sha256:972a0141be402bb18e3201448c8ae62958c9c7923dfaa3b3d4530c835ac81aed"}, - {file = "pyarrow-15.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:f01fc5cf49081426429127aa2d427d9d98e1cb94a32cb961d583a70b7c4504e6"}, - {file = "pyarrow-15.0.0.tar.gz", hash = "sha256:876858f549d540898f927eba4ef77cd549ad8d24baa3207cf1b72e5788b50e83"}, -] - -[package.dependencies] -numpy = ">=1.16.6,<2" + {file = "pyarrow-11.0.0-cp310-cp310-macosx_10_14_x86_64.whl", hash = "sha256:40bb42afa1053c35c749befbe72f6429b7b5f45710e85059cdd534553ebcf4f2"}, + {file = "pyarrow-11.0.0-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:7c28b5f248e08dea3b3e0c828b91945f431f4202f1a9fe84d1012a761324e1ba"}, + {file = "pyarrow-11.0.0-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a37bc81f6c9435da3c9c1e767324ac3064ffbe110c4e460660c43e144be4ed85"}, + {file = "pyarrow-11.0.0-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ad7c53def8dbbc810282ad308cc46a523ec81e653e60a91c609c2233ae407689"}, + {file = "pyarrow-11.0.0-cp310-cp310-win_amd64.whl", hash = "sha256:25aa11c443b934078bfd60ed63e4e2d42461682b5ac10f67275ea21e60e6042c"}, + {file = "pyarrow-11.0.0-cp311-cp311-macosx_10_14_x86_64.whl", hash = "sha256:e217d001e6389b20a6759392a5ec49d670757af80101ee6b5f2c8ff0172e02ca"}, + {file = "pyarrow-11.0.0-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:ad42bb24fc44c48f74f0d8c72a9af16ba9a01a2ccda5739a517aa860fa7e3d56"}, + {file = "pyarrow-11.0.0-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2d942c690ff24a08b07cb3df818f542a90e4d359381fbff71b8f2aea5bf58841"}, + {file = "pyarrow-11.0.0-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f010ce497ca1b0f17a8243df3048055c0d18dcadbcc70895d5baf8921f753de5"}, + {file = "pyarrow-11.0.0-cp311-cp311-win_amd64.whl", hash = "sha256:2f51dc7ca940fdf17893227edb46b6784d37522ce08d21afc56466898cb213b2"}, + {file = "pyarrow-11.0.0-cp37-cp37m-macosx_10_14_x86_64.whl", hash = "sha256:1cbcfcbb0e74b4d94f0b7dde447b835a01bc1d16510edb8bb7d6224b9bf5bafc"}, + {file = "pyarrow-11.0.0-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:aaee8f79d2a120bf3e032d6d64ad20b3af6f56241b0ffc38d201aebfee879d00"}, + {file = "pyarrow-11.0.0-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:410624da0708c37e6a27eba321a72f29d277091c8f8d23f72c92bada4092eb5e"}, + {file = "pyarrow-11.0.0-cp37-cp37m-win_amd64.whl", hash = "sha256:2d53ba72917fdb71e3584ffc23ee4fcc487218f8ff29dd6df3a34c5c48fe8c06"}, + {file = "pyarrow-11.0.0-cp38-cp38-macosx_10_14_x86_64.whl", hash = "sha256:f12932e5a6feb5c58192209af1d2607d488cb1d404fbc038ac12ada60327fa34"}, + {file = "pyarrow-11.0.0-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:41a1451dd895c0b2964b83d91019e46f15b5564c7ecd5dcb812dadd3f05acc97"}, + {file = "pyarrow-11.0.0-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:becc2344be80e5dce4e1b80b7c650d2fc2061b9eb339045035a1baa34d5b8f1c"}, + {file = "pyarrow-11.0.0-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:8f40be0d7381112a398b93c45a7e69f60261e7b0269cc324e9f739ce272f4f70"}, + {file = "pyarrow-11.0.0-cp38-cp38-win_amd64.whl", hash = "sha256:362a7c881b32dc6b0eccf83411a97acba2774c10edcec715ccaab5ebf3bb0835"}, + {file = "pyarrow-11.0.0-cp39-cp39-macosx_10_14_x86_64.whl", hash = "sha256:ccbf29a0dadfcdd97632b4f7cca20a966bb552853ba254e874c66934931b9841"}, + {file = "pyarrow-11.0.0-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:3e99be85973592051e46412accea31828da324531a060bd4585046a74ba45854"}, + {file = "pyarrow-11.0.0-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:69309be84dcc36422574d19c7d3a30a7ea43804f12552356d1ab2a82a713c418"}, + {file = "pyarrow-11.0.0-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:da93340fbf6f4e2a62815064383605b7ffa3e9eeb320ec839995b1660d69f89b"}, + {file = "pyarrow-11.0.0-cp39-cp39-win_amd64.whl", hash = "sha256:caad867121f182d0d3e1a0d36f197df604655d0b466f1bc9bafa903aa95083e4"}, + {file = "pyarrow-11.0.0.tar.gz", hash = "sha256:5461c57dbdb211a632a48facb9b39bbeb8a7905ec95d768078525283caef5f6d"}, +] + +[package.dependencies] +numpy = ">=1.16.6" [[package]] name = "pyarrow-hotfix" @@ -4707,6 +4980,20 @@ files = [ [package.dependencies] stdlib-list = "*" +[[package]] +name = "pydot" +version = "1.4.2" +description = "Python interface to Graphviz's Dot" +optional = true +python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*" +files = [ + {file = "pydot-1.4.2-py2.py3-none-any.whl", hash = "sha256:66c98190c65b8d2e2382a441b4c0edfdb4f4c025ef9cb9874de478fb0793a451"}, + {file = "pydot-1.4.2.tar.gz", hash = "sha256:248081a39bcb56784deb018977e428605c1c758f10897a339fce1dd728ff007d"}, +] + +[package.dependencies] +pyparsing = ">=2.1.4" + [[package]] name = "pydruid" version = "0.6.6" @@ -4860,6 +5147,109 @@ dev = ["coverage[toml] (==5.0.4)", "cryptography (>=3.4.0)", "pre-commit", "pyte docs = ["sphinx (>=4.5.0,<5.0.0)", "sphinx-rtd-theme", "zope.interface"] tests = ["coverage[toml] (==5.0.4)", "pytest (>=6.0.0,<7.0.0)"] +[[package]] +name = "pymongo" +version = "4.6.1" +description = "Python driver for MongoDB " +optional = true +python-versions = ">=3.7" +files = [ + {file = "pymongo-4.6.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:4344c30025210b9fa80ec257b0e0aab5aa1d5cca91daa70d82ab97b482cc038e"}, + {file = "pymongo-4.6.1-cp310-cp310-manylinux1_i686.whl", hash = "sha256:1c5654bb8bb2bdb10e7a0bc3c193dd8b49a960b9eebc4381ff5a2043f4c3c441"}, + {file = "pymongo-4.6.1-cp310-cp310-manylinux2014_aarch64.whl", hash = "sha256:eaf2f65190c506def2581219572b9c70b8250615dc918b3b7c218361a51ec42e"}, + {file = "pymongo-4.6.1-cp310-cp310-manylinux2014_i686.whl", hash = "sha256:262356ea5fcb13d35fb2ab6009d3927bafb9504ef02339338634fffd8a9f1ae4"}, + {file = "pymongo-4.6.1-cp310-cp310-manylinux2014_ppc64le.whl", hash = "sha256:2dd2f6960ee3c9360bed7fb3c678be0ca2d00f877068556785ec2eb6b73d2414"}, + {file = "pymongo-4.6.1-cp310-cp310-manylinux2014_s390x.whl", hash = "sha256:ff925f1cca42e933376d09ddc254598f8c5fcd36efc5cac0118bb36c36217c41"}, + {file = "pymongo-4.6.1-cp310-cp310-manylinux2014_x86_64.whl", hash = "sha256:3cadf7f4c8e94d8a77874b54a63c80af01f4d48c4b669c8b6867f86a07ba994f"}, + {file = "pymongo-4.6.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:55dac73316e7e8c2616ba2e6f62b750918e9e0ae0b2053699d66ca27a7790105"}, + {file = "pymongo-4.6.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:154b361dcb358ad377d5d40df41ee35f1cc14c8691b50511547c12404f89b5cb"}, + {file = "pymongo-4.6.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:2940aa20e9cc328e8ddeacea8b9a6f5ddafe0b087fedad928912e787c65b4909"}, + {file = "pymongo-4.6.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:010bc9aa90fd06e5cc52c8fac2c2fd4ef1b5f990d9638548dde178005770a5e8"}, + {file = "pymongo-4.6.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:e470fa4bace5f50076c32f4b3cc182b31303b4fefb9b87f990144515d572820b"}, + {file = "pymongo-4.6.1-cp310-cp310-win32.whl", hash = "sha256:da08ea09eefa6b960c2dd9a68ec47949235485c623621eb1d6c02b46765322ac"}, + {file = "pymongo-4.6.1-cp310-cp310-win_amd64.whl", hash = "sha256:13d613c866f9f07d51180f9a7da54ef491d130f169e999c27e7633abe8619ec9"}, + {file = "pymongo-4.6.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:6a0ae7a48a6ef82ceb98a366948874834b86c84e288dbd55600c1abfc3ac1d88"}, + {file = "pymongo-4.6.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:5bd94c503271e79917b27c6e77f7c5474da6930b3fb9e70a12e68c2dff386b9a"}, + {file = "pymongo-4.6.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2d4ccac3053b84a09251da8f5350bb684cbbf8c8c01eda6b5418417d0a8ab198"}, + {file = "pymongo-4.6.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:349093675a2d3759e4fb42b596afffa2b2518c890492563d7905fac503b20daa"}, + {file = "pymongo-4.6.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:88beb444fb438385e53dc9110852910ec2a22f0eab7dd489e827038fdc19ed8d"}, + {file = "pymongo-4.6.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d8e62d06e90f60ea2a3d463ae51401475568b995bafaffd81767d208d84d7bb1"}, + {file = "pymongo-4.6.1-cp311-cp311-win32.whl", hash = "sha256:5556e306713e2522e460287615d26c0af0fe5ed9d4f431dad35c6624c5d277e9"}, + {file = "pymongo-4.6.1-cp311-cp311-win_amd64.whl", hash = "sha256:b10d8cda9fc2fcdcfa4a000aa10413a2bf8b575852cd07cb8a595ed09689ca98"}, + {file = "pymongo-4.6.1-cp312-cp312-macosx_10_9_universal2.whl", hash = "sha256:b435b13bb8e36be11b75f7384a34eefe487fe87a6267172964628e2b14ecf0a7"}, + {file = "pymongo-4.6.1-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:e438417ce1dc5b758742e12661d800482200b042d03512a8f31f6aaa9137ad40"}, + {file = "pymongo-4.6.1-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8b47ebd89e69fbf33d1c2df79759d7162fc80c7652dacfec136dae1c9b3afac7"}, + {file = "pymongo-4.6.1-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:bbed8cccebe1169d45cedf00461b2842652d476d2897fd1c42cf41b635d88746"}, + {file = "pymongo-4.6.1-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c30a9e06041fbd7a7590693ec5e407aa8737ad91912a1e70176aff92e5c99d20"}, + {file = "pymongo-4.6.1-cp312-cp312-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b8729dbf25eb32ad0dc0b9bd5e6a0d0b7e5c2dc8ec06ad171088e1896b522a74"}, + {file = "pymongo-4.6.1-cp312-cp312-win32.whl", hash = "sha256:3177f783ae7e08aaf7b2802e0df4e4b13903520e8380915e6337cdc7a6ff01d8"}, + {file = "pymongo-4.6.1-cp312-cp312-win_amd64.whl", hash = "sha256:00c199e1c593e2c8b033136d7a08f0c376452bac8a896c923fcd6f419e07bdd2"}, + {file = "pymongo-4.6.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:6dcc95f4bb9ed793714b43f4f23a7b0c57e4ef47414162297d6f650213512c19"}, + {file = "pymongo-4.6.1-cp37-cp37m-manylinux1_i686.whl", hash = "sha256:13552ca505366df74e3e2f0a4f27c363928f3dff0eef9f281eb81af7f29bc3c5"}, + {file = "pymongo-4.6.1-cp37-cp37m-manylinux1_x86_64.whl", hash = "sha256:77e0df59b1a4994ad30c6d746992ae887f9756a43fc25dec2db515d94cf0222d"}, + {file = "pymongo-4.6.1-cp37-cp37m-manylinux2014_aarch64.whl", hash = "sha256:3a7f02a58a0c2912734105e05dedbee4f7507e6f1bd132ebad520be0b11d46fd"}, + {file = "pymongo-4.6.1-cp37-cp37m-manylinux2014_i686.whl", hash = "sha256:026a24a36394dc8930cbcb1d19d5eb35205ef3c838a7e619e04bd170713972e7"}, + {file = "pymongo-4.6.1-cp37-cp37m-manylinux2014_ppc64le.whl", hash = "sha256:3b287e814a01deddb59b88549c1e0c87cefacd798d4afc0c8bd6042d1c3d48aa"}, + {file = "pymongo-4.6.1-cp37-cp37m-manylinux2014_s390x.whl", hash = "sha256:9a710c184ba845afb05a6f876edac8f27783ba70e52d5eaf939f121fc13b2f59"}, + {file = "pymongo-4.6.1-cp37-cp37m-manylinux2014_x86_64.whl", hash = "sha256:30b2c9caf3e55c2e323565d1f3b7e7881ab87db16997dc0cbca7c52885ed2347"}, + {file = "pymongo-4.6.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ff62ba8ff70f01ab4fe0ae36b2cb0b5d1f42e73dfc81ddf0758cd9f77331ad25"}, + {file = "pymongo-4.6.1-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:547dc5d7f834b1deefda51aedb11a7af9c51c45e689e44e14aa85d44147c7657"}, + {file = "pymongo-4.6.1-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1de3c6faf948f3edd4e738abdb4b76572b4f4fdfc1fed4dad02427e70c5a6219"}, + {file = "pymongo-4.6.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a2831e05ce0a4df10c4ac5399ef50b9a621f90894c2a4d2945dc5658765514ed"}, + {file = "pymongo-4.6.1-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:144a31391a39a390efce0c5ebcaf4bf112114af4384c90163f402cec5ede476b"}, + {file = "pymongo-4.6.1-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:33bb16a07d3cc4e0aea37b242097cd5f7a156312012455c2fa8ca396953b11c4"}, + {file = "pymongo-4.6.1-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:b7b1a83ce514700276a46af3d9e481ec381f05b64939effc9065afe18456a6b9"}, + {file = "pymongo-4.6.1-cp37-cp37m-win32.whl", hash = "sha256:3071ec998cc3d7b4944377e5f1217c2c44b811fae16f9a495c7a1ce9b42fb038"}, + {file = "pymongo-4.6.1-cp37-cp37m-win_amd64.whl", hash = "sha256:2346450a075625c4d6166b40a013b605a38b6b6168ce2232b192a37fb200d588"}, + {file = "pymongo-4.6.1-cp38-cp38-macosx_11_0_universal2.whl", hash = "sha256:061598cbc6abe2f382ab64c9caa83faa2f4c51256f732cdd890bcc6e63bfb67e"}, + {file = "pymongo-4.6.1-cp38-cp38-manylinux1_i686.whl", hash = "sha256:d483793a384c550c2d12cb794ede294d303b42beff75f3b3081f57196660edaf"}, + {file = "pymongo-4.6.1-cp38-cp38-manylinux1_x86_64.whl", hash = "sha256:f9756f1d25454ba6a3c2f1ef8b7ddec23e5cdeae3dc3c3377243ae37a383db00"}, + {file = "pymongo-4.6.1-cp38-cp38-manylinux2014_aarch64.whl", hash = "sha256:1ed23b0e2dac6f84f44c8494fbceefe6eb5c35db5c1099f56ab78fc0d94ab3af"}, + {file = "pymongo-4.6.1-cp38-cp38-manylinux2014_i686.whl", hash = "sha256:3d18a9b9b858ee140c15c5bfcb3e66e47e2a70a03272c2e72adda2482f76a6ad"}, + {file = "pymongo-4.6.1-cp38-cp38-manylinux2014_ppc64le.whl", hash = "sha256:c258dbacfff1224f13576147df16ce3c02024a0d792fd0323ac01bed5d3c545d"}, + {file = "pymongo-4.6.1-cp38-cp38-manylinux2014_s390x.whl", hash = "sha256:f7acc03a4f1154ba2643edeb13658d08598fe6e490c3dd96a241b94f09801626"}, + {file = "pymongo-4.6.1-cp38-cp38-manylinux2014_x86_64.whl", hash = "sha256:76013fef1c9cd1cd00d55efde516c154aa169f2bf059b197c263a255ba8a9ddf"}, + {file = "pymongo-4.6.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3f0e6a6c807fa887a0c51cc24fe7ea51bb9e496fe88f00d7930063372c3664c3"}, + {file = "pymongo-4.6.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:dd1fa413f8b9ba30140de198e4f408ffbba6396864c7554e0867aa7363eb58b2"}, + {file = "pymongo-4.6.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8d219b4508f71d762368caec1fc180960569766049bbc4d38174f05e8ef2fe5b"}, + {file = "pymongo-4.6.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:27b81ecf18031998ad7db53b960d1347f8f29e8b7cb5ea7b4394726468e4295e"}, + {file = "pymongo-4.6.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:56816e43c92c2fa8c11dc2a686f0ca248bea7902f4a067fa6cbc77853b0f041e"}, + {file = "pymongo-4.6.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:ef801027629c5b511cf2ba13b9be29bfee36ae834b2d95d9877818479cdc99ea"}, + {file = "pymongo-4.6.1-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:d4c2be9760b112b1caf649b4977b81b69893d75aa86caf4f0f398447be871f3c"}, + {file = "pymongo-4.6.1-cp38-cp38-win32.whl", hash = "sha256:39d77d8bbb392fa443831e6d4ae534237b1f4eee6aa186f0cdb4e334ba89536e"}, + {file = "pymongo-4.6.1-cp38-cp38-win_amd64.whl", hash = "sha256:4497d49d785482cc1a44a0ddf8830b036a468c088e72a05217f5b60a9e025012"}, + {file = "pymongo-4.6.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:69247f7a2835fc0984bbf0892e6022e9a36aec70e187fcfe6cae6a373eb8c4de"}, + {file = "pymongo-4.6.1-cp39-cp39-manylinux1_i686.whl", hash = "sha256:7bb0e9049e81def6829d09558ad12d16d0454c26cabe6efc3658e544460688d9"}, + {file = "pymongo-4.6.1-cp39-cp39-manylinux1_x86_64.whl", hash = "sha256:6a1810c2cbde714decf40f811d1edc0dae45506eb37298fd9d4247b8801509fe"}, + {file = "pymongo-4.6.1-cp39-cp39-manylinux2014_aarch64.whl", hash = "sha256:e2aced6fb2f5261b47d267cb40060b73b6527e64afe54f6497844c9affed5fd0"}, + {file = "pymongo-4.6.1-cp39-cp39-manylinux2014_i686.whl", hash = "sha256:d0355cff58a4ed6d5e5f6b9c3693f52de0784aa0c17119394e2a8e376ce489d4"}, + {file = "pymongo-4.6.1-cp39-cp39-manylinux2014_ppc64le.whl", hash = "sha256:3c74f4725485f0a7a3862cfd374cc1b740cebe4c133e0c1425984bcdcce0f4bb"}, + {file = "pymongo-4.6.1-cp39-cp39-manylinux2014_s390x.whl", hash = "sha256:9c79d597fb3a7c93d7c26924db7497eba06d58f88f58e586aa69b2ad89fee0f8"}, + {file = "pymongo-4.6.1-cp39-cp39-manylinux2014_x86_64.whl", hash = "sha256:8ec75f35f62571a43e31e7bd11749d974c1b5cd5ea4a8388725d579263c0fdf6"}, + {file = "pymongo-4.6.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a5e641f931c5cd95b376fd3c59db52770e17bec2bf86ef16cc83b3906c054845"}, + {file = "pymongo-4.6.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9aafd036f6f2e5ad109aec92f8dbfcbe76cff16bad683eb6dd18013739c0b3ae"}, + {file = "pymongo-4.6.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1f2b856518bfcfa316c8dae3d7b412aecacf2e8ba30b149f5eb3b63128d703b9"}, + {file = "pymongo-4.6.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5ec31adc2e988fd7db3ab509954791bbc5a452a03c85e45b804b4bfc31fa221d"}, + {file = "pymongo-4.6.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9167e735379ec43d8eafa3fd675bfbb12e2c0464f98960586e9447d2cf2c7a83"}, + {file = "pymongo-4.6.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.whl", hash = "sha256:1461199b07903fc1424709efafe379205bf5f738144b1a50a08b0396357b5abf"}, + {file = "pymongo-4.6.1-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:3094c7d2f820eecabadae76bfec02669567bbdd1730eabce10a5764778564f7b"}, + {file = "pymongo-4.6.1-cp39-cp39-win32.whl", hash = "sha256:c91ea3915425bd4111cb1b74511cdc56d1d16a683a48bf2a5a96b6a6c0f297f7"}, + {file = "pymongo-4.6.1-cp39-cp39-win_amd64.whl", hash = "sha256:ef102a67ede70e1721fe27f75073b5314911dbb9bc27cde0a1c402a11531e7bd"}, + {file = "pymongo-4.6.1.tar.gz", hash = "sha256:31dab1f3e1d0cdd57e8df01b645f52d43cc1b653ed3afd535d2891f4fc4f9712"}, +] + +[package.dependencies] +dnspython = ">=1.16.0,<3.0.0" + +[package.extras] +aws = ["pymongo-auth-aws (<2.0.0)"] +encryption = ["certifi", "pymongo[aws]", "pymongocrypt (>=1.6.0,<2.0.0)"] +gssapi = ["pykerberos", "winkerberos (>=0.5.0)"] +ocsp = ["certifi", "cryptography (>=2.5)", "pyopenssl (>=17.2.0)", "requests (<3.0.0)", "service-identity (>=18.1.0)"] +snappy = ["python-snappy"] +test = ["pytest (>=7)"] +zstd = ["zstandard"] + [[package]] name = "pymysql" version = "1.1.0" @@ -4951,7 +5341,7 @@ diagrams = ["jinja2", "railroad-diagrams"] name = "pyproj" version = "3.6.1" description = "Python interface to PROJ (cartographic projections and coordinate transformations library)" -optional = false +optional = true python-versions = ">=3.9" files = [ {file = "pyproj-3.6.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:ab7aa4d9ff3c3acf60d4b285ccec134167a948df02347585fdd934ebad8811b4"}, @@ -5236,6 +5626,20 @@ files = [ [package.dependencies] pytest = ">=3.0.0" +[[package]] +name = "pytest-split" +version = "0.8.2" +description = "Pytest plugin which splits the test suite to equally sized sub suites based on test execution time." +optional = false +python-versions = ">=3.7.1,<4.0" +files = [ + {file = "pytest_split-0.8.2-py3-none-any.whl", hash = "sha256:b7fa704659cb224b9f7f5c24536bc04eff351f42d852bf0312e03774fd9c0972"}, + {file = "pytest_split-0.8.2.tar.gz", hash = "sha256:446f330e3607572027f3861058c27d9b3eaa80d83dc86675abe2978bbf50c02f"}, +] + +[package.dependencies] +pytest = ">=5,<9" + [[package]] name = "pytest-xdist" version = "3.5.0" @@ -6137,7 +6541,7 @@ testing-integration = ["build[virtualenv] (>=1.0.3)", "filelock (>=3.4.0)", "jar name = "shapely" version = "2.0.2" description = "Manipulation and analysis of geometric objects" -optional = false +optional = true python-versions = ">=3.7" files = [ {file = "shapely-2.0.2-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:6ca8cffbe84ddde8f52b297b53f8e0687bd31141abb2c373fd8a9f032df415d6"}, @@ -7315,7 +7719,7 @@ cffi = {version = ">=1.11", markers = "platform_python_implementation == \"PyPy\ cffi = ["cffi (>=1.11)"] [extras] -all = ["black", "clickhouse-connect", "dask", "datafusion", "db-dtypes", "deltalake", "duckdb", "geopandas", "google-cloud-bigquery", "google-cloud-bigquery-storage", "graphviz", "impyla", "oracledb", "packaging", "pins", "polars", "psycopg2", "pydata-google-auth", "pydruid", "pyexasol", "pymysql", "pyodbc", "pyspark", "regex", "shapely", "snowflake-connector-python", "trino"] +all = ["apache-flink", "black", "clickhouse-connect", "dask", "datafusion", "db-dtypes", "deltalake", "duckdb", "geopandas", "google-cloud-bigquery", "google-cloud-bigquery-storage", "graphviz", "impyla", "oracledb", "packaging", "pins", "polars", "psycopg2", "pydata-google-auth", "pydruid", "pyexasol", "pymysql", "pyodbc", "pyspark", "regex", "shapely", "snowflake-connector-python", "trino"] bigquery = ["db-dtypes", "google-cloud-bigquery", "google-cloud-bigquery-storage", "pydata-google-auth"] clickhouse = ["clickhouse-connect"] dask = ["dask", "regex"] @@ -7326,7 +7730,7 @@ druid = ["pydruid"] duckdb = ["duckdb"] examples = ["pins"] exasol = ["pyexasol"] -flink = [] +flink = ["apache-flink"] geospatial = ["geopandas", "shapely"] impala = ["impyla"] mssql = ["pyodbc"] @@ -7345,4 +7749,4 @@ visualization = ["graphviz"] [metadata] lock-version = "2.0" python-versions = "^3.9" -content-hash = "f82d6de75eac875bd188dd40749b10ffd6aaf6cb7d63642223a7a16437563080" +content-hash = "65e2325649ab7c8cb5e4aedca14702d0bc6da53104f2bc55014205f361d01bc0" diff --git a/pyproject.toml b/pyproject.toml index 6fdf826666cae..893d7b825826e 100644 --- a/pyproject.toml +++ b/pyproject.toml @@ -52,6 +52,7 @@ sqlglot = ">=18.12.0,<=20.11" toolz = ">=0.11,<1" typing-extensions = ">=4.3.0,<5" black = { version = ">=22.1.0,<25", optional = true } +apache-flink = { version = ">=1.18.0,<2", optional = true } clickhouse-connect = { version = ">=0.5.23,<1", optional = true, extras = [ "arrow", "numpy", @@ -119,6 +120,7 @@ pytest-mock = ">=3.6.1,<4" pytest-randomly = ">=3.10.1,<4" pytest-repeat = ">=0.9.1,<0.10" pytest-snapshot = ">=0.9.0,<1" +pytest-split = ">=0.8.1,<1" pytest-xdist = ">=2.3.0,<4" requests = ">=2,<3" @@ -136,11 +138,11 @@ requests = { version = ">=2,<3", python = ">=3.10,<3.13" } scikit-learn = { version = ">=1.3,<2", python = ">=3.10,<3.13" } seaborn = { version = ">=0.12.2,<1", python = ">=3.10,<3.13" } leafmap = { version = ">=0.29.6,<1", python = ">=3.10,<3.13" } -lonboard = { version = ">=0.5.0,<1", python = ">=3.10,<3.13" } [tool.poetry.extras] # generate the `all` extra using nix run '.#gen-all-extras' all = [ + "apache-flink", "black", "clickhouse-connect", "dask", @@ -181,7 +183,7 @@ datafusion = ["datafusion"] druid = ["pydruid"] duckdb = ["duckdb"] exasol = ["pyexasol"] -flink = [] +flink = ["apache-flink"] geospatial = ["geopandas", "shapely"] impala = ["impyla"] mssql = ["pyodbc"] @@ -270,7 +272,7 @@ filterwarnings = [ "ignore:Concatenating dataframes with unknown divisions:UserWarning", "ignore:Possible nested set at position:FutureWarning", 'ignore:\s+You did not provide metadata:UserWarning', - "ignore:Minimal version of pyarrow will soon be increased:FutureWarning", + 'ignore:Minimal version of pyarrow will soon be increased to 14:FutureWarning', # numpy by way of dask 'ignore:np\.find_common_type is deprecated:DeprecationWarning', # pandas @@ -313,7 +315,7 @@ filterwarnings = [ "ignore:the imp module is deprecated in favour of importlib:DeprecationWarning", # pytest raises a syntax error when encountering this from *any* module, including third party modules "ignore:invalid escape sequence:DeprecationWarning", - # geopandas raises usr warning on geometry column + # geopandas raises user warning on geometry column "ignore:Geometry is in a geographic CRS", # `is_sparse` deprecation was addressed in pyarrow 13.0.0 (see https://github.com/apache/arrow/pull/35366), # but flink requires apache-beam<2.49, which caps its pyarrow dependency (see https://github.com/apache/beam/blob/v2.48.0/sdks/python/setup.py#L144) diff --git a/requirements-dev.txt b/requirements-dev.txt index 05b6eb0c6bc4c..a5ed7c33f6560 100644 --- a/requirements-dev.txt +++ b/requirements-dev.txt @@ -2,7 +2,9 @@ aiohttp==3.9.2 ; python_version >= "3.9" and python_version < "4.0" aiosignal==1.3.1 ; python_version >= "3.9" and python_version < "4.0" altair==5.2.0 ; python_version >= "3.10" and python_version < "3.13" annotated-types==0.6.0 ; python_version >= "3.10" and python_version < "3.13" -anywidget==0.7.1 ; python_version >= "3.10" and python_version < "3.13" +apache-beam==2.48.0 ; python_version >= "3.9" and python_version < "4.0" +apache-flink-libraries==1.18.1 ; python_version >= "3.9" and python_version < "4.0" +apache-flink==1.18.1 ; python_version >= "3.9" and python_version < "4.0" appdirs==1.4.4 ; python_version >= "3.9" and python_version < "4.0" appnope==0.1.3 ; python_version >= "3.10" and python_version < "3.13" and platform_system == "Darwin" asn1crypto==1.5.1 ; python_version >= "3.9" and python_version < "4.0" @@ -10,6 +12,7 @@ asttokens==2.4.1 ; python_version >= "3.9" and python_version < "4.0" async-timeout==4.0.3 ; python_version >= "3.9" and python_version < "3.11" atpublic==4.0 ; python_version >= "3.9" and python_version < "4.0" attrs==23.2.0 ; python_version >= "3.9" and python_version < "4.0" +avro-python3==1.10.2 ; python_version >= "3.9" and python_version < "4.0" beartype==0.16.4 ; python_version >= "3.10" and python_version < "3.13" beautifulsoup4==4.12.2 ; python_version >= "3.10" and python_version < "3.13" bidict==0.22.1 ; python_version >= "3.9" and python_version < "4.0" @@ -30,7 +33,7 @@ click-plugins==1.1.1 ; python_version >= "3.9" and python_version < "4.0" click==8.1.7 ; python_version >= "3.9" and python_version < "4.0" clickhouse-connect[arrow,numpy,pandas]==0.6.23 ; python_version >= "3.9" and python_version < "4.0" cligj==0.7.2 ; python_version >= "3.9" and python_version < "4" -cloudpickle==3.0.0 ; python_version >= "3.9" and python_version < "4.0" +cloudpickle==2.2.1 ; python_version >= "3.9" and python_version < "4.0" codespell[hard-encoding-detection,toml]==2.2.6 ; python_version >= "3.9" and python_version < "4.0" colorama==0.4.6 ; python_version >= "3.9" and python_version < "4.0" and (sys_platform == "win32" or platform_system == "Windows" or python_version >= "3.10" or os_name == "nt") and (sys_platform == "win32" or platform_system == "Windows" or os_name == "nt" or python_version < "3.13") colour==0.1.5 ; python_version >= "3.10" and python_version < "3.13" @@ -38,6 +41,7 @@ comm==0.2.1 ; python_version >= "3.10" and python_version < "3.13" contourpy==1.2.0 ; python_version >= "3.10" and python_version < "3.13" coverage[toml]==7.4.0 ; python_version >= "3.9" and python_version < "4.0" crashtest==0.4.1 ; python_version >= "3.9" and python_version < "4.0" +crcmod==1.7 ; python_version >= "3.9" and python_version < "4.0" cryptography==41.0.7 ; python_version >= "3.9" and python_version < "4.0" cycler==0.12.1 ; python_version >= "3.10" and python_version < "3.13" dask==2024.1.0 ; python_version >= "3.10" and python_version < "3.13" @@ -47,16 +51,22 @@ db-dtypes==1.2.0 ; python_version >= "3.9" and python_version < "4.0" debugpy==1.8.0 ; python_version >= "3.10" and python_version < "3.13" decorator==5.1.1 ; python_version >= "3.9" and python_version < "4.0" deltalake==0.15.1 ; python_version >= "3.9" and python_version < "4.0" +dill==0.3.1.1 ; python_version >= "3.9" and python_version < "4.0" distlib==0.3.8 ; python_version >= "3.9" and python_version < "4.0" distributed==2024.1.0 ; python_version >= "3.10" and python_version < "3.13" +dnspython==2.5.0 ; python_version >= "3.9" and python_version < "4.0" +docopt==0.6.2 ; python_version >= "3.9" and python_version < "4.0" duckdb==0.9.2 ; python_version >= "3.9" and python_version < "4.0" dulwich==0.21.7 ; python_version >= "3.9" and python_version < "4.0" dunamai==1.19.0 ; python_version >= "3.9" and python_version < "4.0" exceptiongroup==1.2.0 ; python_version >= "3.9" and python_version < "3.11" execnet==2.0.2 ; python_version >= "3.9" and python_version < "4.0" executing==2.0.1 ; python_version >= "3.9" and python_version < "4.0" +fastavro==1.9.3 ; python_version >= "3.9" and python_version < "4.0" +fasteners==0.19 ; python_version >= "3.9" and python_version < "4.0" fastjsonschema==2.19.1 ; python_version >= "3.9" and python_version < "4.0" filelock==3.13.1 ; python_version >= "3.9" and python_version < "4.0" +find-libpython==0.3.1 ; python_version >= "3.9" and python_version < "4.0" and platform_system != "Windows" fiona==1.9.5 ; python_version >= "3.9" and python_version < "4.0" folium==0.15.1 ; python_version >= "3.10" and python_version < "3.13" fonttools==4.47.2 ; python_version >= "3.10" and python_version < "3.13" @@ -82,6 +92,8 @@ greenlet==3.0.3 ; python_version >= "3.9" and (platform_machine == "aarch64" or griffe==0.38.1 ; python_version >= "3.10" and python_version < "3.13" grpcio-status==1.60.0 ; python_version >= "3.9" and python_version < "4.0" grpcio==1.60.0 ; python_version >= "3.9" and python_version < "4.0" +hdfs==2.7.3 ; python_version >= "3.9" and python_version < "4.0" +httplib2==0.22.0 ; python_version >= "3.9" and python_version < "4.0" humanize==4.9.0 ; python_version >= "3.9" and python_version < "4.0" hypothesis==6.93.0 ; python_version >= "3.9" and python_version < "4.0" identify==2.5.33 ; python_version >= "3.9" and python_version < "4.0" @@ -113,7 +125,6 @@ keyring==24.3.0 ; python_version >= "3.9" and python_version < "4.0" kiwisolver==1.4.5 ; python_version >= "3.10" and python_version < "3.13" leafmap==0.30.1 ; python_version >= "3.10" and python_version < "3.13" locket==1.0.0 ; python_version >= "3.9" and python_version < "4.0" -lonboard==0.5.0 ; python_version >= "3.10" and python_version < "3.13" lz4==4.3.3 ; python_version >= "3.9" and python_version < "4.0" markdown-it-py==3.0.0 ; python_version >= "3.9" and python_version < "4.0" markupsafe==2.1.3 ; python_version >= "3.9" and python_version < "4.0" @@ -130,17 +141,19 @@ nbclient==0.9.0 ; python_version >= "3.10" and python_version < "3.13" nbformat==5.9.2 ; python_version >= "3.10" and python_version < "3.13" nest-asyncio==1.5.8 ; python_version >= "3.10" and python_version < "3.13" nodeenv==1.8.0 ; python_version >= "3.9" and python_version < "4.0" -numpy==1.26.3 ; python_version >= "3.9" and python_version < "4.0" +numpy==1.24.4 ; python_version >= "3.9" and python_version < "4.0" oauthlib==3.2.2 ; python_version >= "3.9" and python_version < "4.0" +objsize==0.6.1 ; python_version >= "3.9" and python_version < "4.0" oracledb==2.0.1 ; python_version >= "3.9" and python_version < "4.0" +orjson==3.9.13 ; python_version >= "3.9" and python_version < "4.0" packaging==23.2 ; python_version >= "3.9" and python_version < "4.0" -palettable==3.3.3 ; python_version >= "3.10" and python_version < "3.13" pandas==2.0.3 ; python_version >= "3.9" and python_version < "4.0" parso==0.8.3 ; python_version >= "3.9" and python_version < "4.0" parsy==2.1 ; python_version >= "3.9" and python_version < "4.0" partd==1.4.1 ; python_version >= "3.9" and python_version < "4.0" pathspec==0.12.1 ; python_version >= "3.9" and python_version < "4.0" patsy==0.5.6 ; python_version >= "3.10" and python_version < "3.13" +pemja==0.3.0 ; python_version >= "3.9" and python_version < "4.0" and platform_system != "Windows" pexpect==4.9.0 ; python_version >= "3.9" and python_version < "4.0" pillow==10.2.0 ; python_version >= "3.10" and python_version < "3.13" pins[gcs]==0.8.4 ; python_version >= "3.9" and python_version < "4.0" @@ -159,17 +172,16 @@ pprintpp==0.4.0 ; python_version >= "3.9" and python_version < "4.0" pre-commit==3.6.0 ; python_version >= "3.9" and python_version < "4.0" prompt-toolkit==3.0.43 ; python_version >= "3.9" and python_version < "4.0" proto-plus==1.23.0 ; python_version >= "3.9" and python_version < "4.0" -protobuf==4.25.2 ; python_version >= "3.9" and python_version < "4.0" +protobuf==4.23.4 ; python_version >= "3.9" and python_version < "4.0" psutil==5.9.7 ; python_version >= "3.10" and python_version < "3.13" psycopg2==2.9.9 ; python_version >= "3.9" and python_version < "4.0" -psygnal==0.9.5 ; python_version >= "3.10" and python_version < "3.13" ptyprocess==0.7.0 ; python_version >= "3.9" and python_version < "4.0" pure-eval==0.2.2 ; python_version >= "3.9" and python_version < "4.0" pure-sasl==0.6.2 ; python_version >= "3.9" and python_version < "4.0" py-cpuinfo==9.0.0 ; python_version >= "3.9" and python_version < "4.0" py4j==0.10.9.7 ; python_version >= "3.9" and python_version < "4.0" pyarrow-hotfix==0.6 ; python_version >= "3.9" and python_version < "4.0" -pyarrow==15.0.0 ; python_version >= "3.9" and python_version < "4.0" +pyarrow==11.0.0 ; python_version >= "3.9" and python_version < "4.0" pyasn1-modules==0.3.0 ; python_version >= "3.9" and python_version < "4.0" pyasn1==0.5.1 ; python_version >= "3.9" and python_version < "4" pycparser==2.21 ; python_version >= "3.9" and python_version < "4.0" @@ -177,15 +189,17 @@ pydantic-core==2.14.6 ; python_version >= "3.10" and python_version < "3.13" pydantic==2.5.3 ; python_version >= "3.10" and python_version < "3.13" pydata-google-auth==1.8.2 ; python_version >= "3.9" and python_version < "4.0" pydeps==1.12.17 ; python_version >= "3.9" and python_version < "4.0" +pydot==1.4.2 ; python_version >= "3.9" and python_version < "4.0" pydruid[sqlalchemy]==0.6.6 ; python_version >= "3.9" and python_version < "4.0" pyexasol[pandas]==0.25.2 ; python_version >= "3.9" and python_version < "4.0" pygments==2.17.2 ; python_version >= "3.9" and python_version < "4.0" pyinstrument==4.6.1 ; python_version >= "3.9" and python_version < "4.0" pyjwt==2.8.0 ; python_version >= "3.9" and python_version < "4.0" +pymongo==4.6.1 ; python_version >= "3.9" and python_version < "4.0" pymysql==1.1.0 ; python_version >= "3.9" and python_version < "4.0" pyodbc==5.0.1 ; python_version >= "3.9" and python_version < "4.0" pyopenssl==23.3.0 ; python_version >= "3.9" and python_version < "4.0" -pyparsing==3.1.1 ; python_version >= "3.10" and python_version < "3.13" +pyparsing==3.1.1 ; python_version >= "3.9" and python_version < "4.0" pyproj==3.6.1 ; python_version >= "3.9" and python_version < "4.0" pyproject-hooks==1.0.0 ; python_version >= "3.9" and python_version < "4.0" pyshp==2.3.1 ; python_version >= "3.10" and python_version < "3.13" @@ -201,6 +215,7 @@ pytest-mock==3.12.0 ; python_version >= "3.9" and python_version < "4.0" pytest-randomly==3.15.0 ; python_version >= "3.9" and python_version < "4.0" pytest-repeat==0.9.3 ; python_version >= "3.9" and python_version < "4.0" pytest-snapshot==0.9.0 ; python_version >= "3.9" and python_version < "4.0" +pytest-split==0.8.2 ; python_version >= "3.9" and python_version < "4.0" pytest-xdist==3.5.0 ; python_version >= "3.9" and python_version < "4.0" pytest==8.0.0 ; python_version >= "3.9" and python_version < "4.0" python-box==7.1.1 ; python_version >= "3.10" and python_version < "3.13"